[NO-ISSUE][IDX] Adding support for array-indexes.

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

details:
- Users can now create indexes on array-valued fields, of the general
form "CREATE INDEX ... ON Dataset (UNNEST arrayField SELECT atomicField)".
- Metadata tuples for indexes has been updated to support a list
of "complex search keys", as well as multiple types for one key.
In the presence of a complex search key, all keys (even simple SKs)
will be placed inside this new list.
- Queries involving membership, UNNESTs, quantification
(existential and partially supported universal) will utilize an
applicable array index. Users must set the compiler option "arrayindex"
to true to enable this.
- SIDX maintenance operations now allow the use of nested plans, which
the aforementioned maintenance op will operate on from the end of the
nested plan(s).
- Indexes are now classified by different types in metadata, with a new
"array" index type.

Change-Id: Id0e9eee940cc94819e169a74ed180387b7a3093b
Reviewed-on: https://asterix-gerrit.ics.uci.edu/c/asterixdb/+/7684
Integration-Tests: Jenkins <jenkins@fulliautomatix.ics.uci.edu>
Tested-by: Jenkins <jenkins@fulliautomatix.ics.uci.edu>
Reviewed-by: Dmitry Lychagin <dmitry.lychagin@couchbase.com>
diff --git a/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/algebra/operators/CommitOperator.java b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/algebra/operators/CommitOperator.java
index 6578c9c..135c3e3 100644
--- a/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/algebra/operators/CommitOperator.java
+++ b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/algebra/operators/CommitOperator.java
@@ -44,6 +44,10 @@
         this.isSink = isSink;
     }
 
+    public List<LogicalVariable> getPrimaryKeyLogicalVars() {
+        return this.primaryKeyLogicalVars;
+    }
+
     @Override
     public boolean isMap() {
         return false;
diff --git a/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/algebra/operators/physical/BTreeSearchPOperator.java b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/algebra/operators/physical/BTreeSearchPOperator.java
index 8c2c1df..ed8e9bf 100644
--- a/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/algebra/operators/physical/BTreeSearchPOperator.java
+++ b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/algebra/operators/physical/BTreeSearchPOperator.java
@@ -156,7 +156,7 @@
             return false;
         }
         Index searchIndex = ((DataSourceIndex) idx).getIndex();
-        int numberOfKeyFields = searchIndex.getKeyFieldNames().size();
+        int numberOfKeyFields = ((Index.ValueIndexDetails) searchIndex.getIndexDetails()).getKeyFieldNames().size();
 
         if (lowKeyVarList.size() != numberOfKeyFields || highKeyVarList.size() != numberOfKeyFields) {
             return false;
@@ -186,7 +186,8 @@
 
                 // If this is a composite primary index, then all of the keys should be provided.
                 Index searchIndex = ((DataSourceIndex) idx).getIndex();
-                int numberOfKeyFields = searchIndex.getKeyFieldNames().size();
+                int numberOfKeyFields =
+                        ((Index.ValueIndexDetails) searchIndex.getIndexDetails()).getKeyFieldNames().size();
 
                 if (numberOfKeyFields < 2
                         || (lowKeyVarList.size() == numberOfKeyFields && highKeyVarList.size() == numberOfKeyFields)) {
diff --git a/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/algebra/operators/physical/InvertedIndexPOperator.java b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/algebra/operators/physical/InvertedIndexPOperator.java
index fd664df..2d82f6f 100644
--- a/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/algebra/operators/physical/InvertedIndexPOperator.java
+++ b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/algebra/operators/physical/InvertedIndexPOperator.java
@@ -172,7 +172,7 @@
                 InvertedIndexAccessMethod.getBinaryTokenizerFactory(searchModifierType, searchKeyType, secondaryIndex);
         IFullTextConfigEvaluatorFactory fullTextConfigEvaluatorFactory =
                 FullTextUtil.fetchFilterAndCreateConfigEvaluator(metadataProvider, secondaryIndex.getDataverseName(),
-                        secondaryIndex.getFullTextConfigName());
+                        ((Index.TextIndexDetails) secondaryIndex.getIndexDetails()).getFullTextConfigName());
         IIndexDataflowHelperFactory dataflowHelperFactory = new IndexDataflowHelperFactory(
                 metadataProvider.getStorageComponentProvider().getStorageManager(), secondarySplitsAndConstraint.first);
 
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 ed35026..b7713c9 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
@@ -18,11 +18,15 @@
  */
 package org.apache.asterix.optimizer.rules;
 
+import java.util.ArrayDeque;
 import java.util.ArrayList;
 import java.util.Collections;
+import java.util.Deque;
 import java.util.HashMap;
+import java.util.HashSet;
 import java.util.List;
 import java.util.Map;
+import java.util.Set;
 import java.util.stream.Collectors;
 
 import org.apache.asterix.algebra.operators.CommitOperator;
@@ -38,6 +42,7 @@
 import org.apache.asterix.metadata.entities.Dataset;
 import org.apache.asterix.metadata.entities.Index;
 import org.apache.asterix.metadata.entities.InternalDatasetDetails;
+import org.apache.asterix.metadata.utils.ArrayIndexUtil;
 import org.apache.asterix.om.base.AInt32;
 import org.apache.asterix.om.base.AOrderedList;
 import org.apache.asterix.om.base.AString;
@@ -57,6 +62,7 @@
 import org.apache.hyracks.algebricks.common.utils.Pair;
 import org.apache.hyracks.algebricks.core.algebra.base.ILogicalExpression;
 import org.apache.hyracks.algebricks.core.algebra.base.ILogicalOperator;
+import org.apache.hyracks.algebricks.core.algebra.base.ILogicalPlan;
 import org.apache.hyracks.algebricks.core.algebra.base.IOptimizationContext;
 import org.apache.hyracks.algebricks.core.algebra.base.LogicalExpressionTag;
 import org.apache.hyracks.algebricks.core.algebra.base.LogicalOperatorTag;
@@ -65,17 +71,24 @@
 import org.apache.hyracks.algebricks.core.algebra.expressions.ConstantExpression;
 import org.apache.hyracks.algebricks.core.algebra.expressions.IVariableTypeEnvironment;
 import org.apache.hyracks.algebricks.core.algebra.expressions.ScalarFunctionCallExpression;
+import org.apache.hyracks.algebricks.core.algebra.expressions.UnnestingFunctionCallExpression;
 import org.apache.hyracks.algebricks.core.algebra.expressions.VariableReferenceExpression;
 import org.apache.hyracks.algebricks.core.algebra.functions.FunctionIdentifier;
 import org.apache.hyracks.algebricks.core.algebra.operators.logical.AbstractLogicalOperator;
 import org.apache.hyracks.algebricks.core.algebra.operators.logical.AbstractLogicalOperator.ExecutionMode;
 import org.apache.hyracks.algebricks.core.algebra.operators.logical.AssignOperator;
 import org.apache.hyracks.algebricks.core.algebra.operators.logical.DelegateOperator;
+import org.apache.hyracks.algebricks.core.algebra.operators.logical.DistinctOperator;
 import org.apache.hyracks.algebricks.core.algebra.operators.logical.IndexInsertDeleteUpsertOperator;
 import org.apache.hyracks.algebricks.core.algebra.operators.logical.InsertDeleteUpsertOperator;
 import org.apache.hyracks.algebricks.core.algebra.operators.logical.InsertDeleteUpsertOperator.Kind;
+import org.apache.hyracks.algebricks.core.algebra.operators.logical.NestedTupleSourceOperator;
+import org.apache.hyracks.algebricks.core.algebra.operators.logical.ProjectOperator;
 import org.apache.hyracks.algebricks.core.algebra.operators.logical.ReplicateOperator;
+import org.apache.hyracks.algebricks.core.algebra.operators.logical.SelectOperator;
 import org.apache.hyracks.algebricks.core.algebra.operators.logical.TokenizeOperator;
+import org.apache.hyracks.algebricks.core.algebra.operators.logical.UnnestOperator;
+import org.apache.hyracks.algebricks.core.algebra.plan.ALogicalPlanImpl;
 import org.apache.hyracks.algebricks.core.algebra.util.OperatorManipulationUtil;
 import org.apache.hyracks.algebricks.core.rewriter.base.IAlgebraicRewriteRule;
 import org.apache.hyracks.api.exceptions.SourceLocation;
@@ -87,6 +100,8 @@
  * assign --> insert-delete-upsert --> *(secondary indexes index-insert-delete-upsert) --> sink
  */
 public class IntroduceSecondaryIndexInsertDeleteRule implements IAlgebraicRewriteRule {
+    private IOptimizationContext context;
+    private SourceLocation sourceLoc;
 
     @Override
     public boolean rewritePre(Mutable<ILogicalOperator> opRef, IOptimizationContext context)
@@ -121,7 +136,8 @@
                 primaryIndexModificationOp.getAdditionalNonFilteringExpressions();
         LogicalVariable newRecordVar;
         LogicalVariable newMetaVar = null;
-        SourceLocation sourceLoc = primaryIndexModificationOp.getSourceLocation();
+        sourceLoc = primaryIndexModificationOp.getSourceLocation();
+        this.context = context;
 
         /**
          * inputOp is the assign operator which extracts primary keys from the input
@@ -129,14 +145,14 @@
          */
         AbstractLogicalOperator inputOp =
                 (AbstractLogicalOperator) primaryIndexModificationOp.getInputs().get(0).getValue();
-        newRecordVar = getRecordVar(context, inputOp, newRecordExpr, 0);
+        newRecordVar = getRecordVar(inputOp, newRecordExpr, 0);
         if (newMetaExprs != null && !newMetaExprs.isEmpty()) {
             if (newMetaExprs.size() > 1) {
                 throw new CompilationException(ErrorCode.COMPILATION_ERROR, sourceLoc,
                         "Number of meta records can't be more than 1. Number of meta records found = "
                                 + newMetaExprs.size());
             }
-            newMetaVar = getRecordVar(context, inputOp, newMetaExprs.get(0).getValue(), 1);
+            newMetaVar = getRecordVar(inputOp, newMetaExprs.get(0).getValue(), 1);
         }
 
         /*
@@ -177,9 +193,9 @@
             // for insert, primary key index is handled together when primary index
             indexes = indexes.stream().filter(index -> !index.isPrimaryKeyIndex()).collect(Collectors.toList());
         }
+
         // Set the top operator pointer to the primary IndexInsertDeleteOperator
         ILogicalOperator currentTop = primaryIndexModificationOp;
-        boolean hasSecondaryIndex = false;
 
         // Put an n-gram or a keyword index in the later stage of index-update,
         // since TokenizeOperator needs to be involved.
@@ -216,7 +232,7 @@
 
         // Replicate Operator is applied only when doing the bulk-load.
         ReplicateOperator replicateOp = null;
-        if (secondaryIndexTotalCnt > 1 && primaryIndexModificationOp.isBulkload()) {
+        if (secondaryIndexTotalCnt > 1 && isBulkload) {
             // Split the logical plan into "each secondary index update branch"
             // to replicate each <PK,OBJECT> pair.
             replicateOp = new ReplicateOperator(secondaryIndexTotalCnt);
@@ -247,8 +263,8 @@
                  * is solved
                  */
                 || primaryIndexModificationOp.getOperation() == Kind.DELETE) {
-            injectFieldAccessesForIndexes(context, dataset, indexes, fieldVarsForNewRecord, recType, metaType,
-                    newRecordVar, newMetaVar, primaryIndexModificationOp, false);
+            injectFieldAccessesForIndexes(dataset, indexes, fieldVarsForNewRecord, recType, metaType, newRecordVar,
+                    newMetaVar, primaryIndexModificationOp, false);
             if (replicateOp != null) {
                 context.computeAndSetTypeEnvironmentForOperator(replicateOp);
             }
@@ -260,38 +276,61 @@
          */) {
             List<LogicalVariable> beforeOpMetaVars = primaryIndexModificationOp.getBeforeOpAdditionalNonFilteringVars();
             LogicalVariable beforeOpMetaVar = beforeOpMetaVars == null ? null : beforeOpMetaVars.get(0);
-            currentTop = injectFieldAccessesForIndexes(context, dataset, indexes, fieldVarsForBeforeOperation, recType,
-                    metaType, primaryIndexModificationOp.getBeforeOpRecordVar(), beforeOpMetaVar, currentTop, true);
+            currentTop = injectFieldAccessesForIndexes(dataset, indexes, fieldVarsForBeforeOperation, recType, metaType,
+                    primaryIndexModificationOp.getBeforeOpRecordVar(), beforeOpMetaVar, currentTop, true);
         }
 
-        // Iterate each secondary index and applying Index Update operations.
-        // At first, op1 is the index insert op insertOp
+        // Add the appropriate SIDX maintenance operations.
         for (Index index : indexes) {
             if (!index.isSecondaryIndex()) {
                 continue;
             }
-            hasSecondaryIndex = true;
+
             // Get the secondary fields names and types
-            List<List<String>> secondaryKeyFields = index.getKeyFieldNames();
-            List<IAType> secondaryKeyTypes = index.getKeyFieldTypes();
+            List<List<String>> secondaryKeyFields = null;
+            List<IAType> secondaryKeyTypes = null;
+            List<Integer> secondaryKeySources = null;
+            switch (Index.IndexCategory.of(index.getIndexType())) {
+                case VALUE:
+                    Index.ValueIndexDetails valueIndexDetails = (Index.ValueIndexDetails) index.getIndexDetails();
+                    secondaryKeyFields = valueIndexDetails.getKeyFieldNames();
+                    secondaryKeyTypes = valueIndexDetails.getKeyFieldTypes();
+                    secondaryKeySources = valueIndexDetails.getKeyFieldSourceIndicators();
+                    break;
+                case TEXT:
+                    Index.TextIndexDetails textIndexDetails = (Index.TextIndexDetails) index.getIndexDetails();
+                    secondaryKeyFields = textIndexDetails.getKeyFieldNames();
+                    secondaryKeyTypes = textIndexDetails.getKeyFieldTypes();
+                    secondaryKeySources = textIndexDetails.getKeyFieldSourceIndicators();
+                    break;
+                case ARRAY:
+                    // These details are handled separately for array indexes.
+                    break;
+                default:
+                    continue;
+            }
+
+            // Set our key variables and expressions for non-array indexes. Our secondary keys for array indexes will
+            // always be an empty list.
             List<LogicalVariable> secondaryKeyVars = new ArrayList<>();
             List<Mutable<ILogicalExpression>> secondaryExpressions = new ArrayList<>();
             List<Mutable<ILogicalExpression>> beforeOpSecondaryExpressions = new ArrayList<>();
             ILogicalOperator replicateOutput;
-
-            for (int i = 0; i < secondaryKeyFields.size(); i++) {
-                IndexFieldId indexFieldId = new IndexFieldId(index.getKeyFieldSourceIndicators().get(i),
-                        secondaryKeyFields.get(i), secondaryKeyTypes.get(i).getTypeTag());
-                LogicalVariable skVar = fieldVarsForNewRecord.get(indexFieldId);
-                secondaryKeyVars.add(skVar);
-                VariableReferenceExpression skVarRef = new VariableReferenceExpression(skVar);
-                skVarRef.setSourceLocation(sourceLoc);
-                secondaryExpressions.add(new MutableObject<ILogicalExpression>(skVarRef));
-                if (primaryIndexModificationOp.getOperation() == Kind.UPSERT) {
-                    VariableReferenceExpression varRef =
-                            new VariableReferenceExpression(fieldVarsForBeforeOperation.get(indexFieldId));
-                    varRef.setSourceLocation(sourceLoc);
-                    beforeOpSecondaryExpressions.add(new MutableObject<ILogicalExpression>(varRef));
+            if (!index.getIndexType().equals(IndexType.ARRAY)) {
+                for (int i = 0; i < secondaryKeyFields.size(); i++) {
+                    IndexFieldId indexFieldId = new IndexFieldId(secondaryKeySources.get(i), secondaryKeyFields.get(i),
+                            secondaryKeyTypes.get(i).getTypeTag());
+                    LogicalVariable skVar = fieldVarsForNewRecord.get(indexFieldId);
+                    secondaryKeyVars.add(skVar);
+                    VariableReferenceExpression skVarRef = new VariableReferenceExpression(skVar);
+                    skVarRef.setSourceLocation(sourceLoc);
+                    secondaryExpressions.add(new MutableObject<ILogicalExpression>(skVarRef));
+                    if (primaryIndexModificationOp.getOperation() == Kind.UPSERT) {
+                        VariableReferenceExpression varRef =
+                                new VariableReferenceExpression(fieldVarsForBeforeOperation.get(indexFieldId));
+                        varRef.setSourceLocation(sourceLoc);
+                        beforeOpSecondaryExpressions.add(new MutableObject<ILogicalExpression>(varRef));
+                    }
                 }
             }
 
@@ -301,12 +340,13 @@
                 Mutable<ILogicalExpression> filterExpression =
                         (primaryIndexModificationOp.getOperation() == Kind.UPSERT) ? null
                                 : createFilterExpression(secondaryKeyVars, context.getOutputTypeEnvironment(currentTop),
-                                        index.isOverridingKeyFieldTypes(), sourceLoc);
+                                        index.getIndexDetails().isOverridingKeyFieldTypes());
                 DataSourceIndex dataSourceIndex = new DataSourceIndex(index, dataverseName, datasetName, mp);
 
                 // Introduce the TokenizeOperator only when doing bulk-load,
                 // and index type is keyword or n-gram.
-                if (index.getIndexType() != IndexType.BTREE && primaryIndexModificationOp.isBulkload()) {
+                if (index.getIndexType() != IndexType.BTREE && index.getIndexType() != IndexType.ARRAY
+                        && primaryIndexModificationOp.isBulkload()) {
                     // Note: Bulk load case, we don't need to take care of it for upsert operation
                     // Check whether the index is length-partitioned or not.
                     // If partitioned, [input variables to TokenizeOperator,
@@ -330,8 +370,8 @@
 
                     // Check the field type of the secondary key.
                     IAType secondaryKeyType;
-                    Pair<IAType, Boolean> keyPairType = Index.getNonNullableOpenFieldType(
-                            index.getKeyFieldTypes().get(0), secondaryKeyFields.get(0), recType);
+                    Pair<IAType, Boolean> keyPairType = Index.getNonNullableOpenFieldType(secondaryKeyTypes.get(0),
+                            secondaryKeyFields.get(0), recType);
                     secondaryKeyType = keyPairType.first;
 
                     List<Object> varTypes = new ArrayList<>();
@@ -399,11 +439,82 @@
                         }
                     }
                     indexUpdate.getInputs().add(new MutableObject<ILogicalOperator>(currentTop));
+
+                    // For array indexes we have no secondary keys to reference. We must add separate branches to
+                    // first extract our keys.
+                    if (index.getIndexType() == IndexType.ARRAY && !isBulkload) {
+                        NestedTupleSourceOperator unnestSourceOp =
+                                new NestedTupleSourceOperator(new MutableObject<>(indexUpdate));
+                        unnestSourceOp.setSourceLocation(sourceLoc);
+                        context.computeAndSetTypeEnvironmentForOperator(unnestSourceOp);
+                        UnnestBranchCreator unnestSIDXBranch = buildUnnestBranch(unnestSourceOp, index, newRecordVar,
+                                newMetaVar, recType, metaType, dataset.hasMetaPart());
+                        unnestSIDXBranch.applyProjectDistinct();
+
+                        // If there exists a filter expression, add it to the top of our nested plan.
+                        filterExpression = (primaryIndexModificationOp.getOperation() == Kind.UPSERT) ? null
+                                : createFilterExpression(unnestSIDXBranch.lastFieldVars,
+                                        context.getOutputTypeEnvironment(unnestSIDXBranch.currentTop),
+                                        index.getIndexDetails().isOverridingKeyFieldTypes());
+                        if (filterExpression != null) {
+                            unnestSIDXBranch.applyFilteringExpression(filterExpression);
+                        }
+
+                        // Finalize our nested plan.
+                        ILogicalPlan unnestPlan = unnestSIDXBranch.buildBranch();
+                        indexUpdate.getNestedPlans().add(unnestPlan);
+
+                        // If we have an UPSERT, then create and add a branch to extract our old keys as well.
+                        if (primaryIndexModificationOp.getOperation() == Kind.UPSERT) {
+                            NestedTupleSourceOperator unnestBeforeSourceOp =
+                                    new NestedTupleSourceOperator(new MutableObject<>(indexUpdate));
+                            unnestBeforeSourceOp.setSourceLocation(sourceLoc);
+                            context.computeAndSetTypeEnvironmentForOperator(unnestBeforeSourceOp);
+
+                            List<LogicalVariable> beforeOpMetaVars =
+                                    primaryIndexModificationOp.getBeforeOpAdditionalNonFilteringVars();
+                            LogicalVariable beforeOpMetaVar = beforeOpMetaVars == null ? null : beforeOpMetaVars.get(0);
+                            UnnestBranchCreator unnestBeforeSIDXBranch = buildUnnestBranch(unnestBeforeSourceOp, index,
+                                    primaryIndexModificationOp.getBeforeOpRecordVar(), beforeOpMetaVar, recType,
+                                    metaType, dataset.hasMetaPart());
+                            unnestBeforeSIDXBranch.applyProjectDistinct();
+                            indexUpdate.getNestedPlans().add(unnestBeforeSIDXBranch.buildBranch());
+                        }
+                    } else if (index.getIndexType() == IndexType.ARRAY && isBulkload) {
+                        // If we have a bulk load, we must sort the entire input by <SK, PK>. Do not use any
+                        // nested plans here.
+                        UnnestBranchCreator unnestSIDXBranch = buildUnnestBranch(currentTop, index, newRecordVar,
+                                newMetaVar, recType, metaType, dataset.hasMetaPart());
+                        unnestSIDXBranch.applyProjectDistinct(primaryIndexModificationOp.getPrimaryKeyExpressions(),
+                                primaryIndexModificationOp.getAdditionalFilteringExpressions());
+                        indexUpdate.getInputs().clear();
+                        introduceNewOp(unnestSIDXBranch.currentTop, indexUpdate, true);
+
+                        // Update the secondary expressions of our index.
+                        secondaryExpressions = new ArrayList<>();
+                        for (LogicalVariable var : unnestSIDXBranch.lastFieldVars) {
+                            secondaryExpressions.add(new MutableObject<>(new VariableReferenceExpression(var)));
+                        }
+                        indexUpdate.setSecondaryKeyExprs(secondaryExpressions);
+
+                        // Update the filter expression to include these new keys.
+                        filterExpression = createFilterExpression(unnestSIDXBranch.lastFieldVars,
+                                context.getOutputTypeEnvironment(unnestSIDXBranch.currentTop),
+                                index.getIndexDetails().isOverridingKeyFieldTypes());
+                        indexUpdate.setFilterExpression(filterExpression);
+
+                        if (replicateOp != null) {
+                            // If we have a replicate, then update the replicate operator to include this branch.
+                            replicateOp.getOutputs().add(new MutableObject<>(unnestSIDXBranch.currentBottom));
+                            op0.getInputs().add(new MutableObject<ILogicalOperator>(indexUpdate));
+                            continue;
+                        }
+                    }
                 }
             } else {
                 // Get type, dimensions and number of keys
-                Pair<IAType, Boolean> keyPairType = Index.getNonNullableOpenFieldType(index.getKeyFieldTypes().get(0),
-                        secondaryKeyFields.get(0), recType);
+                Pair<IAType, Boolean> keyPairType =
+                        Index.getNonNullableOpenFieldType(secondaryKeyTypes.get(0), secondaryKeyFields.get(0), recType);
                 IAType spatialType = keyPairType.first;
                 boolean isPointMBR =
                         spatialType.getTypeTag() == ATypeTag.POINT || spatialType.getTypeTag() == ATypeTag.POINT3D;
@@ -485,7 +596,7 @@
                     // nullable.
                     boolean forceFilter = keyPairType.second;
                     filterExpression = createFilterExpression(keyVarList,
-                            context.getOutputTypeEnvironment(assignCoordinates), forceFilter, sourceLoc);
+                            context.getOutputTypeEnvironment(assignCoordinates), forceFilter);
                 }
                 DataSourceIndex dataSourceIndex = new DataSourceIndex(index, dataverseName, datasetName, mp);
                 indexUpdate = new IndexInsertDeleteUpsertOperator(dataSourceIndex,
@@ -532,7 +643,7 @@
                  * "blocking" sort operator since tuples are already sorted. We mark the materialization flag for that
                  * branch to make it blocking. Without "blocking", the activity cluster graph would be messed up
                  */
-                if (index.getKeyFieldNames().isEmpty() && index.getIndexType() == IndexType.BTREE) {
+                if (index.isPrimaryKeyIndex()) {
                     int positionOfSecondaryPrimaryIndex = replicateOp.getOutputs().size() - 1;
                     replicateOp.getOutputMaterializationFlags()[positionOfSecondaryPrimaryIndex] = true;
                 }
@@ -541,10 +652,6 @@
                 // For bulk load, we connect all fanned out insert operator to a single SINK operator
                 op0.getInputs().add(new MutableObject<ILogicalOperator>(indexUpdate));
             }
-
-        }
-        if (!hasSecondaryIndex) {
-            return false;
         }
 
         if (!primaryIndexModificationOp.isBulkload()) {
@@ -557,8 +664,103 @@
         return true;
     }
 
-    private LogicalVariable getRecordVar(IOptimizationContext context, AbstractLogicalOperator inputOp,
-            ILogicalExpression recordExpr, int expectedRecordIndex) throws AlgebricksException {
+    private UnnestBranchCreator buildUnnestBranch(ILogicalOperator unnestSourceOp, Index index,
+            LogicalVariable recordVar, LogicalVariable metaVar, ARecordType recType, ARecordType metaType,
+            boolean hasMetaPart) throws AlgebricksException {
+        Index.ArrayIndexDetails arrayIndexDetails = (Index.ArrayIndexDetails) index.getIndexDetails();
+
+        // First, locate a field having the required UNNEST path. Queue this first, and all other keys will follow.
+        Deque<Integer> keyPositionQueue = new ArrayDeque<>();
+        for (int i = 0; i < arrayIndexDetails.getElementList().size(); i++) {
+            Index.ArrayIndexElement e = arrayIndexDetails.getElementList().get(i);
+            if (e.getUnnestList().isEmpty()) {
+                keyPositionQueue.addLast(i);
+            } else {
+                keyPositionQueue.addFirst(i);
+            }
+        }
+
+        // Get the record variable associated with our record path.
+        Index.ArrayIndexElement workingElement = arrayIndexDetails.getElementList().get(keyPositionQueue.getFirst());
+        int sourceIndicatorForBaseRecord = workingElement.getSourceIndicator();
+        LogicalVariable sourceVarForBaseRecord = hasMetaPart
+                ? ((sourceIndicatorForBaseRecord == Index.RECORD_INDICATOR) ? recordVar : metaVar) : recordVar;
+        VariableReferenceExpression baseRecordVarRef = new VariableReferenceExpression(sourceVarForBaseRecord);
+        baseRecordVarRef.setSourceLocation(sourceLoc);
+        UnnestBranchCreator branchCreator = new UnnestBranchCreator(baseRecordVarRef, unnestSourceOp);
+
+        int initialKeyPositionQueueSize = keyPositionQueue.size();
+        Set<LogicalVariable> secondaryKeyVars = new HashSet<>();
+        for (int i = 0; i < initialKeyPositionQueueSize; i++) {
+
+            // Poll from our queue, and get a key position.
+            int workingKeyPos = keyPositionQueue.pollFirst();
+            workingElement = arrayIndexDetails.getElementList().get(workingKeyPos);
+            int sourceIndicator = workingElement.getSourceIndicator();
+            ARecordType recordType =
+                    hasMetaPart ? ((sourceIndicator == Index.RECORD_INDICATOR) ? recType : metaType) : recType;
+
+            boolean isOpenOrNestedField;
+            if (workingElement.getUnnestList().isEmpty()) {
+                // We have an atomic element (i.e. we have a composite array index).
+                List<String> atomicFieldName = workingElement.getProjectList().get(0);
+                isOpenOrNestedField =
+                        (atomicFieldName.size() != 1) || !recordType.isClosedField(atomicFieldName.get(0));
+
+                // The UNNEST path has already been created (we queued this first), so we look at the current top.
+                LogicalVariable newVar = context.newVar();
+                VariableReferenceExpression varRef = new VariableReferenceExpression(sourceVarForBaseRecord);
+                varRef.setSourceLocation(sourceLoc);
+                AbstractFunctionCallExpression newVarRef = (isOpenOrNestedField)
+                        ? getFieldAccessFunction(new MutableObject<>(varRef),
+                                recordType.getFieldIndex(atomicFieldName.get(0)), atomicFieldName)
+                        : getFieldAccessFunction(new MutableObject<>(varRef), -1, atomicFieldName);
+
+                AssignOperator newAssignOp = new AssignOperator(newVar, new MutableObject<>(newVarRef));
+                newAssignOp.setSourceLocation(sourceLoc);
+                branchCreator.currentTop = introduceNewOp(branchCreator.currentTop, newAssignOp, true);
+                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.
+                List<String> flatFirstFieldName = ArrayIndexUtil.getFlattenedKeyFieldNames(
+                        workingElement.getUnnestList(), workingElement.getProjectList().get(0));
+                List<Integer> firstArrayIndicator = ArrayIndexUtil
+                        .getArrayDepthIndicator(workingElement.getUnnestList(), workingElement.getProjectList().get(0));
+                ArrayIndexUtil.walkArrayPath((isOpenOrNestedField) ? null : recordType, flatFirstFieldName,
+                        firstArrayIndicator, branchCreator);
+
+                // For all other elements in the PROJECT list, add an assign.
+                for (int j = 1; j < workingElement.getProjectList().size(); j++) {
+                    LogicalVariable newVar = context.newVar();
+                    AbstractFunctionCallExpression newVarRef =
+                            getFieldAccessFunction(new MutableObject<>(branchCreator.lastRecordVarRef), -1,
+                                    workingElement.getProjectList().get(j));
+
+                    AssignOperator newAssignOp = new AssignOperator(newVar, new MutableObject<>(newVarRef));
+                    newAssignOp.setSourceLocation(sourceLoc);
+                    branchCreator.currentTop = introduceNewOp(branchCreator.currentTop, newAssignOp, true);
+                    secondaryKeyVars.add(newVar);
+                }
+            }
+
+            branchCreator.lowerIsFirstWalkFlag();
+            secondaryKeyVars.addAll(branchCreator.lastFieldVars);
+        }
+
+        // Update the variables we are to use for the head operators.
+        branchCreator.lastFieldVars.clear();
+        branchCreator.lastFieldVars.addAll(secondaryKeyVars);
+
+        return branchCreator;
+    }
+
+    private LogicalVariable getRecordVar(AbstractLogicalOperator inputOp, ILogicalExpression recordExpr,
+            int expectedRecordIndex) throws AlgebricksException {
         if (exprIsRecord(context.getOutputTypeEnvironment(inputOp), recordExpr)) {
             return ((VariableReferenceExpression) recordExpr).getVariableReference();
         } else {
@@ -598,21 +800,39 @@
         return false;
     }
 
-    private ILogicalOperator injectFieldAccessesForIndexes(IOptimizationContext context, Dataset dataset,
-            List<Index> indexes, Map<IndexFieldId, LogicalVariable> fieldAccessVars, ARecordType recType,
-            ARecordType metaType, LogicalVariable recordVar, LogicalVariable metaVar, ILogicalOperator currentTop,
-            boolean afterOp) throws AlgebricksException {
+    private ILogicalOperator injectFieldAccessesForIndexes(Dataset dataset, List<Index> indexes,
+            Map<IndexFieldId, LogicalVariable> fieldAccessVars, ARecordType recType, ARecordType metaType,
+            LogicalVariable recordVar, LogicalVariable metaVar, ILogicalOperator currentTop, boolean afterOp)
+            throws AlgebricksException {
         List<LogicalVariable> vars = new ArrayList<>();
         List<Mutable<ILogicalExpression>> exprs = new ArrayList<>();
         SourceLocation sourceLoc = currentTop.getSourceLocation();
         for (Index index : indexes) {
-            if (index.isPrimaryIndex()) {
+            if (index.isPrimaryIndex() || index.getIndexType() == IndexType.ARRAY) {
+                // Array indexes require UNNESTs, which must be handled after the PIDX op.
                 continue;
             }
-            List<IAType> skTypes = index.getKeyFieldTypes();
-            List<List<String>> skNames = index.getKeyFieldNames();
-            List<Integer> indicators = index.getKeyFieldSourceIndicators();
-            for (int i = 0; i < index.getKeyFieldNames().size(); i++) {
+            List<List<String>> skNames;
+            List<IAType> skTypes;
+            List<Integer> indicators;
+            switch (Index.IndexCategory.of(index.getIndexType())) {
+                case VALUE:
+                    Index.ValueIndexDetails valueIndexDetails = (Index.ValueIndexDetails) index.getIndexDetails();
+                    skNames = valueIndexDetails.getKeyFieldNames();
+                    skTypes = valueIndexDetails.getKeyFieldTypes();
+                    indicators = valueIndexDetails.getKeyFieldSourceIndicators();
+                    break;
+                case TEXT:
+                    Index.TextIndexDetails textIndexDetails = (Index.TextIndexDetails) index.getIndexDetails();
+                    skNames = textIndexDetails.getKeyFieldNames();
+                    skTypes = textIndexDetails.getKeyFieldTypes();
+                    indicators = textIndexDetails.getKeyFieldSourceIndicators();
+                    break;
+                default:
+                    throw new CompilationException(ErrorCode.COMPILATION_UNKNOWN_INDEX_TYPE,
+                            String.valueOf(index.getIndexType()));
+            }
+            for (int i = 0; i < skNames.size(); i++) {
                 IndexFieldId indexFieldId =
                         new IndexFieldId(indicators.get(i), skNames.get(i), skTypes.get(i).getTypeTag());
                 if (fieldAccessVars.containsKey(indexFieldId)) {
@@ -634,8 +854,8 @@
                     // make handling of records with incorrect value type for this field easier and cleaner
                     context.addNotToBeInlinedVar(fieldVar);
                     // create field access
-                    AbstractFunctionCallExpression fieldAccessFunc = getOpenOrNestedFieldAccessFunction(
-                            new MutableObject<>(varRef), indexFieldId.fieldName, sourceLoc);
+                    AbstractFunctionCallExpression fieldAccessFunc =
+                            getFieldAccessFunction(new MutableObject<>(varRef), -1, indexFieldId.fieldName);
                     // create cast
                     theFieldAccessFunc = new ScalarFunctionCallExpression(FunctionUtil.getFunctionInfo(
                             index.isEnforced() ? BuiltinFunctions.CAST_TYPE : BuiltinFunctions.CAST_TYPE_LAX));
@@ -648,24 +868,25 @@
                     int pos = indexFieldId.fieldName.size() > 1 ? -1
                             : sourceType.getFieldIndex(indexFieldId.fieldName.get(0));
                     // Field not found --> This is either an open field or a nested field. it can't be accessed by index
-                    theFieldAccessFunc = (pos == -1)
-                            ? getOpenOrNestedFieldAccessFunction(new MutableObject<>(varRef), indexFieldId.fieldName,
-                                    sourceLoc)
-                            : getClosedFieldAccessFunction(new MutableObject<>(varRef), pos, sourceLoc);
+                    theFieldAccessFunc =
+                            getFieldAccessFunction(new MutableObject<>(varRef), pos, indexFieldId.fieldName);
                 }
                 vars.add(fieldVar);
                 exprs.add(new MutableObject<ILogicalExpression>(theFieldAccessFunc));
                 fieldAccessVars.put(indexFieldId, fieldVar);
             }
         }
-        // AssignOperator assigns secondary keys to their vars
-        AssignOperator castedFieldAssignOperator = new AssignOperator(vars, exprs);
-        castedFieldAssignOperator.setSourceLocation(sourceLoc);
-        return introduceNewOp(context, currentTop, castedFieldAssignOperator, afterOp);
+        if (!vars.isEmpty()) {
+            // AssignOperator assigns secondary keys to their vars
+            AssignOperator castedFieldAssignOperator = new AssignOperator(vars, exprs);
+            castedFieldAssignOperator.setSourceLocation(sourceLoc);
+            return introduceNewOp(currentTop, castedFieldAssignOperator, afterOp);
+        }
+        return currentTop;
     }
 
-    private static ILogicalOperator introduceNewOp(IOptimizationContext context, ILogicalOperator currentTopOp,
-            ILogicalOperator newOp, boolean afterOp) throws AlgebricksException {
+    private ILogicalOperator introduceNewOp(ILogicalOperator currentTopOp, ILogicalOperator newOp, boolean afterOp)
+            throws AlgebricksException {
         if (afterOp) {
             newOp.getInputs().add(new MutableObject<>(currentTopOp));
             context.computeAndSetTypeEnvironmentForOperator(newOp);
@@ -680,34 +901,34 @@
         }
     }
 
-    private static AbstractFunctionCallExpression getClosedFieldAccessFunction(Mutable<ILogicalExpression> varRef,
-            int position, SourceLocation sourceLoc) {
-        Mutable<ILogicalExpression> indexRef =
-                new MutableObject<>(new ConstantExpression(new AsterixConstantValue(new AInt32(position))));
-        ScalarFunctionCallExpression fnExpr = new ScalarFunctionCallExpression(
-                FunctionUtil.getFunctionInfo(BuiltinFunctions.FIELD_ACCESS_BY_INDEX), varRef, indexRef);
-        fnExpr.setSourceLocation(sourceLoc);
-        return fnExpr;
-    }
+    private AbstractFunctionCallExpression getFieldAccessFunction(Mutable<ILogicalExpression> varRef, int fieldPos,
+            List<String> fieldName) {
+        if (fieldName.size() == 1 && fieldPos != -1) {
+            Mutable<ILogicalExpression> indexRef =
+                    new MutableObject<>(new ConstantExpression(new AsterixConstantValue(new AInt32(fieldPos))));
+            ScalarFunctionCallExpression fnExpr = new ScalarFunctionCallExpression(
+                    FunctionUtil.getFunctionInfo(BuiltinFunctions.FIELD_ACCESS_BY_INDEX), varRef, indexRef);
+            fnExpr.setSourceLocation(sourceLoc);
+            return fnExpr;
 
-    private static AbstractFunctionCallExpression getOpenOrNestedFieldAccessFunction(Mutable<ILogicalExpression> varRef,
-            List<String> fields, SourceLocation sourceLoc) {
-        ScalarFunctionCallExpression func;
-        if (fields.size() > 1) {
-            IAObject fieldList = stringListToAOrderedList(fields);
-            Mutable<ILogicalExpression> fieldRef = constantToMutableLogicalExpression(fieldList);
-            // Create an expression for the nested case
-            func = new ScalarFunctionCallExpression(FunctionUtil.getFunctionInfo(BuiltinFunctions.FIELD_ACCESS_NESTED),
-                    varRef, fieldRef);
         } else {
-            IAObject fieldList = new AString(fields.get(0));
-            Mutable<ILogicalExpression> fieldRef = constantToMutableLogicalExpression(fieldList);
-            // Create an expression for the open field case (By name)
-            func = new ScalarFunctionCallExpression(FunctionUtil.getFunctionInfo(BuiltinFunctions.FIELD_ACCESS_BY_NAME),
-                    varRef, fieldRef);
+            ScalarFunctionCallExpression func;
+            if (fieldName.size() > 1) {
+                IAObject fieldList = stringListToAOrderedList(fieldName);
+                Mutable<ILogicalExpression> fieldRef = constantToMutableLogicalExpression(fieldList);
+                // Create an expression for the nested case
+                func = new ScalarFunctionCallExpression(
+                        FunctionUtil.getFunctionInfo(BuiltinFunctions.FIELD_ACCESS_NESTED), varRef, fieldRef);
+            } else {
+                IAObject fieldList = new AString(fieldName.get(0));
+                Mutable<ILogicalExpression> fieldRef = constantToMutableLogicalExpression(fieldList);
+                // Create an expression for the open field case (By name)
+                func = new ScalarFunctionCallExpression(
+                        FunctionUtil.getFunctionInfo(BuiltinFunctions.FIELD_ACCESS_BY_NAME), varRef, fieldRef);
+            }
+            func.setSourceLocation(sourceLoc);
+            return func;
         }
-        func.setSourceLocation(sourceLoc);
-        return func;
     }
 
     private static AOrderedList stringListToAOrderedList(List<String> fields) {
@@ -723,8 +944,7 @@
     }
 
     private Mutable<ILogicalExpression> createFilterExpression(List<LogicalVariable> secondaryKeyVars,
-            IVariableTypeEnvironment typeEnv, boolean forceFilter, SourceLocation sourceLoc)
-            throws AlgebricksException {
+            IVariableTypeEnvironment typeEnv, boolean forceFilter) throws AlgebricksException {
         List<Mutable<ILogicalExpression>> filterExpressions = new ArrayList<>();
         // Add 'is not null' to all nullable secondary index keys as a filtering
         // condition.
@@ -762,6 +982,137 @@
         return filterExpression;
     }
 
+    /**
+     * Builds the nested plan required for array index maintenance.
+     */
+    private class UnnestBranchCreator implements ArrayIndexUtil.TypeTrackerCommandExecutor {
+        private final List<LogicalVariable> lastFieldVars;
+        private VariableReferenceExpression lastRecordVarRef;
+        private ILogicalOperator currentTop, currentBottom;
+        private boolean isFirstWalk = true;
+
+        public UnnestBranchCreator(VariableReferenceExpression recordVarRef, ILogicalOperator sourceOperator) {
+            this.lastRecordVarRef = recordVarRef;
+            this.currentTop = sourceOperator;
+            this.lastFieldVars = new ArrayList<>();
+        }
+
+        public ILogicalPlan buildBranch() {
+            return new ALogicalPlanImpl(new MutableObject<>(currentTop));
+        }
+
+        public void lowerIsFirstWalkFlag() {
+            isFirstWalk = false;
+        }
+
+        @SafeVarargs
+        public final void applyProjectDistinct(List<Mutable<ILogicalExpression>>... auxiliaryExpressions)
+                throws AlgebricksException {
+            // Apply the following: PROJECT(SK, AK) --> (ORDER (SK, AK)) implicitly --> DISTINCT (SK, AK) .
+            List<LogicalVariable> projectVars = new ArrayList<>(this.lastFieldVars);
+            List<Mutable<ILogicalExpression>> distinctVarRefs =
+                    OperatorManipulationUtil.createVariableReferences(projectVars, sourceLoc);
+
+            // If we have any additional expressions to be added to our index, append them here.
+            if (auxiliaryExpressions.length > 0) {
+                for (List<Mutable<ILogicalExpression>> exprList : auxiliaryExpressions) {
+                    if (exprList != null) {
+                        // Sanity check: ensure that we only have variable references.
+                        if (exprList.stream().anyMatch(
+                                e -> !e.getValue().getExpressionTag().equals(LogicalExpressionTag.VARIABLE))) {
+                            throw new CompilationException(ErrorCode.COMPILATION_ILLEGAL_STATE, sourceLoc,
+                                    "Given auxiliary expression list contains non-variable reference expressions. We"
+                                            + " cannot apply DISTINCT to this expression at this stage.");
+                        }
+
+                        distinctVarRefs.addAll(OperatorManipulationUtil.cloneExpressions(exprList));
+                        for (Mutable<ILogicalExpression> e : OperatorManipulationUtil.cloneExpressions(exprList)) {
+                            projectVars.add(((VariableReferenceExpression) e.getValue()).getVariableReference());
+                        }
+                    }
+                }
+            }
+
+            ProjectOperator projectOperator = new ProjectOperator(projectVars);
+            projectOperator.setSourceLocation(sourceLoc);
+            this.currentTop = introduceNewOp(currentTop, projectOperator, true);
+            DistinctOperator distinctOperator = new DistinctOperator(distinctVarRefs);
+            distinctOperator.setSourceLocation(sourceLoc);
+            this.currentTop = introduceNewOp(currentTop, distinctOperator, true);
+        }
+
+        public void applyFilteringExpression(Mutable<ILogicalExpression> filterExpression) throws AlgebricksException {
+            SelectOperator selectOperator = new SelectOperator(filterExpression, false, null);
+            selectOperator.setSourceLocation(sourceLoc);
+            this.currentTop = introduceNewOp(currentTop, selectOperator, true);
+        }
+
+        @Override
+        public void executeActionOnEachArrayStep(ARecordType startingStepRecordType, IAType workingType,
+                List<String> fieldName, boolean isFirstArrayStep, boolean isFirstUnnestInStep,
+                boolean isLastUnnestInIntermediateStep) throws AlgebricksException {
+            if (!isFirstWalk) {
+                // We have already built the UNNEST path, do not build again.
+                return;
+            }
+
+            ILogicalExpression accessToUnnestVar;
+            if (isFirstUnnestInStep) {
+                // This is the first UNNEST step. Get the field we want to UNNEST from our record.
+                accessToUnnestVar = (startingStepRecordType != null)
+                        ? getFieldAccessFunction(new MutableObject<>(lastRecordVarRef),
+                                startingStepRecordType.getFieldIndex(fieldName.get(0)), fieldName)
+                        : getFieldAccessFunction(new MutableObject<>(lastRecordVarRef), -1, fieldName);
+            } else {
+                // This is the second+ UNNEST step. Refer back to the previously unnested variable.
+                accessToUnnestVar = new VariableReferenceExpression(this.lastFieldVars.get(0));
+                this.lastFieldVars.clear();
+            }
+            UnnestingFunctionCallExpression scanCollection = new UnnestingFunctionCallExpression(
+                    BuiltinFunctions.getBuiltinFunctionInfo(BuiltinFunctions.SCAN_COLLECTION),
+                    Collections.singletonList(new MutableObject<>(accessToUnnestVar)));
+            scanCollection.setReturnsUniqueValues(false);
+            scanCollection.setSourceLocation(sourceLoc);
+            LogicalVariable unnestVar = context.newVar();
+            this.lastFieldVars.add(unnestVar);
+
+            UnnestOperator unnestOp = new UnnestOperator(unnestVar, new MutableObject<>(scanCollection));
+            unnestOp.setSourceLocation(sourceLoc);
+            this.currentTop = introduceNewOp(currentTop, unnestOp, true);
+            if (isFirstArrayStep) {
+                this.currentBottom = unnestOp;
+            }
+
+            if (isLastUnnestInIntermediateStep) {
+                // This is the last UNNEST before the next array step. Update our record variable.
+                this.lastRecordVarRef = new VariableReferenceExpression(unnestVar);
+                this.lastRecordVarRef.setSourceLocation(sourceLoc);
+                this.lastFieldVars.clear();
+            }
+        }
+
+        @Override
+        public void executeActionOnFinalArrayStep(ARecordType startingStepRecordType, List<String> fieldName,
+                boolean isNonArrayStep, boolean requiresOnlyOneUnnest) throws AlgebricksException {
+            // If the final value is nested inside a record, add an additional ASSIGN.
+            if (!isNonArrayStep) {
+                return;
+            }
+
+            // Create the function to access our final field.
+            AbstractFunctionCallExpression accessToFinalVar = (startingStepRecordType != null)
+                    ? getFieldAccessFunction(new MutableObject<>(lastRecordVarRef),
+                            startingStepRecordType.getFieldIndex(fieldName.get(0)), fieldName)
+                    : getFieldAccessFunction(new MutableObject<>(lastRecordVarRef), -1, fieldName);
+
+            LogicalVariable finalVar = context.newVar();
+            this.lastFieldVars.add(finalVar);
+            AssignOperator assignOperator = new AssignOperator(finalVar, new MutableObject<>(accessToFinalVar));
+            assignOperator.setSourceLocation(sourceLoc);
+            this.currentTop = introduceNewOp(currentTop, assignOperator, true);
+        }
+    }
+
     private final class IndexFieldId {
         private final int indicator;
         private final List<String> fieldName;
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 1946f82..76b57ef 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
@@ -19,9 +19,7 @@
 package org.apache.asterix.optimizer.rules.am;
 
 import java.util.ArrayList;
-import java.util.Arrays;
 import java.util.Collection;
-import java.util.Collections;
 import java.util.HashMap;
 import java.util.HashSet;
 import java.util.Iterator;
@@ -38,20 +36,15 @@
 import org.apache.asterix.metadata.declared.MetadataProvider;
 import org.apache.asterix.metadata.entities.Dataset;
 import org.apache.asterix.metadata.entities.Index;
+import org.apache.asterix.metadata.utils.ArrayIndexUtil;
 import org.apache.asterix.metadata.utils.DatasetUtil;
 import org.apache.asterix.metadata.utils.MetadataUtil;
-import org.apache.asterix.om.base.AOrderedList;
-import org.apache.asterix.om.base.AString;
-import org.apache.asterix.om.constants.AsterixConstantValue;
 import org.apache.asterix.om.functions.BuiltinFunctions;
-import org.apache.asterix.om.typecomputer.impl.TypeComputeUtils;
-import org.apache.asterix.om.types.ARecordType;
 import org.apache.asterix.om.types.ATypeTag;
 import org.apache.asterix.om.types.AbstractCollectionType;
 import org.apache.asterix.om.types.BuiltinType;
 import org.apache.asterix.om.types.IAType;
 import org.apache.asterix.om.types.hierachy.ATypeHierarchy;
-import org.apache.asterix.om.utils.ConstantExpressionUtil;
 import org.apache.asterix.optimizer.base.AnalysisUtil;
 import org.apache.asterix.optimizer.rules.am.OptimizableOperatorSubTree.DataSourceType;
 import org.apache.asterix.optimizer.rules.util.FullTextUtil;
@@ -59,6 +52,7 @@
 import org.apache.commons.lang3.mutable.MutableInt;
 import org.apache.hyracks.algebricks.common.exceptions.AlgebricksException;
 import org.apache.hyracks.algebricks.common.utils.Pair;
+import org.apache.hyracks.algebricks.common.utils.Triple;
 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.IOptimizationContext;
@@ -66,8 +60,6 @@
 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.AbstractFunctionCallExpression;
-import org.apache.hyracks.algebricks.core.algebra.expressions.AbstractLogicalExpression;
-import org.apache.hyracks.algebricks.core.algebra.expressions.ConstantExpression;
 import org.apache.hyracks.algebricks.core.algebra.expressions.IVariableTypeEnvironment;
 import org.apache.hyracks.algebricks.core.algebra.expressions.VariableReferenceExpression;
 import org.apache.hyracks.algebricks.core.algebra.functions.AlgebricksBuiltinFunctions;
@@ -83,7 +75,6 @@
 import org.apache.hyracks.algebricks.core.rewriter.base.IAlgebraicRewriteRule;
 
 import com.google.common.base.Strings;
-import com.google.common.collect.ImmutableSet;
 
 /**
  * Class that embodies the commonalities between rewrite rules for access
@@ -93,15 +84,6 @@
 
     protected MetadataProvider metadataProvider;
 
-    // Function Identifier sets that retain the original field variable through each function's arguments
-    private final ImmutableSet<FunctionIdentifier> funcIDSetThatRetainFieldName =
-            ImmutableSet.of(BuiltinFunctions.WORD_TOKENS, BuiltinFunctions.GRAM_TOKENS, BuiltinFunctions.SUBSTRING,
-                    BuiltinFunctions.SUBSTRING_BEFORE, BuiltinFunctions.SUBSTRING_AFTER,
-                    BuiltinFunctions.CREATE_POLYGON, BuiltinFunctions.CREATE_MBR, BuiltinFunctions.CREATE_RECTANGLE,
-                    BuiltinFunctions.CREATE_CIRCLE, BuiltinFunctions.CREATE_LINE, BuiltinFunctions.CREATE_POINT,
-                    BuiltinFunctions.NUMERIC_ADD, BuiltinFunctions.NUMERIC_SUBTRACT, BuiltinFunctions.NUMERIC_MULTIPLY,
-                    BuiltinFunctions.NUMERIC_DIVIDE, BuiltinFunctions.NUMERIC_DIV, BuiltinFunctions.NUMERIC_MOD);
-
     public abstract Map<FunctionIdentifier, List<IAccessMethod>> getAccessMethods();
 
     protected static void registerAccessMethod(IAccessMethod accessMethod,
@@ -243,6 +225,7 @@
                 boolean isNgramIndexChosen = indexType == IndexType.LENGTH_PARTITIONED_NGRAM_INVIX
                         || indexType == IndexType.SINGLE_PARTITION_NGRAM_INVIX;
                 if ((chosenAccessMethod == BTreeAccessMethod.INSTANCE && indexType == IndexType.BTREE)
+                        || (chosenAccessMethod == ArrayBTreeAccessMethod.INSTANCE && indexType == IndexType.ARRAY)
                         || (chosenAccessMethod == RTreeAccessMethod.INSTANCE && indexType == IndexType.RTREE)
                         // the inverted index will be utilized
                         // For Ngram, the full-text config used in the index and in the query are always the default one,
@@ -263,8 +246,7 @@
                         // 1) the full-text ftcontains() function
                         // 2) functions that take keyword as an argument, e.g. edit_distance_check() when the threshold is 1
                         || (chosenAccessMethod == InvertedIndexAccessMethod.INSTANCE && isKeywordIndexChosen
-                                && isSameFullTextConfigInIndexAndQuery(analysisCtx,
-                                        chosenIndex.getFullTextConfigName()))) {
+                                && isSameFullTextConfigInIndexAndQuery(analysisCtx, chosenIndex.getIndexDetails()))) {
 
                     if (resultVarsToIndexTypesMap.containsKey(indexEntry.getValue())) {
                         List<IndexType> appliedIndexTypes = resultVarsToIndexTypesMap.get(indexEntry.getValue());
@@ -285,7 +267,9 @@
     }
 
     private boolean isSameFullTextConfigInIndexAndQuery(AccessMethodAnalysisContext analysisCtx,
-            String indexFullTextConfig) {
+            Index.IIndexDetails indexDetails) {
+        String indexFullTextConfig = ((Index.TextIndexDetails) indexDetails).getFullTextConfigName();
+
         IOptimizableFuncExpr expr = analysisCtx.getMatchedFuncExpr(0);
         if (FullTextUtil.isFullTextContainsFunctionExpr(expr)) {
             // ftcontains()
@@ -327,6 +311,39 @@
         while (indexExprAndVarIt.hasNext()) {
             Map.Entry<Index, List<Pair<Integer, Integer>>> indexExprAndVarEntry = indexExprAndVarIt.next();
             Index index = indexExprAndVarEntry.getKey();
+            IndexType indexType = index.getIndexType();
+            if (!accessMethod.matchIndexType(indexType)) {
+                indexExprAndVarIt.remove();
+                continue;
+            }
+            List<List<String>> keyFieldNames;
+            List<IAType> keyFieldTypes;
+            switch (Index.IndexCategory.of(indexType)) {
+                case ARRAY:
+                    Index.ArrayIndexDetails arrayIndexDetails = (Index.ArrayIndexDetails) index.getIndexDetails();
+                    keyFieldNames = new ArrayList<>();
+                    keyFieldTypes = new ArrayList<>();
+                    for (Index.ArrayIndexElement e : arrayIndexDetails.getElementList()) {
+                        for (int i = 0; i < e.getProjectList().size(); i++) {
+                            List<String> project = e.getProjectList().get(i);
+                            keyFieldNames.add(ArrayIndexUtil.getFlattenedKeyFieldNames(e.getUnnestList(), project));
+                            keyFieldTypes.add(e.getTypeList().get(i));
+                        }
+                    }
+                    break;
+                case VALUE:
+                    Index.ValueIndexDetails valueIndexDetails = (Index.ValueIndexDetails) index.getIndexDetails();
+                    keyFieldNames = valueIndexDetails.getKeyFieldNames();
+                    keyFieldTypes = valueIndexDetails.getKeyFieldTypes();
+                    break;
+                case TEXT:
+                    Index.TextIndexDetails textIndexDetails = (Index.TextIndexDetails) index.getIndexDetails();
+                    keyFieldNames = textIndexDetails.getKeyFieldNames();
+                    keyFieldTypes = textIndexDetails.getKeyFieldTypes();
+                    break;
+                default:
+                    throw new CompilationException(ErrorCode.COMPILATION_UNKNOWN_INDEX_TYPE, String.valueOf(indexType));
+            }
 
             boolean allUsed = true;
             int lastFieldMatched = -1;
@@ -334,9 +351,9 @@
             // Used to keep track of matched expressions (added for prefix search)
             int numMatchedKeys = 0;
 
-            for (int i = 0; i < index.getKeyFieldNames().size(); i++) {
-                List<String> keyField = index.getKeyFieldNames().get(i);
-                final IAType keyType = index.getKeyFieldTypes().get(i);
+            for (int i = 0; i < keyFieldNames.size(); i++) {
+                List<String> keyField = keyFieldNames.get(i);
+                final IAType keyType = keyFieldTypes.get(i);
                 boolean foundKeyField = false;
                 Iterator<Pair<Integer, Integer>> exprsAndVarIter = indexExprAndVarEntry.getValue().iterator();
                 while (exprsAndVarIter.hasNext()) {
@@ -631,15 +648,52 @@
         List<Index> indexCandidates = new ArrayList<>();
         // Add an index to the candidates if one of the indexed fields is fieldName
         for (Index index : datasetIndexes) {
+            List<List<String>> keyFieldNames;
+            List<IAType> keyFieldTypes;
+            List<Integer> keySources;
+            boolean isOverridingKeyFieldTypes;
+            switch (Index.IndexCategory.of(index.getIndexType())) {
+                case ARRAY:
+                    Index.ArrayIndexDetails arrayIndexDetails = (Index.ArrayIndexDetails) index.getIndexDetails();
+                    keyFieldNames = new ArrayList<>();
+                    keyFieldTypes = new ArrayList<>();
+                    keySources = new ArrayList<>();
+                    for (Index.ArrayIndexElement e : arrayIndexDetails.getElementList()) {
+                        for (int i = 0; i < e.getProjectList().size(); i++) {
+                            List<String> project = e.getProjectList().get(i);
+                            keyFieldNames.add(ArrayIndexUtil.getFlattenedKeyFieldNames(e.getUnnestList(), project));
+                            keyFieldTypes.add(e.getTypeList().get(i).getType());
+                            keySources.add(e.getSourceIndicator());
+                        }
+                    }
+                    isOverridingKeyFieldTypes = arrayIndexDetails.isOverridingKeyFieldTypes();
+                    break;
+                case VALUE:
+                    Index.ValueIndexDetails valueIndexDetails = (Index.ValueIndexDetails) index.getIndexDetails();
+                    keyFieldNames = valueIndexDetails.getKeyFieldNames();
+                    keyFieldTypes = valueIndexDetails.getKeyFieldTypes();
+                    keySources = valueIndexDetails.getKeyFieldSourceIndicators();
+                    isOverridingKeyFieldTypes = valueIndexDetails.isOverridingKeyFieldTypes();
+                    break;
+                case TEXT:
+                    Index.TextIndexDetails textIndexDetails = (Index.TextIndexDetails) index.getIndexDetails();
+                    keyFieldNames = textIndexDetails.getKeyFieldNames();
+                    keyFieldTypes = textIndexDetails.getKeyFieldTypes();
+                    keySources = textIndexDetails.getKeyFieldSourceIndicators();
+                    isOverridingKeyFieldTypes = textIndexDetails.isOverridingKeyFieldTypes();
+                    break;
+                default:
+                    throw new CompilationException(ErrorCode.COMPILATION_UNKNOWN_INDEX_TYPE,
+                            String.valueOf(index.getIndexType()));
+            }
             // Need to also verify the index is pending no op
-            int keyIdx = index.getKeyFieldNames().indexOf(fieldName);
-            List<Integer> keySources = index.getKeyFieldSourceIndicators();
+            int keyIdx = keyFieldNames.indexOf(fieldName);
             if (keyIdx >= 0 && keySourceMatches(keySources, keyIdx, fieldSource)
                     && index.getPendingOp() == MetadataUtil.PENDING_NO_OP) {
                 indexCandidates.add(index);
                 boolean isFieldTypeUnknown = fieldType == BuiltinType.AMISSING || fieldType == BuiltinType.ANY;
-                if (isFieldTypeUnknown && (!index.isOverridingKeyFieldTypes() || index.isEnforced())) {
-                    IAType indexedType = index.getKeyFieldTypes().get(keyIdx);
+                if (isFieldTypeUnknown && (!isOverridingKeyFieldTypes || index.isEnforced())) {
+                    IAType indexedType = keyFieldTypes.get(keyIdx);
                     optFuncExpr.setFieldType(varIdx, indexedType);
                 }
                 analysisCtx.addIndexExpr(matchedSubTree.getDataset(), index, matchedFuncExprIndex, varIdx);
@@ -731,9 +785,13 @@
             varRef.setSourceLocation(unnestOp.getSourceLocation());
             optFuncExpr.setLogicalExpr(funcVarIndex, varRef);
         } else {
-            fieldName = getFieldNameFromSubTree(optFuncExpr, subTree, assignOrUnnestIndex, 0, subTree.getRecordType(),
-                    funcVarIndex, optFuncExpr.getFuncExpr().getArguments().get(funcVarIndex).getValue(),
-                    subTree.getMetaRecordType(), datasetMetaVar, fieldSource);
+            if (subTree.getDataSourceType() == DataSourceType.DATASOURCE_SCAN) {
+                subTree.setLastMatchedDataSourceVars(0, funcVarIndex);
+            }
+            fieldName = AccessMethodUtils.getFieldNameFromSubTree(optFuncExpr, subTree, assignOrUnnestIndex, 0,
+                    subTree.getRecordType(), funcVarIndex,
+                    optFuncExpr.getFuncExpr().getArguments().get(funcVarIndex).getValue(), subTree.getMetaRecordType(),
+                    datasetMetaVar, fieldSource, false);
             if (fieldName.isEmpty()) {
                 return;
             }
@@ -756,25 +814,40 @@
             OptimizableOperatorSubTree subTree, int assignOrUnnestIndex, LogicalVariable datasetMetaVar,
             IOptimizationContext context, List<Index> datasetIndexes, int optFuncExprIndex,
             AccessMethodAnalysisContext analysisCtx) throws AlgebricksException {
+        boolean doesArrayIndexQualify = context.getPhysicalOptimizationConfig().isArrayIndexEnabled()
+                && datasetIndexes.stream().anyMatch(i -> i.getIndexType() == IndexType.ARRAY)
+                && assignOrUnnestIndex == subTree.getAssignsAndUnnests().size() - 1;
         List<LogicalVariable> varList = assignOp.getVariables();
         MutableInt fieldSource = new MutableInt(0);
         for (int varIndex = 0; varIndex < varList.size(); varIndex++) {
             LogicalVariable var = varList.get(varIndex);
             int optVarIndex = optFuncExpr.findLogicalVar(var);
-            // No matching var in optFuncExpr.
             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);
+                    if (fieldTriplet != null && subTree.hasDataSource()) {
+                        fillIndexExprs(datasetIndexes, fieldTriplet.second, fieldTriplet.third, optFuncExpr,
+                                optFuncExprIndex, fieldTriplet.first, subTree, analysisCtx, fieldSource.intValue());
+                    }
+                }
                 continue;
             }
             // At this point we have matched the optimizable func
             // expr at optFuncExprIndex to an assigned variable.
             // Remember matching subtree.
             optFuncExpr.setOptimizableSubTree(optVarIndex, subTree);
+            if (subTree.getDataSourceType() == DataSourceType.DATASOURCE_SCAN) {
+                subTree.setLastMatchedDataSourceVars(varIndex, optVarIndex);
+            }
 
             fieldSource.setValue(0);
-            List<String> fieldName = getFieldNameFromSubTree(optFuncExpr, subTree, assignOrUnnestIndex, varIndex,
-                    subTree.getRecordType(), optVarIndex,
+            List<String> fieldName = AccessMethodUtils.getFieldNameFromSubTree(optFuncExpr, subTree,
+                    assignOrUnnestIndex, varIndex, subTree.getRecordType(), optVarIndex,
                     optFuncExpr.getFuncExpr().getArguments().get(optVarIndex).getValue(), subTree.getMetaRecordType(),
-                    datasetMetaVar, fieldSource);
+                    datasetMetaVar, fieldSource, false);
 
             IAType fieldType = (IAType) context.getOutputTypeEnvironment(assignOp).getVarType(var);
             // Set the fieldName in the corresponding matched
@@ -870,220 +943,6 @@
     }
 
     /**
-     * Returns the field name corresponding to the assigned variable at
-     * varIndex. Returns Collections.emptyList() if the expr at varIndex does not yield to a field
-     * access function after following a set of allowed functions.
-     *
-     * @throws AlgebricksException
-     */
-    protected List<String> getFieldNameFromSubTree(IOptimizableFuncExpr optFuncExpr, OptimizableOperatorSubTree subTree,
-            int opIndex, int assignVarIndex, ARecordType recordType, int funcVarIndex,
-            ILogicalExpression parentFuncExpr, ARecordType metaType, LogicalVariable metaVar, MutableInt fieldSource)
-            throws AlgebricksException {
-        // Get expression corresponding to opVar at varIndex.
-        AbstractLogicalExpression expr = null;
-        AbstractFunctionCallExpression childFuncExpr = null;
-        AbstractLogicalOperator op = subTree.getAssignsAndUnnests().get(opIndex);
-        if (op.getOperatorTag() == LogicalOperatorTag.ASSIGN) {
-            AssignOperator assignOp = (AssignOperator) op;
-            expr = (AbstractLogicalExpression) assignOp.getExpressions().get(assignVarIndex).getValue();
-            // Can't get a field name from a constant expression. So, return null.
-            if (expr.getExpressionTag() == LogicalExpressionTag.CONSTANT) {
-                return Collections.emptyList();
-            }
-            childFuncExpr = (AbstractFunctionCallExpression) expr;
-        } else {
-            UnnestOperator unnestOp = (UnnestOperator) op;
-            expr = (AbstractLogicalExpression) unnestOp.getExpressionRef().getValue();
-            if (expr.getExpressionTag() != LogicalExpressionTag.FUNCTION_CALL) {
-                return Collections.emptyList();
-            }
-            childFuncExpr = (AbstractFunctionCallExpression) expr;
-            if (childFuncExpr.getFunctionIdentifier() != BuiltinFunctions.SCAN_COLLECTION) {
-                return Collections.emptyList();
-            }
-            expr = (AbstractLogicalExpression) childFuncExpr.getArguments().get(0).getValue();
-        }
-        if (expr.getExpressionTag() != LogicalExpressionTag.FUNCTION_CALL) {
-            return Collections.emptyList();
-        }
-        AbstractFunctionCallExpression funcExpr = (AbstractFunctionCallExpression) expr;
-        FunctionIdentifier funcIdent = funcExpr.getFunctionIdentifier();
-
-        boolean isByName = false;
-        boolean isFieldAccess = false;
-        String fieldName = null;
-        List<String> nestedAccessFieldName = null;
-        int fieldIndex = -1;
-        if (funcIdent == BuiltinFunctions.FIELD_ACCESS_BY_NAME) {
-            fieldName = ConstantExpressionUtil.getStringArgument(funcExpr, 1);
-            if (fieldName == null) {
-                return Collections.emptyList();
-            }
-            isFieldAccess = true;
-            isByName = true;
-        } else if (funcIdent == BuiltinFunctions.FIELD_ACCESS_BY_INDEX) {
-            Integer idx = ConstantExpressionUtil.getIntArgument(funcExpr, 1);
-            if (idx == null) {
-                return Collections.emptyList();
-            }
-            fieldIndex = idx;
-            isFieldAccess = true;
-        } else if (funcIdent == BuiltinFunctions.FIELD_ACCESS_NESTED) {
-            ILogicalExpression nameArg = funcExpr.getArguments().get(1).getValue();
-            if (nameArg.getExpressionTag() != LogicalExpressionTag.CONSTANT) {
-                return Collections.emptyList();
-            }
-            ConstantExpression constExpr = (ConstantExpression) nameArg;
-            AOrderedList orderedNestedFieldName =
-                    (AOrderedList) ((AsterixConstantValue) constExpr.getValue()).getObject();
-            nestedAccessFieldName = new ArrayList<>();
-            for (int i = 0; i < orderedNestedFieldName.size(); i++) {
-                nestedAccessFieldName.add(((AString) orderedNestedFieldName.getItem(i)).getStringValue());
-            }
-            isFieldAccess = true;
-            isByName = true;
-        }
-        if (isFieldAccess) {
-            LogicalVariable sourceVar =
-                    ((VariableReferenceExpression) funcExpr.getArguments().get(0).getValue()).getVariableReference();
-            if (sourceVar.equals(metaVar)) {
-                fieldSource.setValue(1);
-            } else {
-                fieldSource.setValue(0);
-            }
-            if (optFuncExpr != null) {
-                optFuncExpr.setLogicalExpr(funcVarIndex, parentFuncExpr);
-            }
-            int[] assignAndExpressionIndexes = null;
-
-            //go forward through nested assigns until you find the relevant one
-            for (int i = opIndex + 1; i < subTree.getAssignsAndUnnests().size(); i++) {
-                AbstractLogicalOperator subOp = subTree.getAssignsAndUnnests().get(i);
-                List<LogicalVariable> varList;
-
-                if (subOp.getOperatorTag() == LogicalOperatorTag.ASSIGN) {
-                    //Nested was an assign
-                    varList = ((AssignOperator) subOp).getVariables();
-                } else if (subOp.getOperatorTag() == LogicalOperatorTag.UNNEST) {
-                    //Nested is not an assign
-                    varList = ((UnnestOperator) subOp).getVariables();
-                } else {
-                    break;
-                }
-
-                //Go through variables in assign to check for match
-                for (int varIndex = 0; varIndex < varList.size(); varIndex++) {
-                    LogicalVariable var = varList.get(varIndex);
-                    ArrayList<LogicalVariable> parentVars = new ArrayList<>();
-                    expr.getUsedVariables(parentVars);
-
-                    if (parentVars.contains(var)) {
-                        //Found the variable we are looking for.
-                        //return assign and index of expression
-                        int[] returnValues = { i, varIndex };
-                        assignAndExpressionIndexes = returnValues;
-                    }
-                }
-            }
-            if (assignAndExpressionIndexes != null && assignAndExpressionIndexes[0] > -1) {
-                //We found the nested assign
-
-                //Recursive call on nested assign
-                List<String> parentFieldNames = getFieldNameFromSubTree(optFuncExpr, subTree,
-                        assignAndExpressionIndexes[0], assignAndExpressionIndexes[1], recordType, funcVarIndex,
-                        parentFuncExpr, metaType, metaVar, fieldSource);
-
-                if (parentFieldNames.isEmpty()) {
-                    //Nested assign was not a field access.
-                    //We will not use index
-                    return Collections.emptyList();
-                }
-
-                if (!isByName) {
-                    IAType subFieldType = sourceVar.equals(metaVar) ? metaType.getSubFieldType(parentFieldNames)
-                            : recordType.getSubFieldType(parentFieldNames);
-                    // Sub-field type can be AUnionType in case if it's optional. Thus, needs to get the actual type.
-                    subFieldType = TypeComputeUtils.getActualType(subFieldType);
-                    if (subFieldType.getTypeTag() != ATypeTag.OBJECT) {
-                        throw CompilationException.create(ErrorCode.TYPE_CONVERT, subFieldType,
-                                ARecordType.class.getName());
-                    }
-                    fieldName = ((ARecordType) subFieldType).getFieldNames()[fieldIndex];
-                }
-                if (optFuncExpr != null) {
-                    optFuncExpr.setSourceVar(funcVarIndex, ((AssignOperator) op).getVariables().get(assignVarIndex));
-                }
-                //add fieldName to the nested fieldName, return
-                if (nestedAccessFieldName != null) {
-                    for (int i = 0; i < nestedAccessFieldName.size(); i++) {
-                        parentFieldNames.add(nestedAccessFieldName.get(i));
-                    }
-                } else {
-                    parentFieldNames.add(fieldName);
-                }
-                return (parentFieldNames);
-            }
-
-            if (optFuncExpr != null) {
-                optFuncExpr.setSourceVar(funcVarIndex, ((AssignOperator) op).getVariables().get(assignVarIndex));
-            }
-            //no nested assign, we are at the lowest level.
-            if (isByName) {
-                if (nestedAccessFieldName != null) {
-                    return nestedAccessFieldName;
-                }
-                return new ArrayList<>(Arrays.asList(fieldName));
-            }
-            return new ArrayList<>(Arrays.asList(sourceVar.equals(metaVar) ? metaType.getFieldNames()[fieldIndex]
-                    : recordType.getFieldNames()[fieldIndex]));
-
-        }
-
-        if (!funcIDSetThatRetainFieldName.contains(funcIdent)) {
-            return Collections.emptyList();
-        }
-        // We use a part of the field in edit distance computation
-        if (optFuncExpr != null
-                && optFuncExpr.getFuncExpr().getFunctionIdentifier() == BuiltinFunctions.EDIT_DISTANCE_CHECK) {
-            optFuncExpr.setPartialField(true);
-        }
-        // We expect the function's argument to be a variable, otherwise we
-        // cannot apply an index.
-        ILogicalExpression argExpr = funcExpr.getArguments().get(0).getValue();
-        if (argExpr.getExpressionTag() != LogicalExpressionTag.VARIABLE) {
-            return Collections.emptyList();
-        }
-        LogicalVariable curVar = ((VariableReferenceExpression) argExpr).getVariableReference();
-        // We look for the assign or unnest operator that produces curVar below
-        // the current operator
-        for (int assignOrUnnestIndex = opIndex + 1; assignOrUnnestIndex < subTree.getAssignsAndUnnests()
-                .size(); assignOrUnnestIndex++) {
-            AbstractLogicalOperator curOp = subTree.getAssignsAndUnnests().get(assignOrUnnestIndex);
-            if (curOp.getOperatorTag() == LogicalOperatorTag.ASSIGN) {
-                AssignOperator assignOp = (AssignOperator) curOp;
-                List<LogicalVariable> varList = assignOp.getVariables();
-                for (int varIndex = 0; varIndex < varList.size(); varIndex++) {
-                    LogicalVariable var = varList.get(varIndex);
-                    if (var.equals(curVar) && optFuncExpr != null) {
-                        optFuncExpr.setSourceVar(funcVarIndex, var);
-                        return getFieldNameFromSubTree(optFuncExpr, subTree, assignOrUnnestIndex, varIndex, recordType,
-                                funcVarIndex, childFuncExpr, metaType, metaVar, fieldSource);
-                    }
-                }
-            } else {
-                UnnestOperator unnestOp = (UnnestOperator) curOp;
-                LogicalVariable var = unnestOp.getVariable();
-                if (var.equals(curVar)) {
-                    getFieldNameFromSubTree(optFuncExpr, subTree, assignOrUnnestIndex, 0, recordType, funcVarIndex,
-                            childFuncExpr, metaType, metaVar, fieldSource);
-                }
-            }
-        }
-        return Collections.emptyList();
-    }
-
-    /**
      * Finds the field name of each variable in the ASSIGN or UNNEST operators of the sub-tree.
      */
     protected void fillFieldNamesInTheSubTree(OptimizableOperatorSubTree subTree) throws AlgebricksException {
@@ -1107,9 +966,9 @@
                     // funcVarIndex is not required. Thus, we set it to -1.
                     // optFuncExpr and parentFuncExpr are not required, too. Thus, we set them to null.
                     fieldSource.setValue(0);
-                    List<String> fieldName = getFieldNameFromSubTree(null, subTree, assignOrUnnestIndex, varIndex,
-                            subTree.getRecordType(), -1, null, subTree.getMetaRecordType(), datasetMetaVar,
-                            fieldSource);
+                    List<String> fieldName = AccessMethodUtils.getFieldNameFromSubTree(null, subTree,
+                            assignOrUnnestIndex, varIndex, subTree.getRecordType(), -1, null,
+                            subTree.getMetaRecordType(), datasetMetaVar, fieldSource, false);
                     if (fieldName != null && !fieldName.isEmpty()) {
                         subTree.getVarsToFieldNameMap().put(var, fieldName);
                     }
@@ -1122,8 +981,9 @@
                     // funcVarIndex is not required. Thus, we set it to -1.
                     // optFuncExpr and parentFuncExpr are not required, too. Thus, we set them to null.
                     fieldSource.setValue(0);
-                    fieldName = getFieldNameFromSubTree(null, subTree, assignOrUnnestIndex, 0, subTree.getRecordType(),
-                            -1, null, subTree.getMetaRecordType(), datasetMetaVar, fieldSource);
+                    fieldName = AccessMethodUtils.getFieldNameFromSubTree(null, subTree, assignOrUnnestIndex, 0,
+                            subTree.getRecordType(), -1, null, subTree.getMetaRecordType(), datasetMetaVar, fieldSource,
+                            false);
                     if (fieldName != null && !fieldName.isEmpty()) {
                         subTree.getVarsToFieldNameMap().put(var, fieldName);
                     }
@@ -1149,9 +1009,9 @@
                     // funcVarIndex is not required. Thus, we set it to -1.
                     // optFuncExpr and parentFuncExpr are not required, too. Thus, we set them to null.
                     fieldSource.setValue(0);
-                    List<String> fieldName = getFieldNameFromSubTree(null, subTree, assignOrUnnestIndex, varIndex,
-                            subTree.getRecordType(), -1, null, subTree.getMetaRecordType(), datasetMetaVar,
-                            fieldSource);
+                    List<String> fieldName = AccessMethodUtils.getFieldNameFromSubTree(null, subTree,
+                            assignOrUnnestIndex, varIndex, subTree.getRecordType(), -1, null,
+                            subTree.getMetaRecordType(), datasetMetaVar, fieldSource, false);
                     if (fieldName != null && !fieldName.isEmpty()) {
                         subTree.getVarsToFieldNameMap().put(var, fieldName);
                     }
@@ -1167,10 +1027,10 @@
                 subTree.getPrimaryKeyVars(null, primaryKeyVarList);
 
                 Index primaryIndex = getPrimaryIndexFromDataSourceScanOp(subTree.getDataSourceRef().getValue());
-
+                List<List<String>> keyFieldNames =
+                        ((Index.ValueIndexDetails) primaryIndex.getIndexDetails()).getKeyFieldNames();
                 for (int i = 0; i < primaryKeyVarList.size(); i++) {
-                    subTree.getVarsToFieldNameMap().put(primaryKeyVarList.get(i),
-                            primaryIndex.getKeyFieldNames().get(i));
+                    subTree.getVarsToFieldNameMap().put(primaryKeyVarList.get(i), keyFieldNames.get(i));
                 }
             }
 
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 e003ea5..09f9e41 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
@@ -20,6 +20,7 @@
 package org.apache.asterix.optimizer.rules.am;
 
 import java.util.ArrayList;
+import java.util.Arrays;
 import java.util.Collection;
 import java.util.Collections;
 import java.util.HashSet;
@@ -27,6 +28,7 @@
 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;
@@ -41,9 +43,11 @@
 import org.apache.asterix.metadata.entities.Dataset;
 import org.apache.asterix.metadata.entities.ExternalDatasetDetails;
 import org.apache.asterix.metadata.entities.Index;
+import org.apache.asterix.metadata.utils.ArrayIndexUtil;
 import org.apache.asterix.metadata.utils.KeyFieldTypeUtil;
 import org.apache.asterix.om.base.ABoolean;
 import org.apache.asterix.om.base.AInt32;
+import org.apache.asterix.om.base.AOrderedList;
 import org.apache.asterix.om.base.AString;
 import org.apache.asterix.om.base.IACursor;
 import org.apache.asterix.om.base.IAObject;
@@ -59,6 +63,7 @@
 import org.apache.asterix.om.types.hierachy.ATypeHierarchy.TypeCastingMathFunctionType;
 import org.apache.asterix.om.utils.ConstantExpressionUtil;
 import org.apache.commons.lang3.mutable.Mutable;
+import org.apache.commons.lang3.mutable.MutableInt;
 import org.apache.commons.lang3.mutable.MutableObject;
 import org.apache.hyracks.algebricks.common.exceptions.AlgebricksException;
 import org.apache.hyracks.algebricks.common.utils.Pair;
@@ -72,6 +77,7 @@
 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.AbstractFunctionCallExpression;
+import org.apache.hyracks.algebricks.core.algebra.expressions.AbstractLogicalExpression;
 import org.apache.hyracks.algebricks.core.algebra.expressions.ConstantExpression;
 import org.apache.hyracks.algebricks.core.algebra.expressions.IVariableTypeEnvironment;
 import org.apache.hyracks.algebricks.core.algebra.expressions.ScalarFunctionCallExpression;
@@ -88,6 +94,7 @@
 import org.apache.hyracks.algebricks.core.algebra.operators.logical.AbstractUnnestMapOperator;
 import org.apache.hyracks.algebricks.core.algebra.operators.logical.AggregateOperator;
 import org.apache.hyracks.algebricks.core.algebra.operators.logical.AssignOperator;
+import org.apache.hyracks.algebricks.core.algebra.operators.logical.DistinctOperator;
 import org.apache.hyracks.algebricks.core.algebra.operators.logical.GroupByOperator;
 import org.apache.hyracks.algebricks.core.algebra.operators.logical.LeftOuterUnnestMapOperator;
 import org.apache.hyracks.algebricks.core.algebra.operators.logical.OrderOperator;
@@ -95,6 +102,7 @@
 import org.apache.hyracks.algebricks.core.algebra.operators.logical.SplitOperator;
 import org.apache.hyracks.algebricks.core.algebra.operators.logical.UnionAllOperator;
 import org.apache.hyracks.algebricks.core.algebra.operators.logical.UnnestMapOperator;
+import org.apache.hyracks.algebricks.core.algebra.operators.logical.UnnestOperator;
 import org.apache.hyracks.algebricks.core.algebra.operators.logical.WindowOperator;
 import org.apache.hyracks.algebricks.core.algebra.operators.logical.visitors.VariableUtilities;
 import org.apache.hyracks.algebricks.core.algebra.plan.ALogicalPlanImpl;
@@ -103,6 +111,8 @@
 import org.apache.hyracks.api.exceptions.SourceLocation;
 import org.apache.hyracks.storage.am.lsm.invertedindex.tokenizers.DelimitedUTF8StringBinaryTokenizer;
 
+import com.google.common.collect.ImmutableSet;
+
 /**
  * Static helper functions for rewriting plans using indexes.
  */
@@ -115,6 +125,15 @@
         CONDITIONAL_SPLIT_VAR
     }
 
+    // Function Identifier sets that retain the original field variable through each function's arguments
+    private final static ImmutableSet<FunctionIdentifier> funcIDSetThatRetainFieldName =
+            ImmutableSet.of(BuiltinFunctions.WORD_TOKENS, BuiltinFunctions.GRAM_TOKENS, BuiltinFunctions.SUBSTRING,
+                    BuiltinFunctions.SUBSTRING_BEFORE, BuiltinFunctions.SUBSTRING_AFTER,
+                    BuiltinFunctions.CREATE_POLYGON, BuiltinFunctions.CREATE_MBR, BuiltinFunctions.CREATE_RECTANGLE,
+                    BuiltinFunctions.CREATE_CIRCLE, BuiltinFunctions.CREATE_LINE, BuiltinFunctions.CREATE_POINT,
+                    BuiltinFunctions.NUMERIC_ADD, BuiltinFunctions.NUMERIC_SUBTRACT, BuiltinFunctions.NUMERIC_MULTIPLY,
+                    BuiltinFunctions.NUMERIC_DIVIDE, BuiltinFunctions.NUMERIC_DIV, BuiltinFunctions.NUMERIC_MOD);
+
     public static void appendPrimaryIndexTypes(Dataset dataset, IAType itemType, IAType metaItemType,
             List<Object> target) throws AlgebricksException {
         ARecordType recordType = (ARecordType) itemType;
@@ -324,6 +343,9 @@
         boolean primaryKeysOnly = isInvertedIndex(index);
         if (!primaryKeysOnly) {
             switch (index.getIndexType()) {
+                case ARRAY:
+                    dest.addAll(KeyFieldTypeUtil.getArrayBTreeIndexKeyTypes(index, recordType, metaRecordType));
+                    break;
                 case BTREE:
                     dest.addAll(KeyFieldTypeUtil.getBTreeIndexKeyTypes(index, recordType, metaRecordType));
                     break;
@@ -952,10 +974,40 @@
 
     private static AbstractUnnestMapOperator createFinalNonIndexOnlySearchPlan(Dataset dataset,
             ILogicalOperator inputOp, IOptimizationContext context, boolean sortPrimaryKeys, boolean retainInput,
-            boolean retainMissing, boolean requiresBroadcast, List<LogicalVariable> primaryKeyVars,
-            List<LogicalVariable> primaryIndexUnnestVars, List<Object> primaryIndexOutputTypes)
-            throws AlgebricksException {
+            boolean retainMissing, boolean requiresBroadcast, boolean requiresDistinct,
+            List<LogicalVariable> primaryKeyVars, List<LogicalVariable> primaryIndexUnnestVars,
+            List<LogicalVariable> auxDistinctVars, List<Object> primaryIndexOutputTypes) throws AlgebricksException {
         SourceLocation sourceLoc = inputOp.getSourceLocation();
+
+        // Sanity check: requiresDistinct and sortPrimaryKeys are mutually exclusive.
+        if (requiresDistinct && sortPrimaryKeys) {
+            throw new CompilationException(ErrorCode.COMPILATION_ILLEGAL_STATE, sourceLoc,
+                    "Non-index search plan " + "cannot include a DISTINCT and an ORDER.");
+        }
+
+        // If we have an array index, then we must only give unique keys to our primary-index scan.
+        DistinctOperator distinct = null;
+        if (requiresDistinct) {
+            List<Mutable<ILogicalExpression>> distinctExprs = new ArrayList<>();
+            for (LogicalVariable pkVar : primaryKeyVars) {
+                VariableReferenceExpression pkVarRef = new VariableReferenceExpression(pkVar);
+                pkVarRef.setSourceLocation(sourceLoc);
+                Mutable<ILogicalExpression> vRef = new MutableObject<>(pkVarRef);
+                distinctExprs.add(vRef);
+            }
+            for (LogicalVariable auxVar : auxDistinctVars) {
+                VariableReferenceExpression auxVarRef = new VariableReferenceExpression(auxVar);
+                auxVarRef.setSourceLocation(sourceLoc);
+                Mutable<ILogicalExpression> vRef = new MutableObject<>(auxVarRef);
+                distinctExprs.add(vRef);
+            }
+            distinct = new DistinctOperator(distinctExprs);
+            distinct.setSourceLocation(sourceLoc);
+            distinct.getInputs().add(new MutableObject<>(inputOp));
+            distinct.setExecutionMode(ExecutionMode.LOCAL);
+            context.computeAndSetTypeEnvironmentForOperator(distinct);
+        }
+
         // Optionally add a sort on the primary-index keys before searching the primary index.
         OrderOperator order = null;
         if (sortPrimaryKeys) {
@@ -977,7 +1029,9 @@
         AbstractUnnestMapOperator primaryIndexUnnestMapOp =
                 createPrimaryIndexUnnestMapOp(dataset, retainInput, retainMissing, requiresBroadcast, primaryKeyVars,
                         primaryIndexUnnestVars, primaryIndexOutputTypes, sourceLoc);
-        if (sortPrimaryKeys) {
+        if (requiresDistinct) {
+            primaryIndexUnnestMapOp.getInputs().add(new MutableObject<ILogicalOperator>(distinct));
+        } else if (sortPrimaryKeys) {
             primaryIndexUnnestMapOp.getInputs().add(new MutableObject<ILogicalOperator>(order));
         } else {
             primaryIndexUnnestMapOp.getInputs().add(new MutableObject<>(inputOp));
@@ -1020,7 +1074,8 @@
         // key search (SK, PK) and those in the original plan (datasource scan).
         LinkedHashMap<LogicalVariable, LogicalVariable> origVarToSIdxUnnestMapOpVarMap = new LinkedHashMap<>();
 
-        List<List<String>> chosenIndexFieldNames = secondaryIndex.getKeyFieldNames();
+        Index.ValueIndexDetails secondaryIndexDetails = (Index.ValueIndexDetails) secondaryIndex.getIndexDetails();
+        List<List<String>> chosenIndexFieldNames = secondaryIndexDetails.getKeyFieldNames();
         IndexType idxType = secondaryIndex.getIndexType();
 
         // variables used in SELECT or JOIN operator
@@ -1549,7 +1604,7 @@
      * (i.e., we can guarantee the correctness of the result.)
      *
      * Case A) non-index-only plan
-     * sidx-search -> (optional) sort -> pdix-search
+     * sidx-search -> (optional) sort -> (optional) distinct -> pdix-search
      *
      * Case B) index-only plan
      * left path (an instantTryLock() on the PK fail path):
@@ -1563,8 +1618,8 @@
             Dataset dataset, ARecordType recordType, ARecordType metaRecordType, ILogicalOperator inputOp,
             IOptimizationContext context, boolean sortPrimaryKeys, boolean retainInput, boolean retainMissing,
             boolean requiresBroadcast, Index secondaryIndex, AccessMethodAnalysisContext analysisCtx,
-            OptimizableOperatorSubTree subTree, LogicalVariable newMissingPlaceHolderForLOJ)
-            throws AlgebricksException {
+            OptimizableOperatorSubTree indexSubTree, OptimizableOperatorSubTree probeSubTree,
+            LogicalVariable newMissingPlaceHolderForLOJ) throws AlgebricksException {
         // Common part for the non-index-only plan and index-only plan
         // Variables and types for the primary-index search.
         List<LogicalVariable> primaryIndexUnnestVars = new ArrayList<>();
@@ -1577,21 +1632,32 @@
         List<LogicalVariable> pkVarsFromSIdxUnnestMapOp = AccessMethodUtils.getKeyVarsFromSecondaryUnnestMap(dataset,
                 recordType, metaRecordType, inputOp, secondaryIndex, SecondaryUnnestMapOutputVarType.PRIMARY_KEY);
 
-        // Index-only plan or not?
+        // Index-only plan or not? Array-index involved or not?
         boolean isIndexOnlyPlan = analysisCtx.getIndexOnlyPlanInfo().getFirst();
+        boolean isArrayIndex = secondaryIndex.getIndexType() == IndexType.ARRAY;
 
-        // Non-index-only plan case: creates ORDER -> UNNEST-MAP(Primary-index search) and return that unnest-map op.
+        // Non-index-only plan case: creates (ORDER)? -> (DISTINCT)? -> UNNEST-MAP(PIDX) and return that unnest-map op.
         if (!isIndexOnlyPlan) {
-            return createFinalNonIndexOnlySearchPlan(dataset, inputOp, context, sortPrimaryKeys, retainInput,
-                    retainMissing, requiresBroadcast, pkVarsFromSIdxUnnestMapOp, primaryIndexUnnestVars,
-                    primaryIndexOutputTypes);
-        } else {
+            // If we have a join + an array index, we need add the join source PK to the DISTINCT + ORDER.
+            List<LogicalVariable> joinPKVars = Collections.emptyList();
+            if (isArrayIndex && probeSubTree != null) {
+                joinPKVars = probeSubTree.getDataSourceVariables().subList(0,
+                        probeSubTree.getDataSourceVariables().size() - 1);
+            }
+
+            return createFinalNonIndexOnlySearchPlan(dataset, inputOp, context, !isArrayIndex && sortPrimaryKeys,
+                    retainInput, retainMissing, requiresBroadcast, isArrayIndex, pkVarsFromSIdxUnnestMapOp,
+                    primaryIndexUnnestVars, joinPKVars, primaryIndexOutputTypes);
+        } else if (!isArrayIndex) {
             // Index-only plan case: creates a UNIONALL operator that has two paths after the secondary unnest-map op,
             // and returns it.
             return createFinalIndexOnlySearchPlan(afterTopOpRefs, topOpRef, conditionRef, assignsBeforeTopOpRef,
                     dataset, recordType, metaRecordType, inputOp, context, retainInput, retainMissing,
-                    requiresBroadcast, secondaryIndex, analysisCtx, subTree, newMissingPlaceHolderForLOJ,
+                    requiresBroadcast, secondaryIndex, analysisCtx, indexSubTree, newMissingPlaceHolderForLOJ,
                     pkVarsFromSIdxUnnestMapOp, primaryIndexUnnestVars, primaryIndexOutputTypes);
+        } else {
+            throw new CompilationException(ErrorCode.COMPILATION_ILLEGAL_STATE, inputOp.getSourceLocation(),
+                    "Cannot use index-only plan with array indexes.");
         }
     }
 
@@ -1890,19 +1956,19 @@
         // Since index-only plan doesn't access the primary index, we can't get the actual value in this case.
         // Also, if no-index-only option is given, we stop here to honor that request.
         boolean noIndexOnlyPlanOption = !context.getPhysicalOptimizationConfig().isIndexOnly();
-        // TODO: For the inverted index access-method cases only:
+        // TODO: For the inverted index / array index access-method cases only:
         // Since an inverted index can contain multiple secondary key entries per one primary key,
         // Index-only plan can't be applied. For example, suppose there are two entries (SK1, SK2) for one PK.
         // Since we can't access <SK1, PK>, <SK2, PK> at the same time unless we use tryLock (we use instantTryLock),
         // right now, we can't support an index-only plan on an inverted index.
         // Once this issue is resolved, we can apply an index-only plan.
-        // One additional condition:
+        // One additional condition for inverted indexes:
         // Even if the above is resolved, if a secondary key field is used after
         // SELECT or JOIN operator, this can't be qualified as an index-only plan since
         // an inverted index contains a part of a field value, not all of it.
         if (noIndexOnlyPlanOption || dataset.getDatasetType() == DatasetType.EXTERNAL || chosenIndex.isPrimaryIndex()
-                || chosenIndex.isOverridingKeyFieldTypes() || chosenIndex.isEnforced()
-                || isInvertedIndex(chosenIndex)) {
+                || chosenIndex.getIndexDetails().isOverridingKeyFieldTypes() || chosenIndex.isEnforced()
+                || isInvertedIndex(chosenIndex) || chosenIndex.getIndexType() == IndexType.ARRAY) {
             indexOnlyPlanInfo.setFirst(false);
             return;
         }
@@ -2002,7 +2068,8 @@
         // assign or data-source-scan in the subtree and the field-name of those variables are only PK or SK.
         // Needs to check whether variables from the given select (join) operator only contain SK and/or PK condition.
         List<List<String>> pkFieldNames = dataset.getPrimaryKeys();
-        List<List<String>> chosenIndexFieldNames = chosenIndex.getKeyFieldNames();
+        Index.ValueIndexDetails chosenIndexDetails = (Index.ValueIndexDetails) chosenIndex.getIndexDetails();
+        List<List<String>> chosenIndexFieldNames = chosenIndexDetails.getKeyFieldNames();
         List<LogicalVariable> chosenIndexVars = new ArrayList<>();
 
         // Collects variables that contain a CONSTANT expression in ASSIGN operators in the subtree.
@@ -2026,7 +2093,7 @@
         }
 
         // For the composite index, a secondary-index search generates a superset of the results.
-        if (chosenIndex.getKeyFieldNames().size() > 1 && indexApplicableVarFoundCount > 1) {
+        if (chosenIndexDetails.getKeyFieldNames().size() > 1 && indexApplicableVarFoundCount > 1) {
             requireVerificationAfterSIdxSearch = true;
         }
 
@@ -2674,4 +2741,451 @@
         AbstractExpressionAnnotationWithIndexNames ann = optFuncExpr.getFuncExpr().getAnnotation(annClass);
         return ann == null ? null : ann.getIndexNames();
     }
+
+    /**
+     * Returns the field name corresponding to the assigned variable at
+     * varIndex. Returns Collections.emptyList() if the expr at varIndex does not yield to a field
+     * access function after following a set of allowed functions.
+     *
+     * @throws AlgebricksException
+     */
+    public static List<String> getFieldNameFromSubTree(IOptimizableFuncExpr optFuncExpr,
+            OptimizableOperatorSubTree subTree, int opIndex, int assignVarIndex, ARecordType recordType,
+            int funcVarIndex, ILogicalExpression parentFuncExpr, ARecordType metaType, LogicalVariable metaVar,
+            MutableInt fieldSource, boolean isUnnestOverVarAllowed) throws AlgebricksException {
+        // Get expression corresponding to opVar at varIndex.
+        AbstractLogicalExpression expr = null;
+        AbstractFunctionCallExpression childFuncExpr = null;
+        AbstractLogicalOperator op = subTree.getAssignsAndUnnests().get(opIndex);
+        if (op.getOperatorTag() == LogicalOperatorTag.ASSIGN) {
+            AssignOperator assignOp = (AssignOperator) op;
+            expr = (AbstractLogicalExpression) assignOp.getExpressions().get(assignVarIndex).getValue();
+            // Can't get a field name from a constant expression. So, return null.
+            if (expr.getExpressionTag() == LogicalExpressionTag.CONSTANT) {
+                return Collections.emptyList();
+            }
+            childFuncExpr = (AbstractFunctionCallExpression) expr;
+        } else {
+            UnnestOperator unnestOp = (UnnestOperator) op;
+            expr = (AbstractLogicalExpression) unnestOp.getExpressionRef().getValue();
+            if (expr.getExpressionTag() != LogicalExpressionTag.FUNCTION_CALL) {
+                return Collections.emptyList();
+            }
+            childFuncExpr = (AbstractFunctionCallExpression) expr;
+            if (childFuncExpr.getFunctionIdentifier() != BuiltinFunctions.SCAN_COLLECTION) {
+                return Collections.emptyList();
+            }
+            expr = (AbstractLogicalExpression) childFuncExpr.getArguments().get(0).getValue();
+        }
+        if (expr.getExpressionTag() != LogicalExpressionTag.FUNCTION_CALL) {
+            return Collections.emptyList();
+        }
+        AbstractFunctionCallExpression funcExpr = (AbstractFunctionCallExpression) expr;
+        FunctionIdentifier funcIdent = funcExpr.getFunctionIdentifier();
+
+        boolean isByName = false;
+        boolean isFieldAccess = false;
+        String fieldName = null;
+        List<String> nestedAccessFieldName = null;
+        int fieldIndex = -1;
+        if (funcIdent == BuiltinFunctions.FIELD_ACCESS_BY_NAME) {
+            fieldName = ConstantExpressionUtil.getStringArgument(funcExpr, 1);
+            if (fieldName == null) {
+                return Collections.emptyList();
+            }
+            isFieldAccess = true;
+            isByName = true;
+        } else if (funcIdent == BuiltinFunctions.FIELD_ACCESS_BY_INDEX) {
+            Integer idx = ConstantExpressionUtil.getIntArgument(funcExpr, 1);
+            if (idx == null) {
+                return Collections.emptyList();
+            }
+            fieldIndex = idx;
+            isFieldAccess = true;
+        } else if (funcIdent == BuiltinFunctions.FIELD_ACCESS_NESTED) {
+            ILogicalExpression nameArg = funcExpr.getArguments().get(1).getValue();
+            if (nameArg.getExpressionTag() != LogicalExpressionTag.CONSTANT) {
+                return Collections.emptyList();
+            }
+            ConstantExpression constExpr = (ConstantExpression) nameArg;
+            AOrderedList orderedNestedFieldName =
+                    (AOrderedList) ((AsterixConstantValue) constExpr.getValue()).getObject();
+            nestedAccessFieldName = new ArrayList<>();
+            for (int i = 0; i < orderedNestedFieldName.size(); i++) {
+                nestedAccessFieldName.add(((AString) orderedNestedFieldName.getItem(i)).getStringValue());
+            }
+            isFieldAccess = true;
+            isByName = true;
+        }
+        if (isFieldAccess) {
+            LogicalVariable sourceVar =
+                    ((VariableReferenceExpression) funcExpr.getArguments().get(0).getValue()).getVariableReference();
+            if (sourceVar.equals(metaVar)) {
+                fieldSource.setValue(1);
+            } else {
+                fieldSource.setValue(0);
+            }
+            if (optFuncExpr != null) {
+                optFuncExpr.setLogicalExpr(funcVarIndex, parentFuncExpr);
+            }
+            int[] assignAndExpressionIndexes = null;
+
+            //go forward through nested assigns until you find the relevant one
+            for (int i = opIndex + 1; i < subTree.getAssignsAndUnnests().size(); i++) {
+                AbstractLogicalOperator subOp = subTree.getAssignsAndUnnests().get(i);
+                List<LogicalVariable> varList;
+
+                if (subOp.getOperatorTag() == LogicalOperatorTag.ASSIGN) {
+                    //Nested was an assign
+                    varList = ((AssignOperator) subOp).getVariables();
+                } else if (subOp.getOperatorTag() == LogicalOperatorTag.UNNEST) {
+                    //Nested is not an assign
+                    varList = ((UnnestOperator) subOp).getVariables();
+                } else {
+                    break;
+                }
+
+                //Go through variables in assign to check for match
+                for (int varIndex = 0; varIndex < varList.size(); varIndex++) {
+                    LogicalVariable var = varList.get(varIndex);
+                    ArrayList<LogicalVariable> parentVars = new ArrayList<>();
+                    expr.getUsedVariables(parentVars);
+
+                    if (parentVars.contains(var)) {
+                        //Found the variable we are looking for.
+                        //return assign and index of expression
+                        int[] returnValues = { i, varIndex };
+                        assignAndExpressionIndexes = returnValues;
+                    }
+                }
+            }
+            if (assignAndExpressionIndexes != null && assignAndExpressionIndexes[0] > -1) {
+                //We found the nested assign
+
+                //Recursive call on nested assign
+                List<String> parentFieldNames = getFieldNameFromSubTree(optFuncExpr, subTree,
+                        assignAndExpressionIndexes[0], assignAndExpressionIndexes[1], recordType, funcVarIndex,
+                        parentFuncExpr, metaType, metaVar, fieldSource, isUnnestOverVarAllowed);
+
+                boolean isPreviousOperatorLegalUnnest = isUnnestOverVarAllowed && subTree.getAssignsAndUnnests()
+                        .get(assignAndExpressionIndexes[0]).getOperatorTag().equals(LogicalOperatorTag.UNNEST);
+                if (parentFieldNames.isEmpty() && !isPreviousOperatorLegalUnnest) {
+                    //Nested assign was not a field access.
+                    //We will not use index
+                    return Collections.emptyList();
+                } else if (isPreviousOperatorLegalUnnest) {
+                    parentFieldNames = new ArrayList<>();
+                }
+
+                if (!isByName) {
+                    IAType subFieldType;
+                    if (isUnnestOverVarAllowed && isPreviousOperatorLegalUnnest) {
+                        // In the case of UNNESTing over a variable, we use the record type given by our caller instead.
+                        subFieldType = sourceVar.equals(metaVar) ? metaType : recordType;
+                    } else {
+                        subFieldType = sourceVar.equals(metaVar) ? metaType.getSubFieldType(parentFieldNames)
+                                : recordType.getSubFieldType(parentFieldNames);
+                        // Sub-field type can be AUnionType in case if optional. Thus, needs to get the actual type.
+                        subFieldType = TypeComputeUtils.getActualType(subFieldType);
+                        if (subFieldType.getTypeTag() != ATypeTag.OBJECT) {
+                            throw CompilationException.create(ErrorCode.TYPE_CONVERT, subFieldType,
+                                    ARecordType.class.getName());
+                        }
+                    }
+                    fieldName = ((ARecordType) subFieldType).getFieldNames()[fieldIndex];
+
+                }
+                if (optFuncExpr != null) {
+                    optFuncExpr.setSourceVar(funcVarIndex, ((AssignOperator) op).getVariables().get(assignVarIndex));
+                }
+                //add fieldName to the nested fieldName, return
+                if (nestedAccessFieldName != null) {
+                    for (int i = 0; i < nestedAccessFieldName.size(); i++) {
+                        parentFieldNames.add(nestedAccessFieldName.get(i));
+                    }
+                } else {
+                    parentFieldNames.add(fieldName);
+                }
+                return (parentFieldNames);
+            }
+
+            if (optFuncExpr != null) {
+                optFuncExpr.setSourceVar(funcVarIndex, ((AssignOperator) op).getVariables().get(assignVarIndex));
+            }
+            //no nested assign, we are at the lowest level.
+            if (isByName) {
+                if (nestedAccessFieldName != null) {
+                    return nestedAccessFieldName;
+                }
+                return new ArrayList<>(Arrays.asList(fieldName));
+            }
+            return new ArrayList<>(Arrays.asList(sourceVar.equals(metaVar) ? metaType.getFieldNames()[fieldIndex]
+                    : recordType.getFieldNames()[fieldIndex]));
+
+        }
+
+        if (!funcIDSetThatRetainFieldName.contains(funcIdent)) {
+            return Collections.emptyList();
+        }
+        // We use a part of the field in edit distance computation
+        if (optFuncExpr != null
+                && optFuncExpr.getFuncExpr().getFunctionIdentifier() == BuiltinFunctions.EDIT_DISTANCE_CHECK) {
+            optFuncExpr.setPartialField(true);
+        }
+        // We expect the function's argument to be a variable, otherwise we
+        // cannot apply an index.
+        ILogicalExpression argExpr = funcExpr.getArguments().get(0).getValue();
+        if (argExpr.getExpressionTag() != LogicalExpressionTag.VARIABLE) {
+            return Collections.emptyList();
+        }
+        LogicalVariable curVar = ((VariableReferenceExpression) argExpr).getVariableReference();
+        // We look for the assign or unnest operator that produces curVar below
+        // the current operator
+        for (int assignOrUnnestIndex = opIndex + 1; assignOrUnnestIndex < subTree.getAssignsAndUnnests()
+                .size(); assignOrUnnestIndex++) {
+            AbstractLogicalOperator curOp = subTree.getAssignsAndUnnests().get(assignOrUnnestIndex);
+            if (curOp.getOperatorTag() == LogicalOperatorTag.ASSIGN) {
+                AssignOperator assignOp = (AssignOperator) curOp;
+                List<LogicalVariable> varList = assignOp.getVariables();
+                for (int varIndex = 0; varIndex < varList.size(); varIndex++) {
+                    LogicalVariable var = varList.get(varIndex);
+                    if (var.equals(curVar) && optFuncExpr != null) {
+                        optFuncExpr.setSourceVar(funcVarIndex, var);
+                        return getFieldNameFromSubTree(optFuncExpr, subTree, assignOrUnnestIndex, varIndex, recordType,
+                                funcVarIndex, childFuncExpr, metaType, metaVar, fieldSource, isUnnestOverVarAllowed);
+                    }
+                }
+            } else {
+                UnnestOperator unnestOp = (UnnestOperator) curOp;
+                LogicalVariable var = unnestOp.getVariable();
+                if (var.equals(curVar)) {
+                    getFieldNameFromSubTree(optFuncExpr, subTree, assignOrUnnestIndex, 0, recordType, funcVarIndex,
+                            childFuncExpr, metaType, metaVar, fieldSource, isUnnestOverVarAllowed);
+                }
+            }
+        }
+        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 {
+
+        for (Index index : datasetIndexes) {
+            if (index.getIndexType() != IndexType.ARRAY) {
+                continue;
+            }
+            Index.ArrayIndexDetails arrayIndexDetails = (Index.ArrayIndexDetails) index.getIndexDetails();
+            for (Index.ArrayIndexElement e : arrayIndexDetails.getElementList()) {
+                if (e.getUnnestList().isEmpty()) {
+                    // Ignore the atomic part of this index (these are handled by the caller).
+                    continue;
+                }
+
+                // 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);
+
+                    // Do not match a variable that we have previously matched.
+                    if (matchedFuncExprs.stream().anyMatch(f -> f.findFieldName(flattenedFieldName) != -1)) {
+                        continue;
+                    }
+
+                    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;
+
+                        // Remember matching subtree.
+                        optFuncExpr.setOptimizableSubTree(optVarIndex, subTree);
+                        MutableInt fieldSource = new MutableInt(0);
+                        optFuncExpr.setFieldName(optVarIndex, fieldName, fieldSource.intValue());
+                        optFuncExpr.setFieldType(optVarIndex, fieldType);
+                        IAType type = (IAType) context.getOutputTypeEnvironment(subTree.getRoot())
+                                .getVarType(optFuncExpr.getLogicalVar(optVarIndex));
+                        optFuncExpr.setFieldType(optVarIndex, type);
+
+                        return fieldTriplet;
+                    }
+                }
+            }
+        }
+
+        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();
+
+        // TODO: (GLENN) Refactor this to use ArrayIndexUtil.
+        // 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/ArrayBTreeAccessMethod.java b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/am/ArrayBTreeAccessMethod.java
new file mode 100644
index 0000000..44e4a18
--- /dev/null
+++ b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/am/ArrayBTreeAccessMethod.java
@@ -0,0 +1,133 @@
+/*
+ * 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.List;
+
+import org.apache.asterix.common.config.DatasetConfig.IndexType;
+import org.apache.asterix.common.exceptions.CompilationException;
+import org.apache.asterix.common.exceptions.ErrorCode;
+import org.apache.asterix.metadata.entities.Index;
+import org.apache.asterix.metadata.utils.ArrayIndexUtil;
+import org.apache.asterix.om.types.IAType;
+import org.apache.asterix.om.utils.NonTaggedFormatUtil;
+import org.apache.commons.lang3.mutable.Mutable;
+import org.apache.hyracks.algebricks.common.exceptions.AlgebricksException;
+import org.apache.hyracks.algebricks.core.algebra.base.ILogicalExpression;
+import org.apache.hyracks.algebricks.core.algebra.base.ILogicalOperator;
+import org.apache.hyracks.algebricks.core.algebra.base.IOptimizationContext;
+import org.apache.hyracks.algebricks.core.algebra.base.LogicalVariable;
+
+// TODO (GLENN): Refactor the BTreeAccessMethod class and this class to extend a new "AbstractBTreeAccessMethod" class.
+/**
+ * Class for helping rewrite rules to choose and apply array BTree indexes.
+ */
+public class ArrayBTreeAccessMethod extends BTreeAccessMethod {
+    public static final ArrayBTreeAccessMethod INSTANCE = new ArrayBTreeAccessMethod();
+
+    @Override
+    public boolean matchAllIndexExprs(Index index) {
+        // Similar to BTree "matchAllIndexExprs", we only require all expressions to be matched if this is a composite
+        // key index with an unknowable field.
+        return ((Index.ArrayIndexDetails) index.getIndexDetails()).getElementList().stream()
+                .map(e -> e.getProjectList().size()).reduce(0, Integer::sum) > 1 && hasUnknownableField(index);
+    }
+
+    @Override
+    public boolean matchPrefixIndexExprs(Index index) {
+        return !matchAllIndexExprs(index);
+    }
+
+    private boolean hasUnknownableField(Index index) {
+        if (index.isSecondaryIndex() && index.getIndexDetails().isOverridingKeyFieldTypes() && !index.isEnforced()) {
+            return true;
+        }
+        for (Index.ArrayIndexElement e : ((Index.ArrayIndexDetails) index.getIndexDetails()).getElementList()) {
+            for (int i = 0; i < e.getProjectList().size(); i++) {
+                if (NonTaggedFormatUtil.isOptional(e.getTypeList().get(i))) {
+                    return true;
+                }
+            }
+        }
+        return false;
+    }
+
+    @Override
+    public ILogicalOperator createIndexSearchPlan(List<Mutable<ILogicalOperator>> afterTopOpRefs,
+            Mutable<ILogicalOperator> topOpRef, Mutable<ILogicalExpression> conditionRef,
+            List<Mutable<ILogicalOperator>> assignBeforeTheOpRefs, OptimizableOperatorSubTree indexSubTree,
+            OptimizableOperatorSubTree probeSubTree, Index chosenIndex, AccessMethodAnalysisContext analysisCtx,
+            boolean retainInput, boolean retainMissing, boolean requiresBroadcast, IOptimizationContext context,
+            LogicalVariable newMissingPlaceHolderForLOJ) throws AlgebricksException {
+
+        Index.ArrayIndexDetails chosenIndexDetails = (Index.ArrayIndexDetails) chosenIndex.getIndexDetails();
+        List<List<String>> chosenIndexKeyFieldNames = new ArrayList<>();
+        List<IAType> chosenIndexKeyFieldTypes = new ArrayList<>();
+        List<Integer> chosenIndexKeyFieldSourceIndicators = new ArrayList<>();
+        for (Index.ArrayIndexElement e : chosenIndexDetails.getElementList()) {
+            for (int i = 0; i < e.getProjectList().size(); i++) {
+                chosenIndexKeyFieldNames
+                        .add(ArrayIndexUtil.getFlattenedKeyFieldNames(e.getUnnestList(), e.getProjectList().get(i)));
+                chosenIndexKeyFieldTypes.add(e.getTypeList().get(i));
+                chosenIndexKeyFieldSourceIndicators.add(e.getSourceIndicator());
+            }
+        }
+
+        return createBTreeIndexSearchPlan(afterTopOpRefs, topOpRef, conditionRef, assignBeforeTheOpRefs, indexSubTree,
+                probeSubTree, chosenIndex, analysisCtx, retainInput, retainMissing, requiresBroadcast, context,
+                newMissingPlaceHolderForLOJ, chosenIndexKeyFieldNames, chosenIndexKeyFieldTypes,
+                chosenIndexKeyFieldSourceIndicators);
+    }
+
+    @Override
+    protected IAType getIndexedKeyType(Index.IIndexDetails chosenIndexDetails, int keyPos) throws CompilationException {
+        // TODO (GLENN): This assumes a flattened key list. Refactor / clarify this when removing depth indicators.
+        Index.ArrayIndexDetails arrayIndexDetails = (Index.ArrayIndexDetails) chosenIndexDetails;
+        int elementPos = 0;
+        for (Index.ArrayIndexElement e : arrayIndexDetails.getElementList()) {
+            for (int i = 0; i < e.getProjectList().size(); i++) {
+                if (elementPos == keyPos) {
+                    return e.getTypeList().get(i);
+                }
+                elementPos++;
+            }
+        }
+
+        throw new CompilationException(ErrorCode.COMPILATION_ILLEGAL_STATE,
+                "No array index element found, but using " + "an array access method.");
+    }
+
+    @Override
+    public boolean matchIndexType(IndexType indexType) {
+        return indexType == IndexType.ARRAY;
+    }
+
+    @Override
+    public String getName() {
+        return "ARRAY_BTREE_ACCESS_METHOD";
+    }
+
+    @Override
+    public int compareTo(IAccessMethod o) {
+        return this.getName().compareTo(o.getName());
+    }
+
+}
diff --git a/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/am/BTreeAccessMethod.java b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/am/BTreeAccessMethod.java
index 21892cc..cad5f12 100644
--- a/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/am/BTreeAccessMethod.java
+++ b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/am/BTreeAccessMethod.java
@@ -96,7 +96,8 @@
     // That is, this function can produce false positive results if it is set to true.
     // In this case, an index-search alone cannot replace the given SELECT condition and
     // that SELECT condition needs to be applied after the index-search to get the correct results.
-    // For B+Tree indexes, there are no false positive results unless the given index is a composite index.
+    // For B+Tree indexes, there are no false positive results unless the given index is a composite index or an array
+    // index.
     private static final List<Pair<FunctionIdentifier, Boolean>> FUNC_IDENTIFIERS = Collections
             .unmodifiableList(Arrays.asList(new Pair<FunctionIdentifier, Boolean>(AlgebricksBuiltinFunctions.EQ, false),
                     new Pair<FunctionIdentifier, Boolean>(AlgebricksBuiltinFunctions.LE, false),
@@ -127,7 +128,8 @@
     public boolean matchAllIndexExprs(Index index) {
         // require all expressions to be matched if this is a composite key index which has an unknownable key field.
         // because we only add a tuple to the index if all its key fields are not null/missing.
-        return index.getKeyFieldTypes().size() > 1 && hasUnknownableField(index);
+        return ((Index.ValueIndexDetails) index.getIndexDetails()).getKeyFieldTypes().size() > 1
+                && hasUnknownableField(index);
     }
 
     @Override
@@ -136,10 +138,10 @@
     }
 
     private boolean hasUnknownableField(Index index) {
-        if (index.isSecondaryIndex() && index.isOverridingKeyFieldTypes() && !index.isEnforced()) {
+        if (index.isSecondaryIndex() && index.getIndexDetails().isOverridingKeyFieldTypes() && !index.isEnforced()) {
             return true;
         }
-        for (IAType fieldType : index.getKeyFieldTypes()) {
+        for (IAType fieldType : ((Index.ValueIndexDetails) index.getIndexDetails()).getKeyFieldTypes()) {
             if (NonTaggedFormatUtil.isOptional(fieldType)) {
                 return true;
             }
@@ -320,6 +322,26 @@
             OptimizableOperatorSubTree probeSubTree, Index chosenIndex, AccessMethodAnalysisContext analysisCtx,
             boolean retainInput, boolean retainMissing, boolean requiresBroadcast, IOptimizationContext context,
             LogicalVariable newMissingPlaceHolderForLOJ) throws AlgebricksException {
+
+        Index.ValueIndexDetails chosenIndexDetails = (Index.ValueIndexDetails) chosenIndex.getIndexDetails();
+        List<List<String>> chosenIndexKeyFieldNames = chosenIndexDetails.getKeyFieldNames();
+        List<IAType> chosenIndexKeyFieldTypes = chosenIndexDetails.getKeyFieldTypes();
+        List<Integer> chosenIndexKeyFieldSourceIndicators = chosenIndexDetails.getKeyFieldSourceIndicators();
+
+        return createBTreeIndexSearchPlan(afterTopOpRefs, topOpRef, conditionRef, assignBeforeTheOpRefs, indexSubTree,
+                probeSubTree, chosenIndex, analysisCtx, retainInput, retainMissing, requiresBroadcast, context,
+                newMissingPlaceHolderForLOJ, chosenIndexKeyFieldNames, chosenIndexKeyFieldTypes,
+                chosenIndexKeyFieldSourceIndicators);
+    }
+
+    protected ILogicalOperator createBTreeIndexSearchPlan(List<Mutable<ILogicalOperator>> afterTopOpRefs,
+            Mutable<ILogicalOperator> topOpRef, Mutable<ILogicalExpression> conditionRef,
+            List<Mutable<ILogicalOperator>> assignBeforeTheOpRefs, OptimizableOperatorSubTree indexSubTree,
+            OptimizableOperatorSubTree probeSubTree, Index chosenIndex, AccessMethodAnalysisContext analysisCtx,
+            boolean retainInput, boolean retainMissing, boolean requiresBroadcast, IOptimizationContext context,
+            LogicalVariable newMissingPlaceHolderForLOJ, List<List<String>> chosenIndexKeyFieldNames,
+            List<IAType> chosenIndexKeyFieldTypes, List<Integer> chosenIndexKeyFieldSourceIndicators)
+            throws AlgebricksException {
         Dataset dataset = indexSubTree.getDataset();
         ARecordType recordType = indexSubTree.getRecordType();
         ARecordType metaRecordType = indexSubTree.getMetaRecordType();
@@ -370,12 +392,12 @@
         for (Pair<Integer, Integer> exprIndex : exprAndVarList) {
             // Position of the field of matchedFuncExprs.get(exprIndex) in the chosen index's indexed exprs.
             IOptimizableFuncExpr optFuncExpr = analysisCtx.getMatchedFuncExpr(exprIndex.first);
-            int keyPos = indexOf(optFuncExpr.getFieldName(0), optFuncExpr.getFieldSource(0),
-                    chosenIndex.getKeyFieldNames(), chosenIndex.getKeyFieldSourceIndicators());
+            int keyPos = indexOf(optFuncExpr.getFieldName(0), optFuncExpr.getFieldSource(0), chosenIndexKeyFieldNames,
+                    chosenIndexKeyFieldSourceIndicators);
             if (keyPos < 0 && optFuncExpr.getNumLogicalVars() > 1) {
                 // If we are optimizing a join, the matching field may be the second field name.
-                keyPos = indexOf(optFuncExpr.getFieldName(1), optFuncExpr.getFieldSource(1),
-                        chosenIndex.getKeyFieldNames(), chosenIndex.getKeyFieldSourceIndicators());
+                keyPos = indexOf(optFuncExpr.getFieldName(1), optFuncExpr.getFieldSource(1), chosenIndexKeyFieldNames,
+                        chosenIndexKeyFieldSourceIndicators);
             }
             if (keyPos < 0) {
                 throw CompilationException.create(ErrorCode.NO_INDEX_FIELD_NAME_FOR_GIVEN_FUNC_EXPR,
@@ -385,7 +407,7 @@
             // The second expression will not be null only if we are creating an EQ search predicate
             // with a FLOAT or a DOUBLE constant that will be fed into an INTEGER index.
             // This is required because of type-casting. Refer to AccessMethodUtils.createSearchKeyExpr for details.
-            IAType indexedFieldType = chosenIndex.getKeyFieldTypes().get(keyPos);
+            IAType indexedFieldType = chosenIndexKeyFieldTypes.get(keyPos);
             Triple<ILogicalExpression, ILogicalExpression, Boolean> returnedSearchKeyExpr =
                     AccessMethodUtils.createSearchKeyExpr(chosenIndex, optFuncExpr, indexedFieldType, probeSubTree);
             ILogicalExpression searchKeyExpr = returnedSearchKeyExpr.first;
@@ -669,7 +691,7 @@
             indexSearchOp = AccessMethodUtils.createRestOfIndexSearchPlan(afterTopOpRefs, topOpRef, conditionRef,
                     assignBeforeTheOpRefs, dataSourceOp, dataset, recordType, metaRecordType, secondaryIndexUnnestOp,
                     context, true, retainInput, retainMissing, false, chosenIndex, analysisCtx, indexSubTree,
-                    newMissingPlaceHolderForLOJ);
+                    probeSubTree, newMissingPlaceHolderForLOJ);
 
             // Replaces the datasource scan with the new plan rooted at
             // Get dataSourceRef operator -
@@ -895,7 +917,8 @@
         return limit;
     }
 
-    private boolean relaxLimitTypeToInclusive(Index chosenIndex, int keyPos, boolean realTypeConvertedToIntegerType) {
+    private boolean relaxLimitTypeToInclusive(Index chosenIndex, int keyPos, boolean realTypeConvertedToIntegerType)
+            throws CompilationException {
         // For a non-enforced index or an enforced index that stores a casted value on the given index,
         // we need to apply the following transformation.
         // For an index on a closed field, this transformation is not necessary since the value between
@@ -922,8 +945,8 @@
             return true;
         }
 
-        if (chosenIndex.isOverridingKeyFieldTypes() && !chosenIndex.isEnforced()) {
-            IAType indexedKeyType = chosenIndex.getKeyFieldTypes().get(keyPos);
+        if (chosenIndex.getIndexDetails().isOverridingKeyFieldTypes() && !chosenIndex.isEnforced()) {
+            IAType indexedKeyType = getIndexedKeyType(chosenIndex.getIndexDetails(), keyPos);
             if (NonTaggedFormatUtil.isOptional(indexedKeyType)) {
                 indexedKeyType = ((AUnionType) indexedKeyType).getActualType();
             }
@@ -943,6 +966,10 @@
         return false;
     }
 
+    protected IAType getIndexedKeyType(Index.IIndexDetails chosenIndexDetails, int keyPos) throws CompilationException {
+        return ((Index.ValueIndexDetails) chosenIndexDetails).getKeyFieldTypes().get(keyPos);
+    }
+
     private boolean probeIsOnLhs(IOptimizableFuncExpr optFuncExpr, OptimizableOperatorSubTree probeSubTree) {
         if (probeSubTree == null) {
             if (optFuncExpr.getConstantExpressions().length == 0) {
@@ -1011,6 +1038,11 @@
     }
 
     @Override
+    public boolean matchIndexType(IndexType indexType) {
+        return indexType == IndexType.BTREE;
+    }
+
+    @Override
     public String getName() {
         return "BTREE_ACCESS_METHOD";
     }
diff --git a/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/am/IAccessMethod.java b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/am/IAccessMethod.java
index 84ee41e..b21cf12 100644
--- a/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/am/IAccessMethod.java
+++ b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/am/IAccessMethod.java
@@ -21,6 +21,7 @@
 import java.util.Collection;
 import java.util.List;
 
+import org.apache.asterix.common.config.DatasetConfig.IndexType;
 import org.apache.asterix.metadata.entities.Index;
 import org.apache.commons.lang3.mutable.Mutable;
 import org.apache.hyracks.algebricks.common.exceptions.AlgebricksException;
@@ -66,6 +67,14 @@
             IOptimizationContext context, IVariableTypeEnvironment typeEnvironment) throws AlgebricksException;
 
     /**
+     * Indicates whether this access method is applicable for the given index type.
+     *
+     * @return boolean
+     * @param indexType
+     */
+    public boolean matchIndexType(IndexType indexType);
+
+    /**
      * Indicates whether all index expressions must be matched in order for this
      * index to be applicable.
      *
diff --git a/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/am/IntroduceJoinAccessMethodRule.java b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/am/IntroduceJoinAccessMethodRule.java
index 199f878..6a5964d 100644
--- a/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/am/IntroduceJoinAccessMethodRule.java
+++ b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/am/IntroduceJoinAccessMethodRule.java
@@ -94,6 +94,7 @@
     protected static Map<FunctionIdentifier, List<IAccessMethod>> accessMethods = new HashMap<>();
 
     static {
+        registerAccessMethod(ArrayBTreeAccessMethod.INSTANCE, accessMethods);
         registerAccessMethod(BTreeAccessMethod.INSTANCE, accessMethods);
         registerAccessMethod(RTreeAccessMethod.INSTANCE, accessMethods);
         registerAccessMethod(InvertedIndexAccessMethod.INSTANCE, accessMethods);
diff --git a/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/am/IntroduceLSMComponentFilterRule.java b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/am/IntroduceLSMComponentFilterRule.java
index bd5d82a..ab0ae5f 100644
--- a/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/am/IntroduceLSMComponentFilterRule.java
+++ b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/am/IntroduceLSMComponentFilterRule.java
@@ -34,6 +34,7 @@
 import org.apache.asterix.metadata.declared.MetadataProvider;
 import org.apache.asterix.metadata.entities.Dataset;
 import org.apache.asterix.metadata.entities.Index;
+import org.apache.asterix.metadata.utils.ArrayIndexUtil;
 import org.apache.asterix.metadata.utils.DatasetUtil;
 import org.apache.asterix.metadata.utils.KeyFieldTypeUtil;
 import org.apache.asterix.om.functions.BuiltinFunctions;
@@ -121,7 +122,7 @@
 
             for (int i = 0; i < analysisCtx.getMatchedFuncExprs().size(); i++) {
                 IOptimizableFuncExpr optFuncExpr = analysisCtx.getMatchedFuncExpr(i);
-                boolean found = findMacthedExprFieldName(optFuncExpr, op, dataset, itemType, datasetIndexes, context,
+                boolean found = findMatchedExprFieldName(optFuncExpr, op, dataset, itemType, datasetIndexes, context,
                         filterSourceIndicator);
                 // the field name source should be consistent with the filter source indicator
                 if (found && optFuncExpr.getFieldName(0).equals(filterFieldName)
@@ -212,7 +213,9 @@
                     AbstractFunctionCallExpression f = (AbstractFunctionCallExpression) unnestExpr;
                     FunctionIdentifier fid = f.getFunctionIdentifier();
                     if (!fid.equals(BuiltinFunctions.INDEX_SEARCH)) {
-                        throw new IllegalStateException();
+                        throw new CompilationException(ErrorCode.COMPILATION_ILLEGAL_STATE,
+                                unnestMapOp.getSourceLocation(),
+                                "Illegal function found, expected an " + "index-search.");
                     }
                     AccessMethodJobGenParams jobGenParams = new AccessMethodJobGenParams();
                     jobGenParams.readFromFuncArgs(f.getArguments());
@@ -308,14 +311,17 @@
                     }
                     break;
                 case ORDER:
+                case DISTINCT:
                     ILogicalOperator child = intersectOrSortOrSplit.getValue().getInputs().get(0).getValue();
                     if (child.getOperatorTag().equals(LogicalOperatorTag.UNNEST_MAP)) {
                         UnnestMapOperator secondaryMap = (UnnestMapOperator) child;
 
-                        propagateFilterInSecondaryUnnsetMap(secondaryMap, filterType, context);
-
-                        setPrimaryFilterVar(primaryOp, secondaryMap.getPropagateIndexMinFilterVar(),
-                                secondaryMap.getPropagateIndexMaxFilterVar(), context);
+                        // If we are already propagating our index filter, do not repeat this action.
+                        if (!secondaryMap.propagateIndexFilter()) {
+                            propagateFilterInSecondaryUnnsetMap(secondaryMap, filterType, context);
+                            setPrimaryFilterVar(primaryOp, secondaryMap.getPropagateIndexMinFilterVar(),
+                                    secondaryMap.getPropagateIndexMaxFilterVar(), context);
+                        }
                     }
                     break;
 
@@ -495,7 +501,7 @@
         }
     }
 
-    private boolean findMacthedExprFieldName(IOptimizableFuncExpr optFuncExpr, AbstractLogicalOperator op,
+    private boolean findMatchedExprFieldName(IOptimizableFuncExpr optFuncExpr, AbstractLogicalOperator op,
             Dataset dataset, ARecordType filterSourceType, List<Index> datasetIndexes, IOptimizationContext context,
             Integer filterSourceIndicator) throws AlgebricksException {
         AbstractLogicalOperator descendantOp = (AbstractLogicalOperator) op.getInputs().get(0).getValue();
@@ -557,7 +563,9 @@
                         AbstractFunctionCallExpression f = (AbstractFunctionCallExpression) unnestExpr;
                         FunctionIdentifier fid = f.getFunctionIdentifier();
                         if (!fid.equals(BuiltinFunctions.INDEX_SEARCH)) {
-                            throw new IllegalStateException();
+                            throw new CompilationException(ErrorCode.COMPILATION_ILLEGAL_STATE,
+                                    unnestMapOp.getSourceLocation(),
+                                    "Illegal function found, expected an " + "index-search.");
                         }
                         AccessMethodJobGenParams jobGenParams = new AccessMethodJobGenParams();
                         jobGenParams.readFromFuncArgs(f.getArguments());
@@ -569,6 +577,11 @@
                             }
                         }
                     }
+                    if (index == null) {
+                        throw new CompilationException(ErrorCode.COMPILATION_ILLEGAL_STATE,
+                                unnestMapOp.getSourceLocation(),
+                                "Could not find the corresponding index for an" + " index search.");
+                    }
 
                     IAType metaItemType = ((MetadataProvider) context.getMetadataProvider())
                             .findType(dataset.getMetaItemTypeDataverseName(), dataset.getMetaItemTypeName());
@@ -584,8 +597,40 @@
                         fieldName = dataset.getPrimaryKeys().get(idx);
                         keySource = getKeySource(DatasetUtil.getKeySourceIndicators(dataset), idx);
                     } else {
-                        fieldName = index.getKeyFieldNames().get(varIndex);
-                        keySource = getKeySource(index.getKeyFieldSourceIndicators(), varIndex);
+                        List<List<String>> keyFieldNames;
+                        List<Integer> keySources;
+                        switch (Index.IndexCategory.of(index.getIndexType())) {
+                            case ARRAY:
+                                Index.ArrayIndexDetails arrayIndexDetails =
+                                        (Index.ArrayIndexDetails) index.getIndexDetails();
+                                keyFieldNames = new ArrayList<>();
+                                keySources = new ArrayList<>();
+                                for (Index.ArrayIndexElement e : arrayIndexDetails.getElementList()) {
+                                    for (List<String> project : e.getProjectList()) {
+                                        keyFieldNames.add(
+                                                ArrayIndexUtil.getFlattenedKeyFieldNames(e.getUnnestList(), project));
+                                        keySources.add(e.getSourceIndicator());
+                                    }
+                                }
+                                break;
+                            case VALUE:
+                                Index.ValueIndexDetails valueIndexDetails =
+                                        (Index.ValueIndexDetails) index.getIndexDetails();
+                                keyFieldNames = valueIndexDetails.getKeyFieldNames();
+                                keySources = valueIndexDetails.getKeyFieldSourceIndicators();
+                                break;
+                            case TEXT:
+                                Index.TextIndexDetails textIndexDetails =
+                                        (Index.TextIndexDetails) index.getIndexDetails();
+                                keyFieldNames = textIndexDetails.getKeyFieldNames();
+                                keySources = textIndexDetails.getKeyFieldSourceIndicators();
+                                break;
+                            default:
+                                throw new CompilationException(ErrorCode.COMPILATION_UNKNOWN_INDEX_TYPE,
+                                        String.valueOf(index.getIndexType()));
+                        }
+                        fieldName = keyFieldNames.get(varIndex);
+                        keySource = getKeySource(keySources, varIndex);
                     }
                     if (fieldName == null) {
                         return false;
diff --git a/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/am/IntroducePrimaryIndexForAggregationRule.java b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/am/IntroducePrimaryIndexForAggregationRule.java
index 64f9068..d221fd1 100644
--- a/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/am/IntroducePrimaryIndexForAggregationRule.java
+++ b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/am/IntroducePrimaryIndexForAggregationRule.java
@@ -284,7 +284,7 @@
         // #2. get all indexes and look for the primary one
         List<Index> indexes = mp.getDatasetIndexes(dataset.getDataverseName(), dataset.getDatasetName());
         for (Index index : indexes) {
-            if (index.getKeyFieldNames().isEmpty()) {
+            if (index.isPrimaryKeyIndex()) {
                 return Pair.of(dataset, index);
             }
         }
diff --git a/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/am/IntroduceSelectAccessMethodRule.java b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/am/IntroduceSelectAccessMethodRule.java
index 4199ece..c64e517 100644
--- a/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/am/IntroduceSelectAccessMethodRule.java
+++ b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/am/IntroduceSelectAccessMethodRule.java
@@ -30,6 +30,7 @@
 import org.apache.asterix.common.exceptions.ErrorCode;
 import org.apache.asterix.metadata.declared.MetadataProvider;
 import org.apache.asterix.metadata.entities.Index;
+import org.apache.asterix.optimizer.rules.util.SelectInSubplanBranchCreator;
 import org.apache.commons.lang3.mutable.Mutable;
 import org.apache.commons.lang3.mutable.MutableObject;
 import org.apache.hyracks.algebricks.common.exceptions.AlgebricksException;
@@ -122,11 +123,13 @@
     protected IVariableTypeEnvironment typeEnvironment = null;
     protected final OptimizableOperatorSubTree subTree = new OptimizableOperatorSubTree();
     protected List<Mutable<ILogicalOperator>> afterSelectRefs = null;
+    private final SelectInSubplanBranchCreator selectInSubplanBranchCreator = new SelectInSubplanBranchCreator();
 
     // Register access methods.
     protected static Map<FunctionIdentifier, List<IAccessMethod>> accessMethods = new HashMap<>();
 
     static {
+        registerAccessMethod(ArrayBTreeAccessMethod.INSTANCE, accessMethods);
         registerAccessMethod(BTreeAccessMethod.INSTANCE, accessMethods);
         registerAccessMethod(RTreeAccessMethod.INSTANCE, accessMethods);
         registerAccessMethod(InvertedIndexAccessMethod.INSTANCE, accessMethods);
@@ -370,6 +373,25 @@
                 analyzedAMs = new TreeMap<>();
             }
 
+            // If there exists a SUBPLAN in our plan, and we are conditioning on a variable,
+            // attempt to rewrite this subplan to allow an array-index AM to be introduced.
+            // This rewrite is to be used **solely** for the purpose of changing a DATA-SCAN into a
+            // non-index-only plan branch. No nodes from this rewrite will be used beyond this point. 
+            // If successful, this will create a non-index only plan that replaces the subplan's
+            // DATA-SCAN with a PIDX SEARCH <- DISTINCT <- ORDER <- SIDX SEARCH.
+            if (continueCheck && context.getPhysicalOptimizationConfig().isArrayIndexEnabled()) {
+                SelectOperator selectRewrite = selectInSubplanBranchCreator.createSelect(selectOp, context);
+                if (selectRewrite != null
+                        && checkAndApplyTheSelectTransformation(new MutableObject<>(selectRewrite), context)) {
+                    return true;
+
+                } else {
+                    // If this optimization or temp-branch creation was not successful, restore our state.
+                    selectRef = selectRefFromThisOp;
+                    selectOp = selectInSubplanBranchCreator.getOriginalSelect();
+                }
+            }
+
             // Check the condition of SELECT operator is a function call since
             // only function call can be transformed using available indexes.
             // If so, initialize the subtree information that will be used later to decide whether
diff --git a/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/am/InvertedIndexAccessMethod.java b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/am/InvertedIndexAccessMethod.java
index 31245fb..8530dee 100644
--- a/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/am/InvertedIndexAccessMethod.java
+++ b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/am/InvertedIndexAccessMethod.java
@@ -455,7 +455,7 @@
         ILogicalOperator primaryIndexUnnestOp = AccessMethodUtils.createRestOfIndexSearchPlan(afterTopOpRefs, topOpRef,
                 conditionRef, assignBeforeTopOpRefs, dataSourceScan, dataset, recordType, metaRecordType,
                 secondaryIndexUnnestOp, context, true, retainInput, retainNull, false, chosenIndex, analysisCtx,
-                indexSubTree, newNullPlaceHolderForLOJ);
+                indexSubTree, null, newNullPlaceHolderForLOJ);
 
         return primaryIndexUnnestOp;
     }
@@ -801,8 +801,8 @@
                 isFilterableArgs.add(new MutableObject<ILogicalExpression>(inputSearchVarRef));
                 // Since we are optimizing a join, the similarity threshold should be the only constant in the optimizable function expression.
                 isFilterableArgs.add(new MutableObject<ILogicalExpression>(optFuncExpr.getConstantExpr(0)));
-                isFilterableArgs.add(new MutableObject<ILogicalExpression>(
-                        AccessMethodUtils.createInt32Constant(chosenIndex.getGramLength())));
+                isFilterableArgs.add(new MutableObject<ILogicalExpression>(AccessMethodUtils.createInt32Constant(
+                        ((Index.TextIndexDetails) chosenIndex.getIndexDetails()).getGramLength())));
                 boolean usePrePost = optFuncExpr.containsPartialField() ? false : true;
                 isFilterableArgs.add(
                         new MutableObject<ILogicalExpression>(AccessMethodUtils.createBooleanConstant(usePrePost)));
@@ -977,7 +977,9 @@
 
     private boolean isEditDistanceFuncJoinOptimizable(Index index, IOptimizableFuncExpr optFuncExpr) {
         if (index.isEnforced()) {
-            return isEditDistanceFuncCompatible(index.getKeyFieldTypes().get(0).getTypeTag(), index.getIndexType());
+            return isEditDistanceFuncCompatible(
+                    ((Index.TextIndexDetails) index.getIndexDetails()).getKeyFieldTypes().get(0).getTypeTag(),
+                    index.getIndexType());
         } else {
             return isEditDistanceFuncCompatible(optFuncExpr.getFieldType(0).getTypeTag(), index.getIndexType());
         }
@@ -1026,13 +1028,14 @@
 
         if (typeTag == ATypeTag.STRING) {
             AString astr = (AString) listOrStrObj;
+            int gramLength = ((Index.TextIndexDetails) index.getIndexDetails()).getGramLength();
             // Compute merge threshold depending on the query grams contain pre- and postfixing
             if (optFuncExpr.containsPartialField()) {
-                mergeThreshold = (astr.getStringValue().length() - index.getGramLength() + 1)
-                        - edThresh.getIntegerValue() * index.getGramLength();
+                mergeThreshold =
+                        (astr.getStringValue().length() - gramLength + 1) - edThresh.getIntegerValue() * gramLength;
             } else {
-                mergeThreshold = (astr.getStringValue().length() + index.getGramLength() - 1)
-                        - edThresh.getIntegerValue() * index.getGramLength();
+                mergeThreshold =
+                        (astr.getStringValue().length() + gramLength - 1) - edThresh.getIntegerValue() * gramLength;
             }
         }
 
@@ -1120,7 +1123,9 @@
 
     private boolean isFullTextContainsFuncJoinOptimizable(Index index, IOptimizableFuncExpr optFuncExpr) {
         if (index.isEnforced()) {
-            return isFullTextContainsFuncCompatible(index.getKeyFieldTypes().get(0).getTypeTag(), index.getIndexType());
+            return isFullTextContainsFuncCompatible(
+                    ((Index.TextIndexDetails) index.getIndexDetails()).getKeyFieldTypes().get(0).getTypeTag(),
+                    index.getIndexType());
         } else {
             return isFullTextContainsFuncCompatible(optFuncExpr.getFieldType(0).getTypeTag(), index.getIndexType());
         }
@@ -1212,7 +1217,7 @@
         // Check that the constant search string has at least gramLength characters.
         if (strObj.getType().getTypeTag() == ATypeTag.STRING) {
             AString astr = (AString) strObj;
-            if (astr.getStringValue().length() >= index.getGramLength()) {
+            if (astr.getStringValue().length() >= ((Index.TextIndexDetails) index.getIndexDetails()).getGramLength()) {
                 return true;
             }
         }
@@ -1221,7 +1226,9 @@
 
     private boolean isContainsFuncJoinOptimizable(Index index, IOptimizableFuncExpr optFuncExpr) {
         if (index.isEnforced()) {
-            return isContainsFuncCompatible(index.getKeyFieldTypes().get(0).getTypeTag(), index.getIndexType());
+            return isContainsFuncCompatible(
+                    ((Index.TextIndexDetails) index.getIndexDetails()).getKeyFieldTypes().get(0).getTypeTag(),
+                    index.getIndexType());
         } else {
             return isContainsFuncCompatible(optFuncExpr.getFieldType(0).getTypeTag(), index.getIndexType());
         }
@@ -1249,7 +1256,7 @@
                 boolean prePost = (searchModifierType == SearchModifierType.CONJUNCTIVE
                         || searchModifierType == SearchModifierType.CONJUNCTIVE_EDIT_DISTANCE) ? false : true;
                 return BinaryTokenizerFactoryProvider.INSTANCE.getNGramTokenizerFactory(searchKeyType,
-                        index.getGramLength(), prePost, false);
+                        ((Index.TextIndexDetails) index.getIndexDetails()).getGramLength(), prePost, false);
             }
             default: {
                 throw new CompilationException(ErrorCode.NO_TOKENIZER_FOR_TYPE, index.getIndexType());
@@ -1280,11 +1287,12 @@
                 switch (index.getIndexType()) {
                     case SINGLE_PARTITION_NGRAM_INVIX:
                     case LENGTH_PARTITIONED_NGRAM_INVIX: {
+                        int gramLength = ((Index.TextIndexDetails) index.getIndexDetails()).getGramLength();
                         // Edit distance on strings, filtered with overlapping grams.
                         if (searchModifierType == SearchModifierType.EDIT_DISTANCE) {
-                            return new EditDistanceSearchModifierFactory(index.getGramLength(), edThresh);
+                            return new EditDistanceSearchModifierFactory(gramLength, edThresh);
                         } else {
-                            return new ConjunctiveEditDistanceSearchModifierFactory(index.getGramLength(), edThresh);
+                            return new ConjunctiveEditDistanceSearchModifierFactory(gramLength, edThresh);
                         }
                     }
                     case SINGLE_PARTITION_WORD_INVIX:
@@ -1320,6 +1328,19 @@
     }
 
     @Override
+    public boolean matchIndexType(IndexType indexType) {
+        switch (indexType) {
+            case SINGLE_PARTITION_WORD_INVIX:
+            case SINGLE_PARTITION_NGRAM_INVIX:
+            case LENGTH_PARTITIONED_NGRAM_INVIX:
+            case LENGTH_PARTITIONED_WORD_INVIX:
+                return true;
+            default:
+                return false;
+        }
+    }
+
+    @Override
     public String getName() {
         return "INVERTED_INDEX_ACCESS_METHOD";
     }
diff --git a/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/am/OptimizableOperatorSubTree.java b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/am/OptimizableOperatorSubTree.java
index 3cbba0d..b765bd0 100644
--- a/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/am/OptimizableOperatorSubTree.java
+++ b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/am/OptimizableOperatorSubTree.java
@@ -72,6 +72,7 @@
     private Mutable<ILogicalOperator> rootRef = null;
     private final List<Mutable<ILogicalOperator>> assignsAndUnnestsRefs = new ArrayList<>();
     private final List<AbstractLogicalOperator> assignsAndUnnests = new ArrayList<>();
+    private final Pair<Integer, Integer> lastMatchedDataSourceVars = new Pair<>(-1, -1);
     private Mutable<ILogicalOperator> dataSourceRef = null;
     private DataSourceType dataSourceType = DataSourceType.NO_DATASOURCE;
 
@@ -389,6 +390,8 @@
         setRecordType(null);
         setMetaRecordType(null);
         setIxJoinOuterAdditionalRecordTypes(null);
+        lastMatchedDataSourceVars.first = -1;
+        lastMatchedDataSourceVars.second = -1;
     }
 
     /**
@@ -590,4 +593,12 @@
         return varsToFieldNameMap;
     }
 
+    public Pair<Integer, Integer> getLastMatchedDataSourceVars() {
+        return lastMatchedDataSourceVars;
+    }
+
+    public void setLastMatchedDataSourceVars(int varIndex, int optVarIndex) {
+        this.lastMatchedDataSourceVars.first = varIndex;
+        this.lastMatchedDataSourceVars.second = optVarIndex;
+    }
 }
diff --git a/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/am/RTreeAccessMethod.java b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/am/RTreeAccessMethod.java
index c1ae61e..b28c15e 100644
--- a/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/am/RTreeAccessMethod.java
+++ b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/am/RTreeAccessMethod.java
@@ -308,7 +308,7 @@
                         secondaryIndexUnnestOp, context, chosenIndex, retainInput, retainNull)
                 : AccessMethodUtils.createRestOfIndexSearchPlan(afterTopRefs, topRef, conditionRef, assignBeforeTopRefs,
                         dataSourceOp, dataset, recordType, metaRecordType, secondaryIndexUnnestOp, context, true,
-                        retainInput, retainNull, false, chosenIndex, analysisCtx, indexSubTree,
+                        retainInput, retainNull, false, chosenIndex, analysisCtx, indexSubTree, null,
                         newNullPlaceHolderForLOJ);
     }
 
@@ -385,6 +385,11 @@
     }
 
     @Override
+    public boolean matchIndexType(IndexType indexType) {
+        return indexType == IndexType.RTREE;
+    }
+
+    @Override
     public String getName() {
         return "RTREE_ACCESS_METHOD";
     }
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
new file mode 100644
index 0000000..6efef16
--- /dev/null
+++ b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/util/SelectInSubplanBranchCreator.java
@@ -0,0 +1,424 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.asterix.optimizer.rules.util;
+
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.List;
+
+import org.apache.asterix.common.exceptions.CompilationException;
+import org.apache.asterix.common.exceptions.ErrorCode;
+import org.apache.asterix.om.base.AInt16;
+import org.apache.asterix.om.base.AInt32;
+import org.apache.asterix.om.base.AInt64;
+import org.apache.asterix.om.base.AInt8;
+import org.apache.asterix.om.constants.AsterixConstantValue;
+import org.apache.asterix.om.functions.BuiltinFunctions;
+import org.apache.commons.lang3.mutable.Mutable;
+import org.apache.commons.lang3.mutable.MutableObject;
+import org.apache.hyracks.algebricks.common.exceptions.AlgebricksException;
+import org.apache.hyracks.algebricks.core.algebra.base.ILogicalExpression;
+import org.apache.hyracks.algebricks.core.algebra.base.ILogicalOperator;
+import org.apache.hyracks.algebricks.core.algebra.base.IOptimizationContext;
+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.AggregateFunctionCallExpression;
+import org.apache.hyracks.algebricks.core.algebra.expressions.ConstantExpression;
+import org.apache.hyracks.algebricks.core.algebra.expressions.IAlgebricksConstantValue;
+import org.apache.hyracks.algebricks.core.algebra.expressions.ScalarFunctionCallExpression;
+import org.apache.hyracks.algebricks.core.algebra.expressions.UnnestingFunctionCallExpression;
+import org.apache.hyracks.algebricks.core.algebra.expressions.VariableReferenceExpression;
+import org.apache.hyracks.algebricks.core.algebra.operators.logical.AggregateOperator;
+import org.apache.hyracks.algebricks.core.algebra.operators.logical.AssignOperator;
+import org.apache.hyracks.algebricks.core.algebra.operators.logical.SelectOperator;
+import org.apache.hyracks.algebricks.core.algebra.operators.logical.SubplanOperator;
+import org.apache.hyracks.algebricks.core.algebra.operators.logical.UnnestOperator;
+import org.apache.hyracks.algebricks.core.algebra.operators.logical.visitors.VariableUtilities;
+import org.apache.hyracks.algebricks.core.algebra.util.OperatorManipulationUtil;
+import org.apache.hyracks.api.exceptions.SourceLocation;
+
+/**
+ * For use in writing a "throwaway" branch which removes NTS and subplan operators. The result of this invocation is to
+ * be given to the {@code IntroduceSelectAccessMethodRule} to check if an array index can be used.
+ * <br>
+ * If we are given the pattern (an existential query):
+ * <pre>
+ * SELECT_1(some variable)
+ * SUBPLAN_1 -------------------------------|
+ * (parent branch input)        AGGREGATE(NON-EMPTY-STREAM)
+ *                              SELECT_2(some predicate)
+ *                              (UNNEST/ASSIGN)*
+ *                              UNNEST(on variable)
+ *                              NESTED-TUPLE-SOURCE
+ * </pre>
+ * We return the following branch:
+ * <pre>
+ * SELECT_2(some predicate)
+ * (UNNEST/ASSIGN)*
+ * UNNEST(on variable)
+ * (parent branch input)
+ * </pre>
+ *
+ * If we are given the pattern (a universal query):
+ * <pre>
+ * SELECT_1(some variable AND array is not empty)
+ * SUBPLAN_1 -------------------------------|
+ * (parent branch input)        AGGREGATE(EMPTY-STREAM)
+ *                              SELECT_2(NOT(IF-MISSING-OR-NULL(some predicate)))
+ *                              (UNNEST/ASSIGN)*
+ *                              UNNEST(on variable)
+ *                              NESTED-TUPLE-SOURCE
+ * </pre>
+ * We return the following branch:
+ * <pre>
+ * SELECT_2(some predicate)  <--- removed the NOT(IF-MISSING-OR-NULL(...))!
+ * (UNNEST/ASSIGN)*
+ * UNNEST(on variable)
+ * (parent branch input)
+ * </pre>
+ *
+ * In the case of nested-subplans, we return a copy of the innermost SELECT followed by all relevant UNNEST/ASSIGNs.
+ */
+public class SelectInSubplanBranchCreator {
+    private final static List<IAlgebricksConstantValue> zerosAsAsterixConstants =
+            Arrays.asList(new IAlgebricksConstantValue[] { new AsterixConstantValue(new AInt64(0)),
+                    new AsterixConstantValue(new AInt32(0)), new AsterixConstantValue(new AInt16((short) 0)),
+                    new AsterixConstantValue(new AInt8((byte) 0)) });
+
+    private IOptimizationContext context;
+    private SourceLocation sourceLocation;
+    private SelectOperator originalSelectRoot;
+
+    /**
+     * Create a new branch to match that of the form:
+     *
+     * <pre>
+     * SELECT (...)
+     * (UNNEST/ASSIGN)*
+     * UNNEST
+     * ...
+     * </pre>
+     *
+     * Operators are *created* here, rather than just reconnected from the original branch.
+     */
+    public SelectOperator createSelect(SelectOperator originalSelect, IOptimizationContext context)
+            throws AlgebricksException {
+        // Reset our context.
+        this.sourceLocation = originalSelect.getSourceLocation();
+        this.originalSelectRoot = originalSelect;
+        this.context = context;
+
+        // We expect a) a SUBPLAN as input to this SELECT, and b) our SELECT to be conditioning on a variable.
+        if (!originalSelect.getInputs().get(0).getValue().getOperatorTag().equals(LogicalOperatorTag.SUBPLAN)
+                || !originalSelect.getCondition().getValue().getExpressionTag().equals(LogicalExpressionTag.VARIABLE)) {
+            return null;
+        }
+        LogicalVariable originalSelectVar =
+                ((VariableReferenceExpression) originalSelect.getCondition().getValue()).getVariableReference();
+
+        // Additionally, verify that the subplan does not produce any other variable other than the SELECT var above.
+        SubplanOperator subplanOperator = (SubplanOperator) originalSelect.getInputs().get(0).getValue();
+        List<LogicalVariable> subplanProducedVars = new ArrayList<>();
+        VariableUtilities.getProducedVariables(subplanOperator, subplanProducedVars);
+        if (subplanProducedVars.size() != 1 || !subplanProducedVars.get(0).equals(originalSelectVar)) {
+            return null;
+        }
+
+        return traverseSubplanBranch(subplanOperator);
+    }
+
+    /**
+     * To undo this process is to return what was passed to us at {@code createSelect} time.
+     */
+    public SelectOperator getOriginalSelect() {
+        return originalSelectRoot;
+    }
+
+    private SelectOperator traverseSubplanBranch(SubplanOperator subplanOperator) throws AlgebricksException {
+        // We only expect one plan, and one root.
+        if (subplanOperator.getNestedPlans().size() > 1
+                || subplanOperator.getNestedPlans().get(0).getRoots().size() > 1) {
+            return null;
+        }
+
+        // This root of our "subplan" should always be an aggregate.
+        ILogicalOperator workingSubplanRoot = subplanOperator.getNestedPlans().get(0).getRoots().get(0).getValue();
+        AggregateOperator workingSubplanRootAsAggregate;
+        if (!workingSubplanRoot.getOperatorTag().equals(LogicalOperatorTag.AGGREGATE)) {
+            return null;
+        }
+        workingSubplanRootAsAggregate = (AggregateOperator) workingSubplanRoot;
+
+        // Try to find a SELECT that we can optimize (i.e. has a function call).
+        SelectOperator optimizableSelect = null;
+        for (Mutable<ILogicalOperator> opInput : workingSubplanRoot.getInputs()) {
+            ILogicalOperator subplanOrSelect = findSubplanOrSelect(opInput.getValue());
+            if (subplanOrSelect == null) {
+                return null;
+
+            } else if (subplanOrSelect.getOperatorTag().equals(LogicalOperatorTag.SUBPLAN)) {
+                optimizableSelect = traverseSubplanBranch((SubplanOperator) subplanOrSelect);
+
+            } else {
+                optimizableSelect = (SelectOperator) subplanOrSelect;
+                break;
+            }
+        }
+        if (optimizableSelect == null) {
+            return null;
+        }
+
+        // We have found a SELECT with a variable. Create a copy, and set this to our rewrite root.
+        SelectOperator newSelectOperator = new SelectOperator(optimizableSelect.getCondition(),
+                optimizableSelect.getRetainMissing(), optimizableSelect.getMissingPlaceholderVariable());
+
+        // Ensure that this SELECT represents a predicate for an existential query, and is a query we can optimize.
+        newSelectOperator = normalizeSelectCondition(workingSubplanRootAsAggregate, newSelectOperator,
+                subplanOperator.getInputs().get(0).getValue());
+        if (newSelectOperator == null) {
+            return null;
+        }
+        newSelectOperator.setSourceLocation(sourceLocation);
+        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;
+        UnnestOperator bottommostNewUnnest = null;
+        while (!workingOriginalOperator.getOperatorTag().equals(LogicalOperatorTag.NESTEDTUPLESOURCE)) {
+            if (workingOriginalOperator.getInputs().isEmpty()) {
+                throw new CompilationException(ErrorCode.COMPILATION_ILLEGAL_STATE,
+                        workingSubplanRoot.getSourceLocation(),
+                        "NESTED-TUPLE-SOURCE expected in nested plan branch," + " but not found.");
+            }
+
+            switch (workingOriginalOperator.getOperatorTag()) {
+                case UNNEST:
+                    UnnestOperator originalUnnest = (UnnestOperator) workingOriginalOperator;
+                    UnnestOperator newUnnest =
+                            new UnnestOperator(originalUnnest.getVariable(), originalUnnest.getExpressionRef());
+                    newUnnest.setSourceLocation(sourceLocation);
+                    workingNewOperator.getInputs().add(new MutableObject<>(newUnnest));
+                    workingNewOperator = newUnnest;
+                    bottommostNewUnnest = (UnnestOperator) workingNewOperator;
+                    break;
+
+                case ASSIGN:
+                    AssignOperator originalAssign = (AssignOperator) workingOriginalOperator;
+                    AssignOperator newAssign =
+                            new AssignOperator(originalAssign.getVariables(), originalAssign.getExpressions());
+                    newAssign.setSourceLocation(sourceLocation);
+                    workingNewOperator.getInputs().add(new MutableObject<>(newAssign));
+                    workingNewOperator = newAssign;
+                    break;
+
+                case SUBPLAN:
+                    // TODO (GLENN): Work on supporting nested universal quantification.
+                    return null;
+
+                case AGGREGATE:
+                case SELECT:
+                    break;
+
+                default:
+                    return null;
+            }
+
+            workingOriginalOperator = workingOriginalOperator.getInputs().get(0).getValue();
+        }
+
+        // If we are working with universal quantification, then we must also check whether or not we have a conjunct 
+        // that asserts that the array should also be non-empty.
+        if (isUniversalQuantification(workingSubplanRootAsAggregate)
+                && !isArrayNonEmptyConjunctIncluded(bottommostNewUnnest, subplanOperator)) {
+            return null;
+        }
+
+        // We have added everything we need in our nested-plan branch. Now, connect the input of our SUBPLAN to our
+        // current working branch.
+        bottommostNewUnnest.getInputs().addAll(subplanOperator.getInputs());
+        OperatorManipulationUtil.computeTypeEnvironmentBottomUp(newSelectOperator, context);
+
+        return newSelectOperator;
+    }
+
+    private boolean isUniversalQuantification(AggregateOperator workingSubplanRoot) throws CompilationException {
+        AggregateFunctionCallExpression aggregateFunctionCallExpression =
+                (AggregateFunctionCallExpression) workingSubplanRoot.getExpressions().get(0).getValue();
+        if (aggregateFunctionCallExpression.getFunctionIdentifier().equals(BuiltinFunctions.EMPTY_STREAM)) {
+            return true;
+        } else if (aggregateFunctionCallExpression.getFunctionIdentifier().equals(BuiltinFunctions.NON_EMPTY_STREAM)) {
+            return false;
+        } else {
+            throw new CompilationException(ErrorCode.COMPILATION_ERROR, workingSubplanRoot.getSourceLocation(),
+                    "Unexpected aggregate function: " + aggregateFunctionCallExpression.getFunctionIdentifier());
+        }
+    }
+
+    private boolean isArrayNonEmptyConjunctIncluded(UnnestOperator firstUnnestInNTS, SubplanOperator subplanOperator) {
+        UnnestingFunctionCallExpression unnestFunction =
+                (UnnestingFunctionCallExpression) firstUnnestInNTS.getExpressionRef().getValue();
+        VariableReferenceExpression unnestVarExpr =
+                (VariableReferenceExpression) unnestFunction.getArguments().get(0).getValue();
+        LogicalVariable arrayVariable = unnestVarExpr.getVariableReference();
+
+        // TODO (GLENN): The SELECT directly below the SUBPLAN is the only operator we explore. This does not cover
+        //  all predicates where the array may be non-empty (say, having an existential predicate located after this 
+        //  subplan).
+        if (!subplanOperator.getInputs().get(0).getValue().getOperatorTag().equals(LogicalOperatorTag.SELECT)) {
+            return false;
+        }
+        SelectOperator subplanInputOperator = (SelectOperator) subplanOperator.getInputs().get(0).getValue();
+        ILogicalExpression selectCondExpr = subplanInputOperator.getCondition().getValue();
+        List<Mutable<ILogicalExpression>> conjunctsFromSelect = new ArrayList<>();
+        if (selectCondExpr.splitIntoConjuncts(conjunctsFromSelect)) {
+            // We have a collection of conjuncts. Analyze each conjunct w/ a function.
+            for (Mutable<ILogicalExpression> mutableConjuct : conjunctsFromSelect) {
+                ILogicalExpression workingConjunct = mutableConjuct.getValue();
+                if (workingConjunct.getExpressionTag().equals(LogicalExpressionTag.FUNCTION_CALL)
+                        && analyzeConjunctForArrayNonEmptiness(arrayVariable,
+                                (ScalarFunctionCallExpression) workingConjunct)) {
+                    return true;
+                }
+            }
+
+            // No such conjunct found.
+            return false;
+        }
+
+        if (!selectCondExpr.getExpressionTag().equals(LogicalExpressionTag.FUNCTION_CALL)) {
+            return false;
+        }
+        return analyzeConjunctForArrayNonEmptiness(arrayVariable, (ScalarFunctionCallExpression) selectCondExpr);
+    }
+
+    private boolean analyzeConjunctForArrayNonEmptiness(LogicalVariable arrayVariable,
+            ScalarFunctionCallExpression workingSelectCondExpr) {
+        // Handle the conjunct: LEN(arrayVar) > 0
+        if (workingSelectCondExpr.getFunctionIdentifier().equals(BuiltinFunctions.GT)) {
+            ILogicalExpression firstArg = workingSelectCondExpr.getArguments().get(0).getValue();
+            ILogicalExpression secondArg = workingSelectCondExpr.getArguments().get(1).getValue();
+
+            if (firstArg.getExpressionTag().equals(LogicalExpressionTag.FUNCTION_CALL)
+                    && ((ScalarFunctionCallExpression) firstArg).getFunctionIdentifier().equals(BuiltinFunctions.LEN)) {
+                ScalarFunctionCallExpression lenFunction = (ScalarFunctionCallExpression) firstArg;
+                List<LogicalVariable> usedVariables = new ArrayList<>();
+                lenFunction.getUsedVariables(usedVariables);
+
+                return usedVariables.contains(arrayVariable)
+                        && secondArg.getExpressionTag().equals(LogicalExpressionTag.CONSTANT)
+                        && zerosAsAsterixConstants.contains(((ConstantExpression) secondArg).getValue());
+            }
+        }
+
+        // Handle the conjunct: 0 < LEN(arrayVar)
+        else if (workingSelectCondExpr.getFunctionIdentifier().equals(BuiltinFunctions.LT)) {
+            ILogicalExpression firstArg = workingSelectCondExpr.getArguments().get(0).getValue();
+            ILogicalExpression secondArg = workingSelectCondExpr.getArguments().get(1).getValue();
+
+            if (secondArg.getExpressionTag().equals(LogicalExpressionTag.FUNCTION_CALL)
+                    && ((ScalarFunctionCallExpression) secondArg).getFunctionIdentifier()
+                            .equals(BuiltinFunctions.LEN)) {
+                ScalarFunctionCallExpression lenFunction = (ScalarFunctionCallExpression) secondArg;
+                List<LogicalVariable> usedVariables = new ArrayList<>();
+                lenFunction.getUsedVariables(usedVariables);
+
+                return usedVariables.contains(arrayVariable)
+                        && firstArg.getExpressionTag().equals(LogicalExpressionTag.CONSTANT)
+                        && zerosAsAsterixConstants.contains(((ConstantExpression) firstArg).getValue());
+            }
+        }
+
+        // TODO (GLENN): Handle the cases 1) where the arrayVar is explicitly indexed, 2) the NOT function.
+        return false;
+    }
+
+    private SelectOperator normalizeSelectCondition(AggregateOperator aggregateOperator, SelectOperator selectOperator,
+            ILogicalOperator subplanInputOperator) throws AlgebricksException {
+        // The purpose of this function is to remove the NOT(IF-MISSING-OR-NULL(...)) functions for a universal
+        // quantification query. The {@code ArrayBTreeAccessMethod} does not recognize the former as optimizable
+        // functions, so we remove them here. This SELECT will never make it to the final query plan (after the
+        // {@code IntroduceSelectAccessMethodRule}), which allows us to get away with this logically incorrect branch.
+        if (!isUniversalQuantification(aggregateOperator)) {
+            // We are working with an existential quantification query. Do not modify the SELECT.
+            return selectOperator;
+
+        } else {
+            // We are working with a universal quantification query.
+            if (!subplanInputOperator.getOperatorTag().equals(LogicalOperatorTag.SELECT)) {
+                return null;
+            }
+
+            ScalarFunctionCallExpression notFunction =
+                    (ScalarFunctionCallExpression) selectOperator.getCondition().getValue();
+            if (!notFunction.getFunctionIdentifier().equals(BuiltinFunctions.NOT)) {
+                return selectOperator;
+            }
+
+            ScalarFunctionCallExpression ifMissingOrNullFunction =
+                    (ScalarFunctionCallExpression) notFunction.getArguments().get(0).getValue();
+            if (!ifMissingOrNullFunction.getFunctionIdentifier().equals(BuiltinFunctions.IF_MISSING_OR_NULL)) {
+                return selectOperator;
+            }
+
+            Mutable<ILogicalExpression> newSelectCondition =
+                    new MutableObject<>(ifMissingOrNullFunction.getArguments().get(0).getValue().cloneExpression());
+            return new SelectOperator(newSelectCondition, selectOperator.getRetainMissing(),
+                    selectOperator.getMissingPlaceholderVariable());
+
+        }
+    }
+
+    private ILogicalOperator findSubplanOrSelect(ILogicalOperator operator) {
+        // We are trying to find a SELECT operator with a function call that is not "NOT(IF-MISSING-OR-NULL(...))".
+        if (operator.getOperatorTag().equals(LogicalOperatorTag.SELECT)) {
+            SelectOperator selectOperator = (SelectOperator) operator;
+            ILogicalExpression selectCondExpr = selectOperator.getCondition().getValue();
+            if (selectCondExpr.getExpressionTag() == LogicalExpressionTag.FUNCTION_CALL) {
+
+                // Follow the chain of NOT(IF-MISSING-OR-NULL(...)) to see if we have a variable at the end.
+                ScalarFunctionCallExpression notFunction =
+                        (ScalarFunctionCallExpression) selectOperator.getCondition().getValue();
+                if (notFunction.getFunctionIdentifier().equals(BuiltinFunctions.NOT)) {
+                    ScalarFunctionCallExpression ifMissingOrNullFunction =
+                            (ScalarFunctionCallExpression) notFunction.getArguments().get(0).getValue();
+                    if (ifMissingOrNullFunction.getFunctionIdentifier().equals(BuiltinFunctions.IF_MISSING_OR_NULL)) {
+                        ILogicalExpression finalExpr = ifMissingOrNullFunction.getArguments().get(0).getValue();
+                        if (finalExpr.getExpressionTag() != LogicalExpressionTag.VARIABLE) {
+                            return selectOperator;
+                        }
+                    }
+
+                } else {
+                    return selectOperator;
+                }
+            }
+        } else if (operator.getOperatorTag().equals(LogicalOperatorTag.SUBPLAN)) {
+            // We have found an additional SUBPLAN branch to explore. Recurse w/ caller function.
+            return operator;
+        }
+
+        // No matching operator found. Recurse on current operator input.
+        if (operator.getInputs().isEmpty()) {
+            return null;
+        } else {
+            return findSubplanOrSelect(operator.getInputs().get(0).getValue());
+        }
+    }
+}
diff --git a/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/translator/util/ValidateUtil.java b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/translator/util/ValidateUtil.java
index b32b05e..13bcfb6 100644
--- a/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/translator/util/ValidateUtil.java
+++ b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/translator/util/ValidateUtil.java
@@ -27,7 +27,6 @@
 import org.apache.asterix.metadata.utils.KeyFieldTypeUtil;
 import org.apache.asterix.om.types.ARecordType;
 import org.apache.asterix.om.types.ATypeTag;
-import org.apache.asterix.om.types.BuiltinType;
 import org.apache.asterix.om.types.IAType;
 import org.apache.asterix.om.utils.RecordUtil;
 import org.apache.hyracks.algebricks.common.exceptions.AlgebricksException;
@@ -184,137 +183,111 @@
     /**
      * Validates the key fields that will be used as keys of an index.
      *
-     * @param recType
-     *            the record type
-     * @param keyFieldNames
-     *            a map of key fields that will be validated
-     * @param keyFieldTypes
-     *            a map of key types (if provided) that will be validated
      * @param indexType
      *            the type of the index that its key fields is being validated
+     * @param fieldType
+     *            a key field type
+     * @param displayFieldName
+     *            a field name to use for error reporting
      * @param sourceLoc
      *            the source location
      * @throws AlgebricksException
      */
-    public static void validateKeyFields(ARecordType recType, ARecordType metaRecType, List<List<String>> keyFieldNames,
-            List<Integer> keySourceIndicators, List<IAType> keyFieldTypes, IndexType indexType,
+    public static void validateIndexFieldType(IndexType indexType, IAType fieldType, List<String> displayFieldName,
             SourceLocation sourceLoc) throws AlgebricksException {
-        List<IAType> fieldTypes =
-                KeyFieldTypeUtil.getKeyTypes(recType, metaRecType, keyFieldNames, keySourceIndicators);
-        int pos = 0;
-        boolean openFieldCompositeIdx = false;
-        for (IAType fieldType : fieldTypes) {
-            List<String> fieldName = keyFieldNames.get(pos);
-            if (fieldType == null) {
-                fieldType = keyFieldTypes.get(pos);
-                if (keyFieldTypes.get(pos) == BuiltinType.AMISSING) {
-                    throw new CompilationException(ErrorCode.COMPILATION_ERROR, sourceLoc,
-                            "A field with this name  \"" + fieldName + "\" could not be found.");
+        switch (indexType) {
+            case ARRAY:
+            case BTREE:
+                switch (fieldType.getTypeTag()) {
+                    case TINYINT:
+                    case SMALLINT:
+                    case INTEGER:
+                    case BIGINT:
+                    case FLOAT:
+                    case DOUBLE:
+                    case STRING:
+                    case BINARY:
+                    case DATE:
+                    case TIME:
+                    case DATETIME:
+                    case UNION:
+                    case UUID:
+                    case YEARMONTHDURATION:
+                    case DAYTIMEDURATION:
+                        break;
+                    default:
+                        throw new CompilationException(ErrorCode.COMPILATION_ERROR, sourceLoc,
+                                "The field \"" + displayFieldName + "\" which is of type " + fieldType.getTypeTag()
+                                        + " cannot be indexed using the BTree index.");
                 }
-            } else if (openFieldCompositeIdx) {
-                throw new CompilationException(ErrorCode.COMPILATION_ERROR, sourceLoc, "A closed field \"" + fieldName
-                        + "\" could be only in a prefix part of the composite index, containing opened field.");
-            }
-            if (keyFieldTypes.get(pos) != BuiltinType.AMISSING
-                    && fieldType.getTypeTag() != keyFieldTypes.get(pos).getTypeTag()) {
-                throw new CompilationException(ErrorCode.COMPILATION_ERROR, sourceLoc,
-                        "A field \"" + fieldName + "\" is already defined with the type \"" + fieldType + "\"");
-            }
-            switch (indexType) {
-                case BTREE:
-                    switch (fieldType.getTypeTag()) {
-                        case TINYINT:
-                        case SMALLINT:
-                        case INTEGER:
-                        case BIGINT:
-                        case FLOAT:
-                        case DOUBLE:
-                        case STRING:
-                        case BINARY:
-                        case DATE:
-                        case TIME:
-                        case DATETIME:
-                        case UNION:
-                        case UUID:
-                        case YEARMONTHDURATION:
-                        case DAYTIMEDURATION:
-                            break;
-                        default:
-                            throw new CompilationException(ErrorCode.COMPILATION_ERROR, sourceLoc,
-                                    "The field \"" + fieldName + "\" which is of type " + fieldType.getTypeTag()
-                                            + " cannot be indexed using the BTree index.");
-                    }
-                    break;
-                case RTREE:
-                    switch (fieldType.getTypeTag()) {
-                        case POINT:
-                        case LINE:
-                        case RECTANGLE:
-                        case CIRCLE:
-                        case POLYGON:
-                        case GEOMETRY:
-                        case UNION:
-                            break;
-                        default:
-                            throw new CompilationException(ErrorCode.COMPILATION_ERROR, sourceLoc,
-                                    "The field \"" + fieldName + "\" which is of type " + fieldType.getTypeTag()
-                                            + " cannot be indexed using the RTree index.");
-                    }
-                    break;
-                case LENGTH_PARTITIONED_NGRAM_INVIX:
-                    switch (fieldType.getTypeTag()) {
-                        case STRING:
-                        case UNION:
-                            break;
-                        default:
-                            throw new CompilationException(ErrorCode.COMPILATION_ERROR, sourceLoc,
-                                    "The field \"" + fieldName + "\" which is of type " + fieldType.getTypeTag()
-                                            + " cannot be indexed using the Length Partitioned N-Gram index.");
-                    }
-                    break;
-                case LENGTH_PARTITIONED_WORD_INVIX:
-                    switch (fieldType.getTypeTag()) {
-                        case STRING:
-                        case MULTISET:
-                        case ARRAY:
-                        case UNION:
-                            break;
-                        default:
-                            throw new CompilationException(ErrorCode.COMPILATION_ERROR, sourceLoc,
-                                    "The field \"" + fieldName + "\" which is of type " + fieldType.getTypeTag()
-                                            + " cannot be indexed using the Length Partitioned Keyword index.");
-                    }
-                    break;
-                case SINGLE_PARTITION_NGRAM_INVIX:
-                    switch (fieldType.getTypeTag()) {
-                        case STRING:
-                        case UNION:
-                            break;
-                        default:
-                            throw new CompilationException(ErrorCode.COMPILATION_ERROR, sourceLoc,
-                                    "The field \"" + fieldName + "\" which is of type " + fieldType.getTypeTag()
-                                            + " cannot be indexed using the N-Gram index.");
-                    }
-                    break;
-                case SINGLE_PARTITION_WORD_INVIX:
-                    switch (fieldType.getTypeTag()) {
-                        case STRING:
-                        case MULTISET:
-                        case ARRAY:
-                        case UNION:
-                            break;
-                        default:
-                            throw new CompilationException(ErrorCode.COMPILATION_ERROR, sourceLoc,
-                                    "The field \"" + fieldName + "\" which is of type " + fieldType.getTypeTag()
-                                            + " cannot be indexed using the Keyword index.");
-                    }
-                    break;
-                default:
-                    throw new CompilationException(ErrorCode.COMPILATION_ERROR, sourceLoc,
-                            "Invalid index type: " + indexType + ".");
-            }
-            pos++;
+                break;
+            case RTREE:
+                switch (fieldType.getTypeTag()) {
+                    case POINT:
+                    case LINE:
+                    case RECTANGLE:
+                    case CIRCLE:
+                    case POLYGON:
+                    case GEOMETRY:
+                    case UNION:
+                        break;
+                    default:
+                        throw new CompilationException(ErrorCode.COMPILATION_ERROR, sourceLoc,
+                                "The field \"" + displayFieldName + "\" which is of type " + fieldType.getTypeTag()
+                                        + " cannot be indexed using the RTree index.");
+                }
+                break;
+            case LENGTH_PARTITIONED_NGRAM_INVIX:
+                switch (fieldType.getTypeTag()) {
+                    case STRING:
+                    case UNION:
+                        break;
+                    default:
+                        throw new CompilationException(ErrorCode.COMPILATION_ERROR, sourceLoc,
+                                "The field \"" + displayFieldName + "\" which is of type " + fieldType.getTypeTag()
+                                        + " cannot be indexed using the Length Partitioned N-Gram index.");
+                }
+                break;
+            case LENGTH_PARTITIONED_WORD_INVIX:
+                switch (fieldType.getTypeTag()) {
+                    case STRING:
+                    case MULTISET:
+                    case ARRAY:
+                    case UNION:
+                        break;
+                    default:
+                        throw new CompilationException(ErrorCode.COMPILATION_ERROR, sourceLoc,
+                                "The field \"" + displayFieldName + "\" which is of type " + fieldType.getTypeTag()
+                                        + " cannot be indexed using the Length Partitioned Keyword index.");
+                }
+                break;
+            case SINGLE_PARTITION_NGRAM_INVIX:
+                switch (fieldType.getTypeTag()) {
+                    case STRING:
+                    case UNION:
+                        break;
+                    default:
+                        throw new CompilationException(ErrorCode.COMPILATION_ERROR, sourceLoc,
+                                "The field \"" + displayFieldName + "\" which is of type " + fieldType.getTypeTag()
+                                        + " cannot be indexed using the N-Gram index.");
+                }
+                break;
+            case SINGLE_PARTITION_WORD_INVIX:
+                switch (fieldType.getTypeTag()) {
+                    case STRING:
+                    case MULTISET:
+                    case ARRAY:
+                    case UNION:
+                        break;
+                    default:
+                        throw new CompilationException(ErrorCode.COMPILATION_ERROR, sourceLoc,
+                                "The field \"" + displayFieldName + "\" which is of type " + fieldType.getTypeTag()
+                                        + " cannot be indexed using the Keyword index.");
+                }
+                break;
+            default:
+                throw new CompilationException(ErrorCode.COMPILATION_UNKNOWN_INDEX_TYPE, sourceLoc,
+                        String.valueOf(indexType));
         }
     }
-
 }
diff --git a/asterixdb/asterix-app/data/yelp-checkin/use-case-1.json b/asterixdb/asterix-app/data/yelp-checkin/use-case-1.json
new file mode 100644
index 0000000..4d66342
--- /dev/null
+++ b/asterixdb/asterix-app/data/yelp-checkin/use-case-1.json
@@ -0,0 +1,30 @@
+{ "business_id": "--1UhMGODdWsrMastO9DZw", "dates": [ "2016-04-26 19:49:16", "2016-08-30 18:36:57", "2016-10-15 02:45:18", "2016-11-18 01:54:50", "2017-04-20 18:39:06", "2017-05-03 17:58:02", "2019-03-19 22:04:48" ] }
+{ "business_id": "--EF5N7P70J_UYBTPypYlA", "dates": [ "2018-05-25 19:52:07", "2018-09-18 16:09:44", "2019-10-18 21:29:09" ] }
+{ "business_id": "--Ni3oJ4VOqfOEu7Sj2Vzg", "dates": [ "2019-06-07 17:54:58" ] }
+{ "business_id": "--Y1Adl1YUWfYIRSd8vkmA", "dates": [ "2011-05-03 20:54:05", "2011-08-23 20:49:45", "2014-12-04 06:13:01", "2016-11-16 19:25:55" ] }
+{ "business_id": "--YPwqIlRJrhHkJcjY3eiA", "dates": [ "2016-06-18 21:35:45", "2016-10-15 18:17:51" ] }
+{ "business_id": "--e8PjCNhEz32pprnPhCwQ", "dates": [ "2015-04-02 21:45:17" ] }
+{ "business_id": "--kinfHwmtdjz03g8B8z8Q", "dates": [ "2014-08-27 17:49:18", "2015-12-19 21:30:31", "2018-11-27 15:53:50" ] }
+{ "business_id": "--q6datkI-f0EoVheXNEeQ", "dates": [ "2014-01-28 20:56:04", "2014-11-16 16:11:58", "2015-11-15 19:21:53", "2015-11-15 19:33:39" ] }
+{ "business_id": "--qvQS4MigHPykD2GV0-zw", "dates": [ "2019-04-11 18:30:12" ] }
+{ "business_id": "--wIGbLEhlpl_UeAIyDmZQ", "dates": [ "2015-06-06 20:01:06", "2019-03-14 22:01:52" ] }
+{ "business_id": "-0FA-Qdi3SPYIoJz9UQw-A", "dates": [ "2018-09-29 18:55:17", "2018-10-20 16:48:05", "2018-10-20 22:20:24" ] }
+{ "business_id": "-0Hj1hb_XW6ybWq2M7QhGA", "dates": [ "2011-04-23 21:11:22", "2014-05-04 19:42:48", "2014-05-11 19:16:08", "2014-06-04 19:14:18", "2015-12-05 19:22:42", "2017-05-15 23:19:00" ] }
+{ "business_id": "-0KMvRFwDWdVBeTpT11iHw", "dates": [ "2012-07-13 21:43:57", "2016-12-24 02:27:31", "2017-08-31 00:35:26" ] }
+{ "business_id": "-0LPtgJC31FWMrMv317p0Q", "dates": [ "2013-04-13 12:35:33", "2013-08-19 23:35:49", "2013-10-04 19:14:56" ] }
+{ "business_id": "-0M3o2uWBnQZwd3hmfEwuw", "dates": [ "2016-09-10 19:26:19", "2018-09-08 14:15:37", "2019-09-13 22:47:25" ] }
+{ "business_id": "-0RRiWDtfnS16AKCtfvBZg", "dates": [ "2017-05-19 14:30:16", "2017-05-19 14:30:25", "2017-08-28 15:49:37", "2017-09-20 20:19:51", "2017-10-01 16:31:05", "2017-10-01 16:56:27", "2017-12-27 23:33:20" ] }
+{ "business_id": "-0Soj75v-XoRcf2ERr8Bmg", "dates": [ "2019-06-05 18:22:49" ] }
+{ "business_id": "-0ZumLlFjMh4ZW1z2nXGug", "dates": [ "2011-09-24 21:37:32", "2014-03-10 20:20:07", "2015-05-27 00:40:24", "2015-08-29 17:58:15", "2018-03-16 15:03:26" ] }
+{ "business_id": "-0aOudcaAyac0VJbMX-L1g", "dates": [ "2015-03-16 23:51:16", "2015-12-21 04:48:01", "2016-10-28 20:22:42", "2016-10-28 20:23:00" ] }
+{ "business_id": "-0b86isaXMY0v4g-V8GZ9Q", "dates": [ "2013-10-22 16:49:21", "2014-11-21 17:39:24" ] }
+{ "business_id": "-0d-BfFSU0bwLcnMaGRxYw", "dates": [ "2014-08-07 18:30:48", "2014-09-16 20:41:45", "2014-10-12 23:22:27", "2015-07-21 20:43:56", "2015-07-21 20:45:07" ] }
+{ "business_id": "-0jz6c3C6i7RG7Ag22K-Pg", "dates": [ "2015-05-02 19:49:05", "2015-05-06 03:52:18", "2015-09-26 01:13:19" ] }
+{ "business_id": "-0y3MZU2oYP8r1ruDP1bfQ", "dates": [ "2015-04-11 13:14:14", "2015-11-21 16:05:56", "2016-05-06 14:10:04", "2017-08-09 15:15:10", "2017-10-21 15:12:56" ] }
+{ "business_id": "-1BPe8UjF2_l3nVk-DFUjA", "dates": [ "2015-12-03 18:44:00", "2016-03-17 18:19:21", "2016-11-02 15:58:38" ] }
+{ "business_id": "-1E2CQu_38mkghvmZgCCRw", "dates": [ "2019-04-04 22:02:37" ] }
+{ "business_id": "-1wzk43IZ5D9Ysu6kzb5xA", "dates": [ "2019-02-27 14:03:08" ] }
+{ "business_id": "-23R9P2eG7VTc6DVLjFKzA", "dates": [ "2011-12-21 19:02:51", "2012-04-15 04:21:39", "2012-04-15 14:23:56", "2013-06-30 22:39:51", "2013-10-04 20:34:13", "2014-07-16 02:28:40" ] }
+{ "business_id": "-26MGfikhJiTfCI-GqmzhQ", "dates": [ "2018-06-13 20:16:07" ] }
+{ "business_id": "-2bLuJsMZ0WhI9daurVQNQ", "dates": [ "2015-05-29 16:46:17", "2015-06-01 15:03:53" ] }
+{ "business_id": "-2hDBMaza_ldqnZdiU06LQ", "dates": [ "2011-10-08 12:02:23", "2014-08-18 02:11:11", "2016-01-07 05:27:51", "2016-10-21 20:15:55", "2016-12-01 03:57:10", "2016-12-29 01:54:42", "2018-07-22 19:55:31", "2018-09-07 01:42:54", "2019-03-08 03:41:06" ] }
diff --git a/asterixdb/asterix-app/data/yelp-checkin/use-case-2.json b/asterixdb/asterix-app/data/yelp-checkin/use-case-2.json
new file mode 100644
index 0000000..2179005
--- /dev/null
+++ b/asterixdb/asterix-app/data/yelp-checkin/use-case-2.json
@@ -0,0 +1,31 @@
+{ "business_id": "--1UhMGODdWsrMastO9DZw", "checkin_times": { "dates": [ "2016-04-26", "2016-08-30", "2016-10-15", "2016-11-18", "2017-04-20", "2017-05-03", "2019-03-19" ], "times": [ "19:49:16", "18:36:57", "02:45:18", "01:54:50", "18:39:06", "17:58:02", "22:04:48" ] } }
+{ "business_id": "--EF5N7P70J_UYBTPypYlA", "checkin_times": { "dates": [ "2018-05-25", "2018-09-18", "2019-10-18" ], "times": [ "19:52:07", "16:09:44", "21:29:09" ] } }
+{ "business_id": "--Ni3oJ4VOqfOEu7Sj2Vzg", "checkin_times": { "dates": [ "2019-06-07" ], "times": [ "17:54:58" ] } }
+{ "business_id": "--Y1Adl1YUWfYIRSd8vkmA", "checkin_times": { "dates": [ "2011-05-03", "2011-08-23", "2014-12-04", "2016-11-16" ], "times": [ "20:54:05", "20:49:45", "06:13:01", "19:25:55" ] } }
+{ "business_id": "--YPwqIlRJrhHkJcjY3eiA", "checkin_times": { "dates": [ "2016-06-18", "2016-10-15" ], "times": [ "21:35:45", "18:17:51" ] } }
+{ "business_id": "--e8PjCNhEz32pprnPhCwQ", "checkin_times": { "dates": [ "2015-04-02" ], "times": [ "21:45:17" ] } }
+{ "business_id": "--kinfHwmtdjz03g8B8z8Q", "checkin_times": { "dates": [ "2014-08-27", "2015-12-19", "2018-11-27" ], "times": [ "17:49:18", "21:30:31", "15:53:50" ] } }
+{ "business_id": "--q6datkI-f0EoVheXNEeQ", "checkin_times": { "dates": [ "2014-01-28", "2014-11-16", "2015-11-15", "2015-11-15" ], "times": [ "20:56:04", "16:11:58", "19:21:53", "19:33:39" ] } }
+{ "business_id": "--qvQS4MigHPykD2GV0-zw", "checkin_times": { "dates": [ "2019-04-11" ], "times": [ "18:30:12" ] } }
+{ "business_id": "--wIGbLEhlpl_UeAIyDmZQ", "checkin_times": { "dates": [ "2015-06-06", "2019-03-14" ], "times": [ "20:01:06", "22:01:52" ] } }
+{ "business_id": "-0FA-Qdi3SPYIoJz9UQw-A", "checkin_times": { "dates": [ "2018-09-29", "2018-10-20", "2018-10-20" ], "times": [ "18:55:17", "16:48:05", "22:20:24" ] } }
+{ "business_id": "-0Hj1hb_XW6ybWq2M7QhGA", "checkin_times": { "dates": [ "2011-04-23", "2014-05-04", "2014-05-11", "2014-06-04", "2015-12-05", "2017-05-15" ], "times": [ "21:11:22", "19:42:48", "19:16:08", "19:14:18", "19:22:42", "23:19:00" ] } }
+{ "business_id": "-0KMvRFwDWdVBeTpT11iHw", "checkin_times": { "dates": [ "2012-07-13", "2016-12-24", "2017-08-31" ], "times": [ "21:43:57", "02:27:31", "00:35:26" ] } }
+{ "business_id": "-0LPtgJC31FWMrMv317p0Q", "checkin_times": { "dates": [ "2013-04-13", "2013-08-19", "2013-10-04" ], "times": [ "12:35:33", "23:35:49", "19:14:56" ] } }
+{ "business_id": "-0M3o2uWBnQZwd3hmfEwuw", "checkin_times": { "dates": [ "2016-09-10", "2018-09-08", "2019-09-13" ], "times": [ "19:26:19", "14:15:37", "22:47:25" ] } }
+{ "business_id": "-0RRiWDtfnS16AKCtfvBZg", "checkin_times": { "dates": [ "2017-05-19", "2017-05-19", "2017-08-28", "2017-09-20", "2017-10-01", "2017-10-01", "2017-12-27" ], "times": [ "14:30:16", "14:30:25", "15:49:37", "20:19:51", "16:31:05", "16:56:27", "23:33:20" ] } }
+{ "business_id": "-0Soj75v-XoRcf2ERr8Bmg", "checkin_times": { "dates": [ "2019-06-05" ], "times": [ "18:22:49" ] } }
+{ "business_id": "-0ZumLlFjMh4ZW1z2nXGug", "checkin_times": { "dates": [ "2011-09-24", "2014-03-10", "2015-05-27", "2015-08-29", "2018-03-16" ], "times": [ "21:37:32", "20:20:07", "00:40:24", "17:58:15", "15:03:26" ] } }
+{ "business_id": "-0aOudcaAyac0VJbMX-L1g", "checkin_times": { "dates": [ "2015-03-16", "2015-12-21", "2016-10-28", "2016-10-28" ], "times": [ "23:51:16", "04:48:01", "20:22:42", "20:23:00" ] } }
+{ "business_id": "-0b86isaXMY0v4g-V8GZ9Q", "checkin_times": { "dates": [ "2013-10-22", "2014-11-21" ], "times": [ "16:49:21", "17:39:24" ] } }
+{ "business_id": "-0d-BfFSU0bwLcnMaGRxYw", "checkin_times": { "dates": [ "2014-08-07", "2014-09-16", "2014-10-12", "2015-07-21", "2015-07-21" ], "times": [ "18:30:48", "20:41:45", "23:22:27", "20:43:56", "20:45:07" ] } }
+{ "business_id": "-0jz6c3C6i7RG7Ag22K-Pg", "checkin_times": { "dates": [ "2015-05-02", "2015-05-06", "2015-09-26" ], "times": [ "19:49:05", "03:52:18", "01:13:19" ] } }
+{ "business_id": "-0y3MZU2oYP8r1ruDP1bfQ", "checkin_times": { "dates": [ "2015-04-11", "2015-11-21", "2016-05-06", "2017-08-09", "2017-10-21" ], "times": [ "13:14:14", "16:05:56", "14:10:04", "15:15:10", "15:12:56" ] } }
+{ "business_id": "-1BPe8UjF2_l3nVk-DFUjA", "checkin_times": { "dates": [ "2015-12-03", "2016-03-17", "2016-11-02" ], "times": [ "18:44:00", "18:19:21", "15:58:38" ] } }
+{ "business_id": "-1E2CQu_38mkghvmZgCCRw", "checkin_times": { "dates": [ "2019-04-04" ], "times": [ "22:02:37" ] } }
+{ "business_id": "-1wzk43IZ5D9Ysu6kzb5xA", "checkin_times": { "dates": [ "2019-02-27" ], "times": [ "14:03:08" ] } }
+{ "business_id": "-23R9P2eG7VTc6DVLjFKzA", "checkin_times": { "dates": [ "2011-12-21", "2012-04-15", "2012-04-15", "2013-06-30", "2013-10-04", "2014-07-16" ], "times": [ "19:02:51", "04:21:39", "14:23:56", "22:39:51", "20:34:13", "02:28:40" ] } }
+{ "business_id": "-26MGfikhJiTfCI-GqmzhQ", "checkin_times": { "dates": [ "2018-06-13" ], "times": [ "20:16:07" ] } }
+{ "business_id": "-2bLuJsMZ0WhI9daurVQNQ", "checkin_times": { "dates": [ "2015-05-29", "2015-06-01" ], "times": [ "16:46:17", "15:03:53" ] } }
+{ "business_id": "-2hDBMaza_ldqnZdiU06LQ", "checkin_times": { "dates": [ "2011-10-08", "2014-08-18", "2016-01-07", "2016-10-21", "2016-12-01", "2016-12-29", "2018-07-22", "2018-09-07", "2019-03-08" ], "times": [ "12:02:23", "02:11:11", "05:27:51", "20:15:55", "03:57:10", "01:54:42", "19:55:31", "01:42:54", "03:41:06" ] } }
+
diff --git a/asterixdb/asterix-app/data/yelp-checkin/use-case-3.json b/asterixdb/asterix-app/data/yelp-checkin/use-case-3.json
new file mode 100644
index 0000000..c5a711f
--- /dev/null
+++ b/asterixdb/asterix-app/data/yelp-checkin/use-case-3.json
@@ -0,0 +1,30 @@
+{ "business_id": "--1UhMGODdWsrMastO9DZw", "checkin_times": [ {"date": "2016-04-26", "time": "19:49:16"}, {"date": "2016-08-30", "time": "18:36:57"}, {"date": "2016-10-15", "time": "02:45:18"}, {"date": "2016-11-18", "time": "01:54:50"}, {"date": "2017-04-20", "time": "18:39:06"}, {"date": "2017-05-03", "time": "17:58:02"}, {"date": "2019-03-19", "time": "22:04:48"} ] }
+{ "business_id": "--EF5N7P70J_UYBTPypYlA", "checkin_times": [ {"date": "2018-05-25", "time": "19:52:07"}, {"date": "2018-09-18", "time": "16:09:44"}, {"date": "2019-10-18", "time": "21:29:09"} ] }
+{ "business_id": "--Ni3oJ4VOqfOEu7Sj2Vzg", "checkin_times": [ {"date": "2019-06-07", "time": "17:54:58"} ] }
+{ "business_id": "--Y1Adl1YUWfYIRSd8vkmA", "checkin_times": [ {"date": "2011-05-03", "time": "20:54:05"}, {"date": "2011-08-23", "time": "20:49:45"}, {"date": "2014-12-04", "time": "06:13:01"}, {"date": "2016-11-16", "time": "19:25:55"} ] }
+{ "business_id": "--YPwqIlRJrhHkJcjY3eiA", "checkin_times": [ {"date": "2016-06-18", "time": "21:35:45"}, {"date": "2016-10-15", "time": "18:17:51"} ] }
+{ "business_id": "--e8PjCNhEz32pprnPhCwQ", "checkin_times": [ {"date": "2015-04-02", "time": "21:45:17"} ] }
+{ "business_id": "--kinfHwmtdjz03g8B8z8Q", "checkin_times": [ {"date": "2014-08-27", "time": "17:49:18"}, {"date": "2015-12-19", "time": "21:30:31"}, {"date": "2018-11-27", "time": "15:53:50"} ] }
+{ "business_id": "--q6datkI-f0EoVheXNEeQ", "checkin_times": [ {"date": "2014-01-28", "time": "20:56:04"}, {"date": "2014-11-16", "time": "16:11:58"}, {"date": "2015-11-15", "time": "19:21:53"}, {"date": "2015-11-15", "time": "19:33:39"} ] }
+{ "business_id": "--qvQS4MigHPykD2GV0-zw", "checkin_times": [ {"date": "2019-04-11", "time": "18:30:12"} ] }
+{ "business_id": "--wIGbLEhlpl_UeAIyDmZQ", "checkin_times": [ {"date": "2015-06-06", "time": "20:01:06"}, {"date": "2019-03-14", "time": "22:01:52"} ] }
+{ "business_id": "-0FA-Qdi3SPYIoJz9UQw-A", "checkin_times": [ {"date": "2018-09-29", "time": "18:55:17"}, {"date": "2018-10-20", "time": "16:48:05"}, {"date": "2018-10-20", "time": "22:20:24"} ] }
+{ "business_id": "-0Hj1hb_XW6ybWq2M7QhGA", "checkin_times": [ {"date": "2011-04-23", "time": "21:11:22"}, {"date": "2014-05-04", "time": "19:42:48"}, {"date": "2014-05-11", "time": "19:16:08"}, {"date": "2014-06-04", "time": "19:14:18"}, {"date": "2015-12-05", "time": "19:22:42"}, {"date": "2017-05-15", "time": "23:19:00"} ] }
+{ "business_id": "-0KMvRFwDWdVBeTpT11iHw", "checkin_times": [ {"date": "2012-07-13", "time": "21:43:57"}, {"date": "2016-12-24", "time": "02:27:31"}, {"date": "2017-08-31", "time": "00:35:26"} ] }
+{ "business_id": "-0LPtgJC31FWMrMv317p0Q", "checkin_times": [ {"date": "2013-04-13", "time": "12:35:33"}, {"date": "2013-08-19", "time": "23:35:49"}, {"date": "2013-10-04", "time": "19:14:56"} ] }
+{ "business_id": "-0M3o2uWBnQZwd3hmfEwuw", "checkin_times": [ {"date": "2016-09-10", "time": "19:26:19"}, {"date": "2018-09-08", "time": "14:15:37"}, {"date": "2019-09-13", "time": "22:47:25"} ] }
+{ "business_id": "-0RRiWDtfnS16AKCtfvBZg", "checkin_times": [ {"date": "2017-05-19", "time": "14:30:16"}, {"date": "2017-05-19", "time": "14:30:25"}, {"date": "2017-08-28", "time": "15:49:37"}, {"date": "2017-09-20", "time": "20:19:51"}, {"date": "2017-10-01", "time": "16:31:05"}, {"date": "2017-10-01", "time": "16:56:27"}, {"date": "2017-12-27", "time": "23:33:20"} ] }
+{ "business_id": "-0Soj75v-XoRcf2ERr8Bmg", "checkin_times": [ {"date": "2019-06-05", "time": "18:22:49"} ] }
+{ "business_id": "-0ZumLlFjMh4ZW1z2nXGug", "checkin_times": [ {"date": "2011-09-24", "time": "21:37:32"}, {"date": "2014-03-10", "time": "20:20:07"}, {"date": "2015-05-27", "time": "00:40:24"}, {"date": "2015-08-29", "time": "17:58:15"}, {"date": "2018-03-16", "time": "15:03:26"} ] }
+{ "business_id": "-0aOudcaAyac0VJbMX-L1g", "checkin_times": [ {"date": "2015-03-16", "time": "23:51:16"}, {"date": "2015-12-21", "time": "04:48:01"}, {"date": "2016-10-28", "time": "20:22:42"}, {"date": "2016-10-28", "time": "20:23:00"} ] }
+{ "business_id": "-0b86isaXMY0v4g-V8GZ9Q", "checkin_times": [ {"date": "2013-10-22", "time": "16:49:21"}, {"date": "2014-11-21", "time": "17:39:24"} ] }
+{ "business_id": "-0d-BfFSU0bwLcnMaGRxYw", "checkin_times": [ {"date": "2014-08-07", "time": "18:30:48"}, {"date": "2014-09-16", "time": "20:41:45"}, {"date": "2014-10-12", "time": "23:22:27"}, {"date": "2015-07-21", "time": "20:43:56"}, {"date": "2015-07-21", "time": "20:45:07"} ] }
+{ "business_id": "-0jz6c3C6i7RG7Ag22K-Pg", "checkin_times": [ {"date": "2015-05-02", "time": "19:49:05"}, {"date": "2015-05-06", "time": "03:52:18"}, {"date": "2015-09-26", "time": "01:13:19"} ] }
+{ "business_id": "-0y3MZU2oYP8r1ruDP1bfQ", "checkin_times": [ {"date": "2015-04-11", "time": "13:14:14"}, {"date": "2015-11-21", "time": "16:05:56"}, {"date": "2016-05-06", "time": "14:10:04"}, {"date": "2017-08-09", "time": "15:15:10"}, {"date": "2017-10-21", "time": "15:12:56"} ] }
+{ "business_id": "-1BPe8UjF2_l3nVk-DFUjA", "checkin_times": [ {"date": "2015-12-03", "time": "18:44:00"}, {"date": "2016-03-17", "time": "18:19:21"}, {"date": "2016-11-02", "time": "15:58:38"} ] }
+{ "business_id": "-1E2CQu_38mkghvmZgCCRw", "checkin_times": [ {"date": "2019-04-04", "time": "22:02:37"} ] }
+{ "business_id": "-1wzk43IZ5D9Ysu6kzb5xA", "checkin_times": [ {"date": "2019-02-27", "time": "14:03:08"} ] }
+{ "business_id": "-23R9P2eG7VTc6DVLjFKzA", "checkin_times": [ {"date": "2011-12-21", "time": "19:02:51"}, {"date": "2012-04-15", "time": "04:21:39"}, {"date": "2012-04-15", "time": "14:23:56"}, {"date": "2013-06-30", "time": "22:39:51"}, {"date": "2013-10-04", "time": "20:34:13"}, {"date": "2014-07-16", "time": "02:28:40"} ] }
+{ "business_id": "-26MGfikhJiTfCI-GqmzhQ", "checkin_times": [ {"date": "2018-06-13", "time": "20:16:07"} ] }
+{ "business_id": "-2bLuJsMZ0WhI9daurVQNQ", "checkin_times": [ {"date": "2015-05-29", "time": "16:46:17"}, {"date": "2015-06-01", "time": "15:03:53"} ] }
+{ "business_id": "-2hDBMaza_ldqnZdiU06LQ", "checkin_times": [ {"date": "2011-10-08", "time": "12:02:23"}, {"date": "2014-08-18", "time": "02:11:11"}, {"date": "2016-01-07", "time": "05:27:51"}, {"date": "2016-10-21", "time": "20:15:55"}, {"date": "2016-12-01", "time": "03:57:10"}, {"date": "2016-12-29", "time": "01:54:42"}, {"date": "2018-07-22", "time": "19:55:31"}, {"date": "2018-09-07", "time": "01:42:54"}, {"date": "2019-03-08", "time": "03:41:06"} ] }
diff --git a/asterixdb/asterix-app/data/yelp-checkin/use-case-4.json b/asterixdb/asterix-app/data/yelp-checkin/use-case-4.json
new file mode 100644
index 0000000..ec90bb4
--- /dev/null
+++ b/asterixdb/asterix-app/data/yelp-checkin/use-case-4.json
@@ -0,0 +1,30 @@
+{ "business_id": "--1UhMGODdWsrMastO9DZw", "checkin_times": [ { "dates": ["2016-04-26", "2016-08-30", "2016-10-15", "2016-11-18"], "times": ["19:49:16", "18:36:57", "02:45:18", "01:54:50"] }, { "dates": ["2017-04-20", "2017-05-03"], "times": ["18:39:06", "17:58:02"] }, { "dates": ["2019-03-19"], "times": ["22:04:48"] } ] }
+{ "business_id": "--EF5N7P70J_UYBTPypYlA", "checkin_times": [ { "dates": ["2018-05-25", "2018-09-18"], "times": ["19:52:07", "16:09:44"] }, { "dates": ["2019-10-18"], "times": ["21:29:09"] } ] }
+{ "business_id": "--Ni3oJ4VOqfOEu7Sj2Vzg", "checkin_times": [ { "dates": ["2019-06-07"], "times": ["17:54:58"] } ] }
+{ "business_id": "--Y1Adl1YUWfYIRSd8vkmA", "checkin_times": [ { "dates": ["2011-05-03", "2011-08-23"], "times": ["20:54:05", "20:49:45"] }, { "dates": ["2014-12-04"], "times": ["06:13:01"] }, { "dates": ["2016-11-16"], "times": ["19:25:55"] } ] }
+{ "business_id": "--YPwqIlRJrhHkJcjY3eiA", "checkin_times": [ { "dates": ["2016-06-18", "2016-10-15"], "times": ["21:35:45", "18:17:51"] } ] }
+{ "business_id": "--e8PjCNhEz32pprnPhCwQ", "checkin_times": [ { "dates": ["2015-04-02"], "times": ["21:45:17"] } ] }
+{ "business_id": "--kinfHwmtdjz03g8B8z8Q", "checkin_times": [ { "dates": ["2014-08-27"], "times": ["17:49:18"] }, { "dates": ["2015-12-19"], "times": ["21:30:31"] }, { "dates": ["2018-11-27"], "times": ["15:53:50"] } ] }
+{ "business_id": "--q6datkI-f0EoVheXNEeQ", "checkin_times": [ { "dates": ["2014-01-28", "2014-11-16"], "times": ["20:56:04", "16:11:58"] }, { "dates": ["2015-11-15", "2015-11-15"], "times": ["19:21:53", "19:33:39"] } ] }
+{ "business_id": "--qvQS4MigHPykD2GV0-zw", "checkin_times": [ { "dates": ["2019-04-11"], "times": ["18:30:12"] } ] }
+{ "business_id": "--wIGbLEhlpl_UeAIyDmZQ", "checkin_times": [ { "dates": ["2015-06-06"], "times": ["20:01:06"] }, { "dates": ["2019-03-14"], "times": ["22:01:52"] } ] }
+{ "business_id": "-0FA-Qdi3SPYIoJz9UQw-A", "checkin_times": [ { "dates": ["2018-09-29", "2018-10-20", "2018-10-20"], "times": ["18:55:17", "16:48:05", "22:20:24"] } ] }
+{ "business_id": "-0Hj1hb_XW6ybWq2M7QhGA", "checkin_times": [ { "dates": ["2011-04-23"], "times": ["21:11:22"] }, { "dates": ["2014-05-04", "2014-05-11", "2014-06-04"], "times": ["19:42:48", "19:16:08", "19:14:18"] }, { "dates": ["2015-12-05"], "times": ["19:22:42"] }, { "dates": ["2017-05-15"], "times": ["23:19:00"] } ] }
+{ "business_id": "-0KMvRFwDWdVBeTpT11iHw", "checkin_times": [ { "dates": ["2012-07-13"], "times": ["21:43:57"] }, { "dates": ["2016-12-24"], "times": ["02:27:31"] }, { "dates": ["2017-08-31"], "times": ["00:35:26"] } ] }
+{ "business_id": "-0LPtgJC31FWMrMv317p0Q", "checkin_times": [ { "dates": ["2013-04-13", "2013-08-19", "2013-10-04"], "times": ["12:35:33", "23:35:49", "19:14:56"] } ] }
+{ "business_id": "-0M3o2uWBnQZwd3hmfEwuw", "checkin_times": [ { "dates": ["2016-09-10"], "times": ["19:26:19"] }, { "dates": ["2018-09-08"], "times": ["14:15:37"] }, { "dates": ["2019-09-13"], "times": ["22:47:25"] } ] }
+{ "business_id": "-0RRiWDtfnS16AKCtfvBZg", "checkin_times": [ { "dates": ["2017-05-19", "2017-05-19", "2017-08-28", "2017-09-20", "2017-10-01", "2017-10-01", "2017-12-27"], "times": ["14:30:16", "14:30:25", "15:49:37", "20:19:51", "16:31:05", "16:56:27", "23:33:20"] } ] }
+{ "business_id": "-0Soj75v-XoRcf2ERr8Bmg", "checkin_times": [ { "dates": ["2019-06-05"], "times": ["18:22:49"] } ] }
+{ "business_id": "-0ZumLlFjMh4ZW1z2nXGug", "checkin_times": [ { "dates": ["2011-09-24"], "times": ["21:37:32"] }, { "dates": ["2014-03-10"], "times": ["20:20:07"] }, { "dates": ["2015-05-27", "2015-08-29"], "times": ["00:40:24", "17:58:15"] }, { "dates": ["2018-03-16"], "times": ["15:03:26"] } ] }
+{ "business_id": "-0aOudcaAyac0VJbMX-L1g", "checkin_times": [ { "dates": ["2015-03-16", "2015-12-21"], "times": ["23:51:16", "04:48:01"] }, { "dates": ["2016-10-28", "2016-10-28"], "times": ["20:22:42", "20:23:00"] } ] }
+{ "business_id": "-0b86isaXMY0v4g-V8GZ9Q", "checkin_times": [ { "dates": ["2013-10-22"], "times": ["16:49:21"] }, { "dates": ["2014-11-21"], "times": ["17:39:24"] } ] }
+{ "business_id": "-0d-BfFSU0bwLcnMaGRxYw", "checkin_times": [ { "dates": ["2014-08-07", "2014-09-16", "2014-10-12"], "times": ["18:30:48", "20:41:45", "23:22:27"] }, { "dates": ["2015-07-21", "2015-07-21"], "times": ["20:43:56", "20:45:07"] } ] }
+{ "business_id": "-0jz6c3C6i7RG7Ag22K-Pg", "checkin_times": [ { "dates": ["2015-05-02", "2015-05-06", "2015-09-26"], "times": ["19:49:05", "03:52:18", "01:13:19"] } ] }
+{ "business_id": "-0y3MZU2oYP8r1ruDP1bfQ", "checkin_times": [ { "dates": ["2015-04-11", "2015-11-21"], "times": ["13:14:14", "16:05:56"] }, { "dates": ["2016-05-06"], "times": ["14:10:04"] }, { "dates": ["2017-08-09", "2017-10-21"], "times": ["15:15:10", "15:12:56"] } ] }
+{ "business_id": "-1BPe8UjF2_l3nVk-DFUjA", "checkin_times": [ { "dates": ["2015-12-03"], "times": ["18:44:00"] }, { "dates": ["2016-03-17", "2016-11-02"], "times": ["18:19:21", "15:58:38"] } ] }
+{ "business_id": "-1E2CQu_38mkghvmZgCCRw", "checkin_times": [ { "dates": ["2019-04-04"], "times": ["22:02:37"] } ] }
+{ "business_id": "-1wzk43IZ5D9Ysu6kzb5xA", "checkin_times": [ { "dates": ["2019-02-27"], "times": ["14:03:08"] } ] }
+{ "business_id": "-23R9P2eG7VTc6DVLjFKzA", "checkin_times": [ { "dates": ["2011-12-21"], "times": ["19:02:51"] }, { "dates": ["2012-04-15", "2012-04-15"], "times": ["04:21:39", "14:23:56"] }, { "dates": ["2013-06-30", "2013-10-04"], "times": ["22:39:51", "20:34:13"] }, { "dates": ["2014-07-16"], "times": ["02:28:40"] } ] }
+{ "business_id": "-26MGfikhJiTfCI-GqmzhQ", "checkin_times": [ { "dates": ["2018-06-13"], "times": ["20:16:07"] } ] }
+{ "business_id": "-2bLuJsMZ0WhI9daurVQNQ", "checkin_times": [ { "dates": ["2015-05-29", "2015-06-01"], "times": ["16:46:17", "15:03:53"] } ] }
+{ "business_id": "-2hDBMaza_ldqnZdiU06LQ", "checkin_times": [ { "dates": ["2011-10-08"], "times": ["12:02:23"] }, { "dates": ["2014-08-18"], "times": ["02:11:11"] }, { "dates": ["2016-01-07", "2016-10-21", "2016-12-01", "2016-12-29"], "times": ["05:27:51", "20:15:55", "03:57:10", "01:54:42"] }, { "dates": ["2018-07-22", "2018-09-07"], "times": ["19:55:31", "01:42:54"] }, { "dates": ["2019-03-08"], "times": ["03:41:06"] } ] }
diff --git a/asterixdb/asterix-app/data/yelp-checkin/with-3-level-record-path.json b/asterixdb/asterix-app/data/yelp-checkin/with-3-level-record-path.json
new file mode 100644
index 0000000..39505f0
--- /dev/null
+++ b/asterixdb/asterix-app/data/yelp-checkin/with-3-level-record-path.json
@@ -0,0 +1,31 @@
+{ "business_id": "--1UhMGODdWsrMastO9DZw", "checkin_data": { "checkin_temporal": { "checkin_times": { "dates": [ "2016-04-26", "2016-08-30", "2016-10-15", "2016-11-18", "2017-04-20", "2017-05-03", "2019-03-19" ], "times": [ "19:49:16", "18:36:57", "02:45:18", "01:54:50", "18:39:06", "17:58:02", "22:04:48" ] } } } }
+{ "business_id": "--EF5N7P70J_UYBTPypYlA", "checkin_data": { "checkin_temporal": { "checkin_times": { "dates": [ "2018-05-25", "2018-09-18", "2019-10-18" ], "times": [ "19:52:07", "16:09:44", "21:29:09" ] } } } }
+{ "business_id": "--Ni3oJ4VOqfOEu7Sj2Vzg", "checkin_data": { "checkin_temporal": { "checkin_times": { "dates": [ "2019-06-07" ], "times": [ "17:54:58" ] } } } }
+{ "business_id": "--Y1Adl1YUWfYIRSd8vkmA", "checkin_data": { "checkin_temporal": { "checkin_times": { "dates": [ "2011-05-03", "2011-08-23", "2014-12-04", "2016-11-16" ], "times": [ "20:54:05", "20:49:45", "06:13:01", "19:25:55" ] } } } }
+{ "business_id": "--YPwqIlRJrhHkJcjY3eiA", "checkin_data": { "checkin_temporal": { "checkin_times": { "dates": [ "2016-06-18", "2016-10-15" ], "times": [ "21:35:45", "18:17:51" ] } } } }
+{ "business_id": "--e8PjCNhEz32pprnPhCwQ", "checkin_data": { "checkin_temporal": { "checkin_times": { "dates": [ "2015-04-02" ], "times": [ "21:45:17" ] } } } }
+{ "business_id": "--kinfHwmtdjz03g8B8z8Q", "checkin_data": { "checkin_temporal": { "checkin_times": { "dates": [ "2014-08-27", "2015-12-19", "2018-11-27" ], "times": [ "17:49:18", "21:30:31", "15:53:50" ] } } } }
+{ "business_id": "--q6datkI-f0EoVheXNEeQ", "checkin_data": { "checkin_temporal": { "checkin_times": { "dates": [ "2014-01-28", "2014-11-16", "2015-11-15", "2015-11-15" ], "times": [ "20:56:04", "16:11:58", "19:21:53", "19:33:39" ] } } } }
+{ "business_id": "--qvQS4MigHPykD2GV0-zw", "checkin_data": { "checkin_temporal": { "checkin_times": { "dates": [ "2019-04-11" ], "times": [ "18:30:12" ] } } } }
+{ "business_id": "--wIGbLEhlpl_UeAIyDmZQ", "checkin_data": { "checkin_temporal": { "checkin_times": { "dates": [ "2015-06-06", "2019-03-14" ], "times": [ "20:01:06", "22:01:52" ] } } } }
+{ "business_id": "-0FA-Qdi3SPYIoJz9UQw-A", "checkin_data": { "checkin_temporal": { "checkin_times": { "dates": [ "2018-09-29", "2018-10-20", "2018-10-20" ], "times": [ "18:55:17", "16:48:05", "22:20:24" ] } } } }
+{ "business_id": "-0Hj1hb_XW6ybWq2M7QhGA", "checkin_data": { "checkin_temporal": { "checkin_times": { "dates": [ "2011-04-23", "2014-05-04", "2014-05-11", "2014-06-04", "2015-12-05", "2017-05-15" ], "times": [ "21:11:22", "19:42:48", "19:16:08", "19:14:18", "19:22:42", "23:19:00" ] } } } }
+{ "business_id": "-0KMvRFwDWdVBeTpT11iHw", "checkin_data": { "checkin_temporal": { "checkin_times": { "dates": [ "2012-07-13", "2016-12-24", "2017-08-31" ], "times": [ "21:43:57", "02:27:31", "00:35:26" ] } } } }
+{ "business_id": "-0LPtgJC31FWMrMv317p0Q", "checkin_data": { "checkin_temporal": { "checkin_times": { "dates": [ "2013-04-13", "2013-08-19", "2013-10-04" ], "times": [ "12:35:33", "23:35:49", "19:14:56" ] } } } }
+{ "business_id": "-0M3o2uWBnQZwd3hmfEwuw", "checkin_data": { "checkin_temporal": { "checkin_times": { "dates": [ "2016-09-10", "2018-09-08", "2019-09-13" ], "times": [ "19:26:19", "14:15:37", "22:47:25" ] } } } }
+{ "business_id": "-0RRiWDtfnS16AKCtfvBZg", "checkin_data": { "checkin_temporal": { "checkin_times": { "dates": [ "2017-05-19", "2017-05-19", "2017-08-28", "2017-09-20", "2017-10-01", "2017-10-01", "2017-12-27" ], "times": [ "14:30:16", "14:30:25", "15:49:37", "20:19:51", "16:31:05", "16:56:27", "23:33:20" ] } } } }
+{ "business_id": "-0Soj75v-XoRcf2ERr8Bmg", "checkin_data": { "checkin_temporal": { "checkin_times": { "dates": [ "2019-06-05" ], "times": [ "18:22:49" ] } } } }
+{ "business_id": "-0ZumLlFjMh4ZW1z2nXGug", "checkin_data": { "checkin_temporal": { "checkin_times": { "dates": [ "2011-09-24", "2014-03-10", "2015-05-27", "2015-08-29", "2018-03-16" ], "times": [ "21:37:32", "20:20:07", "00:40:24", "17:58:15", "15:03:26" ] } } } }
+{ "business_id": "-0aOudcaAyac0VJbMX-L1g", "checkin_data": { "checkin_temporal": { "checkin_times": { "dates": [ "2015-03-16", "2015-12-21", "2016-10-28", "2016-10-28" ], "times": [ "23:51:16", "04:48:01", "20:22:42", "20:23:00" ] } } } }
+{ "business_id": "-0b86isaXMY0v4g-V8GZ9Q", "checkin_data": { "checkin_temporal": { "checkin_times": { "dates": [ "2013-10-22", "2014-11-21" ], "times": [ "16:49:21", "17:39:24" ] } } } }
+{ "business_id": "-0d-BfFSU0bwLcnMaGRxYw", "checkin_data": { "checkin_temporal": { "checkin_times": { "dates": [ "2014-08-07", "2014-09-16", "2014-10-12", "2015-07-21", "2015-07-21" ], "times": [ "18:30:48", "20:41:45", "23:22:27", "20:43:56", "20:45:07" ] } } } }
+{ "business_id": "-0jz6c3C6i7RG7Ag22K-Pg", "checkin_data": { "checkin_temporal": { "checkin_times": { "dates": [ "2015-05-02", "2015-05-06", "2015-09-26" ], "times": [ "19:49:05", "03:52:18", "01:13:19" ] } } } }
+{ "business_id": "-0y3MZU2oYP8r1ruDP1bfQ", "checkin_data": { "checkin_temporal": { "checkin_times": { "dates": [ "2015-04-11", "2015-11-21", "2016-05-06", "2017-08-09", "2017-10-21" ], "times": [ "13:14:14", "16:05:56", "14:10:04", "15:15:10", "15:12:56" ] } } } }
+{ "business_id": "-1BPe8UjF2_l3nVk-DFUjA", "checkin_data": { "checkin_temporal": { "checkin_times": { "dates": [ "2015-12-03", "2016-03-17", "2016-11-02" ], "times": [ "18:44:00", "18:19:21", "15:58:38" ] } } } }
+{ "business_id": "-1E2CQu_38mkghvmZgCCRw", "checkin_data": { "checkin_temporal": { "checkin_times": { "dates": [ "2019-04-04" ], "times": [ "22:02:37" ] } } } }
+{ "business_id": "-1wzk43IZ5D9Ysu6kzb5xA", "checkin_data": { "checkin_temporal": { "checkin_times": { "dates": [ "2019-02-27" ], "times": [ "14:03:08" ] } } } }
+{ "business_id": "-23R9P2eG7VTc6DVLjFKzA", "checkin_data": { "checkin_temporal": { "checkin_times": { "dates": [ "2011-12-21", "2012-04-15", "2012-04-15", "2013-06-30", "2013-10-04", "2014-07-16" ], "times": [ "19:02:51", "04:21:39", "14:23:56", "22:39:51", "20:34:13", "02:28:40" ] } } } }
+{ "business_id": "-26MGfikhJiTfCI-GqmzhQ", "checkin_data": { "checkin_temporal": { "checkin_times": { "dates": [ "2018-06-13" ], "times": [ "20:16:07" ] } } } }
+{ "business_id": "-2bLuJsMZ0WhI9daurVQNQ", "checkin_data": { "checkin_temporal": { "checkin_times": { "dates": [ "2015-05-29", "2015-06-01" ], "times": [ "16:46:17", "15:03:53" ] } } } }
+{ "business_id": "-2hDBMaza_ldqnZdiU06LQ", "checkin_data": { "checkin_temporal": { "checkin_times": { "dates": [ "2011-10-08", "2014-08-18", "2016-01-07", "2016-10-21", "2016-12-01", "2016-12-29", "2018-07-22", "2018-09-07", "2019-03-08" ], "times": [ "12:02:23", "02:11:11", "05:27:51", "20:15:55", "03:57:10", "01:54:42", "19:55:31", "01:42:54", "03:41:06" ] } } } }
+
diff --git a/asterixdb/asterix-app/data/yelp-checkin/with-composite-pk.json b/asterixdb/asterix-app/data/yelp-checkin/with-composite-pk.json
new file mode 100644
index 0000000..1611114
--- /dev/null
+++ b/asterixdb/asterix-app/data/yelp-checkin/with-composite-pk.json
@@ -0,0 +1,30 @@
+{ "checkin_id": 1, "business_id": "--1UhMGODdWsrMastO9DZw", "dates": [ "2016-04-26 19:49:16", "2016-08-30 18:36:57", "2016-10-15 02:45:18", "2016-11-18 01:54:50", "2017-04-20 18:39:06", "2017-05-03 17:58:02", "2019-03-19 22:04:48" ] }
+{ "checkin_id": 2, "business_id": "--EF5N7P70J_UYBTPypYlA", "dates": [ "2018-05-25 19:52:07", "2018-09-18 16:09:44", "2019-10-18 21:29:09" ] }
+{ "checkin_id": 3, "business_id": "--Ni3oJ4VOqfOEu7Sj2Vzg", "dates": [ "2019-06-07 17:54:58" ] }
+{ "checkin_id": 4, "business_id": "--Y1Adl1YUWfYIRSd8vkmA", "dates": [ "2011-05-03 20:54:05", "2011-08-23 20:49:45", "2014-12-04 06:13:01", "2016-11-16 19:25:55" ] }
+{ "checkin_id": 5, "business_id": "--YPwqIlRJrhHkJcjY3eiA", "dates": [ "2016-06-18 21:35:45", "2016-10-15 18:17:51" ] }
+{ "checkin_id": 6, "business_id": "--e8PjCNhEz32pprnPhCwQ", "dates": [ "2015-04-02 21:45:17" ] }
+{ "checkin_id": 7, "business_id": "--kinfHwmtdjz03g8B8z8Q", "dates": [ "2014-08-27 17:49:18", "2015-12-19 21:30:31", "2018-11-27 15:53:50" ] }
+{ "checkin_id": 8, "business_id": "--q6datkI-f0EoVheXNEeQ", "dates": [ "2014-01-28 20:56:04", "2014-11-16 16:11:58", "2015-11-15 19:21:53", "2015-11-15 19:33:39" ] }
+{ "checkin_id": 9, "business_id": "--qvQS4MigHPykD2GV0-zw", "dates": [ "2019-04-11 18:30:12" ] }
+{ "checkin_id": 10, "business_id": "--wIGbLEhlpl_UeAIyDmZQ", "dates": [ "2015-06-06 20:01:06", "2019-03-14 22:01:52" ] }
+{ "checkin_id": 11, "business_id": "-0FA-Qdi3SPYIoJz9UQw-A", "dates": [ "2018-09-29 18:55:17", "2018-10-20 16:48:05", "2018-10-20 22:20:24" ] }
+{ "checkin_id": 12, "business_id": "-0Hj1hb_XW6ybWq2M7QhGA", "dates": [ "2011-04-23 21:11:22", "2014-05-04 19:42:48", "2014-05-11 19:16:08", "2014-06-04 19:14:18", "2015-12-05 19:22:42", "2017-05-15 23:19:00" ] }
+{ "checkin_id": 13, "business_id": "-0KMvRFwDWdVBeTpT11iHw", "dates": [ "2012-07-13 21:43:57", "2016-12-24 02:27:31", "2017-08-31 00:35:26" ] }
+{ "checkin_id": 14, "business_id": "-0LPtgJC31FWMrMv317p0Q", "dates": [ "2013-04-13 12:35:33", "2013-08-19 23:35:49", "2013-10-04 19:14:56" ] }
+{ "checkin_id": 15, "business_id": "-0M3o2uWBnQZwd3hmfEwuw", "dates": [ "2016-09-10 19:26:19", "2018-09-08 14:15:37", "2019-09-13 22:47:25" ] }
+{ "checkin_id": 16, "business_id": "-0RRiWDtfnS16AKCtfvBZg", "dates": [ "2017-05-19 14:30:16", "2017-05-19 14:30:25", "2017-08-28 15:49:37", "2017-09-20 20:19:51", "2017-10-01 16:31:05", "2017-10-01 16:56:27", "2017-12-27 23:33:20" ] }
+{ "checkin_id": 17, "business_id": "-0Soj75v-XoRcf2ERr8Bmg", "dates": [ "2019-06-05 18:22:49" ] }
+{ "checkin_id": 18, "business_id": "-0ZumLlFjMh4ZW1z2nXGug", "dates": [ "2011-09-24 21:37:32", "2014-03-10 20:20:07", "2015-05-27 00:40:24", "2015-08-29 17:58:15", "2018-03-16 15:03:26" ] }
+{ "checkin_id": 19, "business_id": "-0aOudcaAyac0VJbMX-L1g", "dates": [ "2015-03-16 23:51:16", "2015-12-21 04:48:01", "2016-10-28 20:22:42", "2016-10-28 20:23:00" ] }
+{ "checkin_id": 20, "business_id": "-0b86isaXMY0v4g-V8GZ9Q", "dates": [ "2013-10-22 16:49:21", "2014-11-21 17:39:24" ] }
+{ "checkin_id": 21, "business_id": "-0d-BfFSU0bwLcnMaGRxYw", "dates": [ "2014-08-07 18:30:48", "2014-09-16 20:41:45", "2014-10-12 23:22:27", "2015-07-21 20:43:56", "2015-07-21 20:45:07" ] }
+{ "checkin_id": 22, "business_id": "-0jz6c3C6i7RG7Ag22K-Pg", "dates": [ "2015-05-02 19:49:05", "2015-05-06 03:52:18", "2015-09-26 01:13:19" ] }
+{ "checkin_id": 23, "business_id": "-0y3MZU2oYP8r1ruDP1bfQ", "dates": [ "2015-04-11 13:14:14", "2015-11-21 16:05:56", "2016-05-06 14:10:04", "2017-08-09 15:15:10", "2017-10-21 15:12:56" ] }
+{ "checkin_id": 24, "business_id": "-1BPe8UjF2_l3nVk-DFUjA", "dates": [ "2015-12-03 18:44:00", "2016-03-17 18:19:21", "2016-11-02 15:58:38" ] }
+{ "checkin_id": 25, "business_id": "-1E2CQu_38mkghvmZgCCRw", "dates": [ "2019-04-04 22:02:37" ] }
+{ "checkin_id": 26, "business_id": "-1wzk43IZ5D9Ysu6kzb5xA", "dates": [ "2019-02-27 14:03:08" ] }
+{ "checkin_id": 27, "business_id": "-23R9P2eG7VTc6DVLjFKzA", "dates": [ "2011-12-21 19:02:51", "2012-04-15 04:21:39", "2012-04-15 14:23:56", "2013-06-30 22:39:51", "2013-10-04 20:34:13", "2014-07-16 02:28:40" ] }
+{ "checkin_id": 28, "business_id": "-26MGfikhJiTfCI-GqmzhQ", "dates": [ "2018-06-13 20:16:07" ] }
+{ "checkin_id": 29, "business_id": "-2bLuJsMZ0WhI9daurVQNQ", "dates": [ "2015-05-29 16:46:17", "2015-06-01 15:03:53" ] }
+{ "checkin_id": 30, "business_id": "-2hDBMaza_ldqnZdiU06LQ", "dates": [ "2011-10-08 12:02:23", "2014-08-18 02:11:11", "2016-01-07 05:27:51", "2016-10-21 20:15:55", "2016-12-01 03:57:10", "2016-12-29 01:54:42", "2018-07-22 19:55:31", "2018-09-07 01:42:54", "2019-03-08 03:41:06" ] }
diff --git a/asterixdb/asterix-app/src/main/java/org/apache/asterix/api/common/APIFramework.java b/asterixdb/asterix-app/src/main/java/org/apache/asterix/api/common/APIFramework.java
index 9088db6..270f7d2 100644
--- a/asterixdb/asterix-app/src/main/java/org/apache/asterix/api/common/APIFramework.java
+++ b/asterixdb/asterix-app/src/main/java/org/apache/asterix/api/common/APIFramework.java
@@ -135,7 +135,7 @@
             CompilerProperties.COMPILER_SORT_PARALLEL_KEY, CompilerProperties.COMPILER_SORT_SAMPLES_KEY,
             CompilerProperties.COMPILER_INDEXONLY_KEY, CompilerProperties.COMPILER_INTERNAL_SANITYCHECK_KEY,
             CompilerProperties.COMPILER_EXTERNAL_FIELD_PUSHDOWN_KEY, CompilerProperties.COMPILER_SUBPLAN_MERGE_KEY,
-            CompilerProperties.COMPILER_SUBPLAN_NESTEDPUSHDOWN_KEY,
+            CompilerProperties.COMPILER_SUBPLAN_NESTEDPUSHDOWN_KEY, CompilerProperties.COMPILER_ARRAYINDEX_KEY,
             CompilerProperties.COMPILER_MIN_MEMORY_ALLOCATION_KEY, FunctionUtil.IMPORT_PRIVATE_FUNCTIONS,
             FuzzyUtils.SIM_FUNCTION_PROP_NAME, FuzzyUtils.SIM_THRESHOLD_PROP_NAME,
             StartFeedStatement.WAIT_FOR_COMPLETION, FeedActivityDetails.FEED_POLICY_NAME,
diff --git a/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/translator/QueryTranslator.java b/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/translator/QueryTranslator.java
index 8251241..778e79a 100644
--- a/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/translator/QueryTranslator.java
+++ b/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/translator/QueryTranslator.java
@@ -39,6 +39,7 @@
 import java.util.Properties;
 import java.util.Set;
 import java.util.concurrent.ExecutorService;
+import java.util.stream.Collectors;
 
 import org.apache.asterix.active.ActivityState;
 import org.apache.asterix.active.EntityId;
@@ -1041,7 +1042,9 @@
 
             DatasetType datasetType = ds.getDatasetType();
             IndexType indexType = stmtCreateIndex.getIndexType();
-            boolean isSecondaryPrimary = stmtCreateIndex.getFieldExprs().isEmpty();
+            List<CreateIndexStatement.IndexedElement> indexedElements = stmtCreateIndex.getIndexedElements();
+            int indexedElementsCount = indexedElements.size();
+            boolean isSecondaryPrimary = indexedElementsCount == 0;
             validateIndexType(datasetType, indexType, isSecondaryPrimary, sourceLoc);
 
             String indexName = stmtCreateIndex.getIndexName().getValue();
@@ -1056,111 +1059,234 @@
                 }
             }
 
-            List<Integer> keySourceIndicators;
-            if (isSecondaryPrimary && datasetType == DatasetType.INTERNAL) {
-                // find keySourceIndicators for secondary primary index since the parser isn't aware of them
-                keySourceIndicators = ((InternalDatasetDetails) ds.getDatasetDetails()).getKeySourceIndicator();
-            } else {
-                keySourceIndicators = stmtCreateIndex.getFieldSourceIndicators();
-            }
-            // disable creating an index on meta fields (fields with source indicator == 1 are meta fields)
-            if (keySourceIndicators.stream().anyMatch(fieldSource -> fieldSource == 1) && !isSecondaryPrimary) {
-                throw new AsterixException(ErrorCode.COMPILATION_ERROR, sourceLoc,
-                        "Cannot create index on meta fields");
-            }
             Datatype dt = MetadataManager.INSTANCE.getDatatype(metadataProvider.getMetadataTxnContext(),
                     ds.getItemTypeDataverseName(), ds.getItemTypeName());
             ARecordType aRecordType = (ARecordType) dt.getDatatype();
+            /* TODO: unused for now becase indexes on meta are disabled -- see below
             ARecordType metaRecordType = null;
             if (ds.hasMetaPart()) {
                 Datatype metaDt = MetadataManager.INSTANCE.getDatatype(metadataProvider.getMetadataTxnContext(),
                         ds.getMetaItemTypeDataverseName(), ds.getMetaItemTypeName());
                 metaRecordType = (ARecordType) metaDt.getDatatype();
             }
+            */
 
-            List<List<String>> indexFields = new ArrayList<>();
-            List<IAType> indexFieldTypes = new ArrayList<>();
-            int keyIndex = 0;
+            List<List<IAType>> indexFieldTypes = new ArrayList<>(indexedElementsCount);
+            boolean hadUnnest = false;
             boolean overridesFieldTypes = false;
 
             // this set is used to detect duplicates in the specified keys in the create
             // index statement
             // e.g. CREATE INDEX someIdx on dataset(id,id).
-            // checking only the names is not enough. Need also to check the source
-            // indicators for cases like:
-            // CREATE INDEX someIdx on dataset(meta().id, id)
-            Set<Pair<List<String>, Integer>> indexKeysSet = new HashSet<>();
+            // checking only the names is not enough.
+            // Need also to check the source indicators for the most general case
+            // (even though indexes on meta fields are curently disabled -- see below)
+            Set<Triple<Integer, List<List<String>>, List<List<String>>>> indexKeysSet = new HashSet<>();
 
-            for (Pair<List<String>, IndexedTypeExpression> fieldExpr : stmtCreateIndex.getFieldExprs()) {
-                IAType fieldType = null;
-                ARecordType subType =
-                        KeyFieldTypeUtil.chooseSource(keySourceIndicators, keyIndex, aRecordType, metaRecordType);
-                boolean isOpen = subType.isOpen();
-                int i = 0;
-                if (fieldExpr.first.size() > 1 && !isOpen) {
-                    while (i < fieldExpr.first.size() - 1 && !isOpen) {
-                        subType = (ARecordType) subType.getFieldType(fieldExpr.first.get(i));
-                        i++;
-                        isOpen = subType.isOpen();
-                    }
+            for (CreateIndexStatement.IndexedElement indexedElement : indexedElements) {
+                // disable creating an index on meta fields (fields with source indicator == 1 are meta fields)
+                if (indexedElement.getSourceIndicator() != Index.RECORD_INDICATOR) {
+                    throw new AsterixException(ErrorCode.COMPILATION_ERROR, indexedElement.getSourceLocation(),
+                            "Cannot create index on meta fields");
                 }
-                if (fieldExpr.second == null) {
-                    fieldType = subType.getSubFieldType(fieldExpr.first.subList(i, fieldExpr.first.size()));
+                ARecordType sourceRecordType = aRecordType;
+                IAType inputTypePrime;
+                boolean inputTypeNullable, inputTypeMissable;
+                List<Pair<List<String>, IndexedTypeExpression>> projectList = indexedElement.getProjectList();
+                int projectCount = projectList.size();
+                if (indexedElement.hasUnnest()) {
+                    if (indexType != IndexType.ARRAY) {
+                        throw new CompilationException(ErrorCode.COMPILATION_INCOMPATIBLE_INDEX_TYPE,
+                                indexedElement.getSourceLocation(), String.valueOf(indexType));
+                    }
+                    // allow only 1 unnesting element in ARRAY index
+                    if (hadUnnest) {
+                        throw new CompilationException(ErrorCode.COMPILATION_ERROR, indexedElement.getSourceLocation(),
+                                "Cannot create composite index with multiple array fields using different arrays");
+                    }
+                    hadUnnest = true;
+                    if (projectCount == 0) {
+                        // Note. UNNEST with no SELECT is supposed to have 1 project element with 'null' path
+                        throw new CompilationException(ErrorCode.COMPILATION_ERROR, indexedElement.getSourceLocation(),
+                                "Invalid index element");
+                    }
+                    Triple<IAType, Boolean, Boolean> unnestTypeResult = KeyFieldTypeUtil.getKeyUnnestType(
+                            sourceRecordType, indexedElement.getUnnestList(), indexedElement.getSourceLocation());
+                    if (unnestTypeResult == null) {
+                        inputTypePrime = null; // = ANY
+                        inputTypeNullable = inputTypeMissable = true;
+                    } else {
+                        inputTypePrime = unnestTypeResult.first;
+                        inputTypeNullable = unnestTypeResult.second;
+                        inputTypeMissable = unnestTypeResult.third;
+                    }
                 } else {
-                    if (!stmtCreateIndex.isEnforced() && indexType != IndexType.BTREE) {
-                        throw new AsterixException(ErrorCode.INDEX_ILLEGAL_NON_ENFORCED_TYPED, sourceLoc, indexType);
+                    if (projectCount != 1) {
+                        throw new CompilationException(ErrorCode.COMPILATION_ERROR, indexedElement.getSourceLocation(),
+                                "Invalid index element");
                     }
-                    if (stmtCreateIndex.isEnforced() && !fieldExpr.second.isUnknownable()) {
-                        throw new AsterixException(ErrorCode.INDEX_ILLEGAL_ENFORCED_NON_OPTIONAL, sourceLoc,
-                                String.valueOf(fieldExpr.first));
-                    }
-                    // don't allow creating an enforced index on a closed-type field, fields that
-                    // are part of schema.
-                    // get the field type, if it's not null, then the field is closed-type
-                    if (stmtCreateIndex.isEnforced()
-                            && subType.getSubFieldType(fieldExpr.first.subList(i, fieldExpr.first.size())) != null) {
-                        throw new AsterixException(ErrorCode.INDEX_ILLEGAL_ENFORCED_ON_CLOSED_FIELD, sourceLoc,
-                                String.valueOf(fieldExpr.first));
-                    }
-                    if (!isOpen) {
-                        throw new CompilationException(ErrorCode.COMPILATION_ERROR, sourceLoc, "Typed index on \""
-                                + fieldExpr.first + "\" field could be created only for open datatype");
-                    }
-                    if (stmtCreateIndex.hasMetaField()) {
-                        throw new CompilationException(ErrorCode.COMPILATION_ERROR, sourceLoc,
-                                "Typed open index can only be created on the record part");
-                    }
-                    Map<TypeSignature, IAType> typeMap = TypeTranslator.computeTypes(dataverseName, indexName,
-                            fieldExpr.second.getType(), dataverseName, mdTxnCtx);
-                    TypeSignature typeSignature = new TypeSignature(dataverseName, indexName);
-                    fieldType = typeMap.get(typeSignature);
-                    overridesFieldTypes = true;
-                }
-                if (fieldType == null) {
-                    throw new CompilationException(ErrorCode.UNKNOWN_TYPE, sourceLoc, fieldExpr.second == null
-                            ? String.valueOf(fieldExpr.first) : String.valueOf(fieldExpr.second));
+                    inputTypePrime = sourceRecordType;
+                    inputTypeNullable = inputTypeMissable = false;
                 }
 
-                // try to add the key & its source to the set of keys, if key couldn't be added,
-                // there is a duplicate
-                if (!indexKeysSet
-                        .add(new Pair<>(fieldExpr.first, stmtCreateIndex.getFieldSourceIndicators().get(keyIndex)))) {
-                    throw new AsterixException(ErrorCode.INDEX_ILLEGAL_REPETITIVE_FIELD, sourceLoc,
-                            String.valueOf(fieldExpr.first));
+                // at this point 'inputTypePrime' is either a record, or if we had unnest then it could also be anything else.
+                List<IAType> fieldTypes = new ArrayList<>(projectCount);
+                for (int i = 0; i < projectCount; i++) {
+                    Pair<List<String>, IndexedTypeExpression> projectPair = projectList.get(i);
+                    List<String> projectPath = projectPair.first;
+                    IndexedTypeExpression projectTypeExpr = projectPair.second;
+                    IAType projectTypePrime;
+                    boolean projectTypeNullable, projectTypeMissable;
+                    if (projectPath == null) {
+                        boolean emptyPathOk = indexedElement.hasUnnest() && i == 0;
+                        if (!emptyPathOk) {
+                            throw new CompilationException(ErrorCode.COMPILATION_ERROR,
+                                    indexedElement.getSourceLocation(), "Invalid index element");
+                        }
+                        projectTypePrime = inputTypePrime;
+                        projectTypeNullable = inputTypeNullable;
+                        projectTypeMissable = inputTypeMissable;
+                    } else if (inputTypePrime == null) {
+                        projectTypePrime = null; // ANY
+                        projectTypeNullable = projectTypeMissable = true;
+                    } else {
+                        if (inputTypePrime.getTypeTag() != ATypeTag.OBJECT) {
+                            throw new CompilationException(ErrorCode.TYPE_MISMATCH_GENERIC, sourceLoc, ATypeTag.OBJECT,
+                                    inputTypePrime.getTypeTag());
+                        }
+                        ARecordType inputTypePrimeRecord = (ARecordType) inputTypePrime;
+                        Triple<IAType, Boolean, Boolean> projectTypeResult = KeyFieldTypeUtil.getKeyProjectType(
+                                inputTypePrimeRecord, projectPath, indexedElement.getSourceLocation());
+                        if (projectTypeResult != null) {
+                            projectTypePrime = projectTypeResult.first;
+                            projectTypeNullable = inputTypeNullable || projectTypeResult.second;
+                            projectTypeMissable = inputTypeMissable || projectTypeResult.third;
+                        } else {
+                            projectTypePrime = null; // ANY
+                            projectTypeNullable = projectTypeMissable = true;
+                        }
+                    }
+
+                    IAType fieldTypePrime;
+                    boolean fieldTypeNullable, fieldTypeMissable;
+                    if (projectTypeExpr == null) {
+                        fieldTypePrime = projectTypePrime;
+                        fieldTypeNullable = projectTypeNullable;
+                        fieldTypeMissable = projectTypeMissable;
+                    } else {
+                        if (stmtCreateIndex.isEnforced()) {
+                            if (!projectTypeExpr.isUnknownable()) {
+                                throw new CompilationException(ErrorCode.INDEX_ILLEGAL_ENFORCED_NON_OPTIONAL,
+                                        indexedElement.getSourceLocation(), String.valueOf(projectPath));
+                            }
+                            // don't allow creating an enforced index on a closed-type field, fields that
+                            // are part of schema get the field type, if it's not null, then the field is closed-type
+                            if (projectTypePrime != null) {
+                                throw new CompilationException(ErrorCode.INDEX_ILLEGAL_ENFORCED_ON_CLOSED_FIELD,
+                                        indexedElement.getSourceLocation(), String.valueOf(projectPath));
+                            }
+                        } else {
+                            if (indexType != IndexType.BTREE && indexType != IndexType.ARRAY) {
+                                throw new CompilationException(ErrorCode.INDEX_ILLEGAL_NON_ENFORCED_TYPED,
+                                        indexedElement.getSourceLocation(), indexType);
+                            }
+                            if (projectTypePrime != null) {
+                                throw new CompilationException(ErrorCode.COMPILATION_ERROR,
+                                        indexedElement.getSourceLocation(), "Typed index on \"" + projectPath
+                                                + "\" field could be created only for open datatype");
+                            }
+                        }
+
+                        Map<TypeSignature, IAType> typeMap = TypeTranslator.computeTypes(dataverseName, indexName,
+                                projectTypeExpr.getType(), dataverseName, mdTxnCtx);
+                        TypeSignature typeSignature = new TypeSignature(dataverseName, indexName);
+                        fieldTypePrime = typeMap.get(typeSignature);
+                        // BACK-COMPAT: keep prime type only if we're overriding field types
+                        fieldTypeNullable = fieldTypeMissable = false;
+                        overridesFieldTypes = true;
+                    }
+
+                    if (fieldTypePrime == null) {
+                        throw new CompilationException(ErrorCode.UNKNOWN_TYPE, indexedElement.getSourceLocation(),
+                                String.valueOf(projectPath));
+                    }
+                    validateIndexFieldType(indexType, fieldTypePrime, projectPath, indexedElement.getSourceLocation());
+
+                    IAType fieldType =
+                            KeyFieldTypeUtil.makeUnknownableType(fieldTypePrime, fieldTypeNullable, fieldTypeMissable);
+                    fieldTypes.add(fieldType);
                 }
 
-                indexFields.add(fieldExpr.first);
-                indexFieldTypes.add(fieldType);
-                ++keyIndex;
+                // Try to add the key & its source to the set of keys for duplicate detection.
+                if (!indexKeysSet.add(indexedElement.toIdentifier())) {
+                    throw new AsterixException(ErrorCode.INDEX_ILLEGAL_REPETITIVE_FIELD,
+                            indexedElement.getSourceLocation(), indexedElement.getProjectListDisplayForm());
+                }
+
+                indexFieldTypes.add(fieldTypes);
             }
 
-            validateIndexKeyFields(stmtCreateIndex, keySourceIndicators, aRecordType, metaRecordType, indexFields,
-                    indexFieldTypes);
+            Index.IIndexDetails indexDetails;
+            if (Index.IndexCategory.of(indexType) == Index.IndexCategory.ARRAY) {
+                if (!hadUnnest) {
+                    // prohibited by the grammar
+                    throw new CompilationException(ErrorCode.COMPILATION_ILLEGAL_STATE, sourceLoc,
+                            String.valueOf(indexType));
+                }
+                if (stmtCreateIndex.isEnforced()) {
+                    // not supported yet.
+                    throw new CompilationException(ErrorCode.COMPILATION_INCOMPATIBLE_INDEX_TYPE, sourceLoc,
+                            String.valueOf(indexType));
+                }
+                if (indexedElementsCount > 1) {
+                    // TODO (GLENN): Add in support for composite atomic / array indexes.
+                    throw new CompilationException(ErrorCode.COMPILATION_INCOMPATIBLE_INDEX_TYPE, sourceLoc,
+                            String.valueOf(indexType));
+                }
 
-            Index newIndex =
-                    new Index(dataverseName, datasetName, indexName, indexType, indexFields, keySourceIndicators,
-                            indexFieldTypes, stmtCreateIndex.getGramLength(), stmtCreateIndex.getFullTextConfigName(),
-                            overridesFieldTypes, stmtCreateIndex.isEnforced(), false, MetadataUtil.PENDING_ADD_OP);
+                List<Index.ArrayIndexElement> indexElementList = new ArrayList<>(indexedElementsCount);
+                for (int i = 0; i < indexedElementsCount; i++) {
+                    CreateIndexStatement.IndexedElement indexedElement = indexedElements.get(i);
+                    List<List<String>> projectList =
+                            indexedElement.getProjectList().stream().map(Pair::getFirst).collect(Collectors.toList());
+                    indexElementList.add(new Index.ArrayIndexElement(indexedElement.getUnnestList(), projectList,
+                            indexFieldTypes.get(i), indexedElement.getSourceIndicator()));
+                }
+                indexDetails = new Index.ArrayIndexDetails(indexElementList, overridesFieldTypes);
+            } else {
+                List<List<String>> keyFieldNames = new ArrayList<>(indexedElementsCount);
+                List<IAType> keyFieldTypes = new ArrayList<>(indexedElementsCount);
+                List<Integer> keyFieldSourceIndicators = new ArrayList<>(indexedElementsCount);
+                if (isSecondaryPrimary) {
+                    // BACK-COMPAT: secondary primary index has one source indicator
+                    // which is set to META_RECORD_INDICATOR
+                    keyFieldSourceIndicators.add(Index.META_RECORD_INDICATOR);
+                } else {
+                    for (int i = 0; i < indexedElementsCount; i++) {
+                        CreateIndexStatement.IndexedElement indexedElement = indexedElements.get(i);
+                        keyFieldNames.add(indexedElement.getProjectList().get(0).first);
+                        keyFieldTypes.add(indexFieldTypes.get(i).get(0));
+                        keyFieldSourceIndicators.add(indexedElement.getSourceIndicator());
+                    }
+                }
+                switch (Index.IndexCategory.of(indexType)) {
+                    case VALUE:
+                        indexDetails = new Index.ValueIndexDetails(keyFieldNames, keyFieldSourceIndicators,
+                                keyFieldTypes, overridesFieldTypes);
+                        break;
+                    case TEXT:
+                        indexDetails = new Index.TextIndexDetails(keyFieldNames, keyFieldSourceIndicators,
+                                keyFieldTypes, overridesFieldTypes, stmtCreateIndex.getGramLength(),
+                                stmtCreateIndex.getFullTextConfigName());
+                        break;
+                    default:
+                        throw new CompilationException(ErrorCode.COMPILATION_ILLEGAL_STATE, sourceLoc,
+                                String.valueOf(indexType));
+                }
+            }
+
+            Index newIndex = new Index(dataverseName, datasetName, indexName, indexType, indexDetails,
+                    stmtCreateIndex.isEnforced(), false, MetadataUtil.PENDING_ADD_OP);
 
             bActiveTxn = false; // doCreateIndexImpl() takes over the current transaction
             doCreateIndexImpl(hcc, metadataProvider, ds, newIndex, jobFlags, sourceLoc);
@@ -1356,9 +1482,9 @@
                     // Add an entry for the files index
                     filesIndex = new Index(index.getDataverseName(), index.getDatasetName(),
                             IndexingConstants.getFilesIndexName(index.getDatasetName()), IndexType.BTREE,
-                            ExternalIndexingOperations.FILE_INDEX_FIELD_NAMES, null,
-                            ExternalIndexingOperations.FILE_INDEX_FIELD_TYPES, false, false, false,
-                            MetadataUtil.PENDING_ADD_OP);
+                            new Index.ValueIndexDetails(ExternalIndexingOperations.FILE_INDEX_FIELD_NAMES, null,
+                                    ExternalIndexingOperations.FILE_INDEX_FIELD_TYPES, false),
+                            false, false, MetadataUtil.PENDING_ADD_OP);
                     MetadataManager.INSTANCE.addIndex(metadataProvider.getMetadataTxnContext(), filesIndex);
                     // Add files to the external files index
                     for (ExternalFile file : externalFilesSnapshot) {
@@ -1367,10 +1493,6 @@
                     // This is the first index for the external dataset, replicate the files index
                     spec = ExternalIndexingOperations.buildFilesIndexCreateJobSpec(ds, externalFilesSnapshot,
                             metadataProvider);
-                    if (spec == null) {
-                        throw new CompilationException(ErrorCode.COMPILATION_ERROR, sourceLoc,
-                                "Failed to create job spec for replicating Files Index For external dataset");
-                    }
                     filesIndexReplicated = true;
                     runJob(hcc, spec, jobFlags);
                 }
@@ -1378,16 +1500,54 @@
 
             // check whether there exists another enforced index on the same field
             if (index.isEnforced()) {
+                List<List<String>> indexKeyFieldNames;
+                List<IAType> indexKeyFieldTypes;
+                switch (Index.IndexCategory.of(index.getIndexType())) {
+                    case VALUE:
+                        Index.ValueIndexDetails valueIndexDetails = (Index.ValueIndexDetails) index.getIndexDetails();
+                        indexKeyFieldNames = valueIndexDetails.getKeyFieldNames();
+                        indexKeyFieldTypes = valueIndexDetails.getKeyFieldTypes();
+                        break;
+                    case TEXT:
+                        Index.TextIndexDetails textIndexDetails = (Index.TextIndexDetails) index.getIndexDetails();
+                        indexKeyFieldNames = textIndexDetails.getKeyFieldNames();
+                        indexKeyFieldTypes = textIndexDetails.getKeyFieldTypes();
+                        break;
+                    default:
+                        throw new CompilationException(ErrorCode.COMPILATION_ILLEGAL_STATE, sourceLoc, "");
+                }
                 List<Index> indexes = MetadataManager.INSTANCE.getDatasetIndexes(
                         metadataProvider.getMetadataTxnContext(), index.getDataverseName(), index.getDatasetName());
                 for (Index existingIndex : indexes) {
-                    if (existingIndex.getKeyFieldNames().equals(index.getKeyFieldNames())
-                            && !existingIndex.getKeyFieldTypes().equals(index.getKeyFieldTypes())
-                            && existingIndex.isEnforced()) {
-                        throw new CompilationException(ErrorCode.COMPILATION_ERROR, sourceLoc, "Cannot create index "
-                                + index.getIndexName() + " , enforced index " + existingIndex.getIndexName()
-                                + " on field \"" + StringUtils.join(index.getKeyFieldNames(), ',')
-                                + "\" is already defined with type \"" + existingIndex.getKeyFieldTypes() + "\"");
+                    if (!existingIndex.isEnforced()) {
+                        continue;
+                    }
+                    List<List<String>> existingIndexKeyFieldNames;
+                    List<IAType> existingIndexKeyFieldTypes;
+                    switch (Index.IndexCategory.of(existingIndex.getIndexType())) {
+                        case VALUE:
+                            Index.ValueIndexDetails valueIndexDetails =
+                                    (Index.ValueIndexDetails) existingIndex.getIndexDetails();
+                            existingIndexKeyFieldNames = valueIndexDetails.getKeyFieldNames();
+                            existingIndexKeyFieldTypes = valueIndexDetails.getKeyFieldTypes();
+                            break;
+                        case TEXT:
+                            Index.TextIndexDetails textIndexDetails =
+                                    (Index.TextIndexDetails) existingIndex.getIndexDetails();
+                            existingIndexKeyFieldNames = textIndexDetails.getKeyFieldNames();
+                            existingIndexKeyFieldTypes = textIndexDetails.getKeyFieldTypes();
+                            break;
+                        default:
+                            // ARRAY indexed cannot be enforced yet.
+                            throw new CompilationException(ErrorCode.COMPILATION_ILLEGAL_STATE, sourceLoc, "");
+                    }
+                    if (existingIndexKeyFieldNames.equals(indexKeyFieldNames)
+                            && !existingIndexKeyFieldTypes.equals(indexKeyFieldTypes)) {
+                        throw new CompilationException(ErrorCode.COMPILATION_ERROR, sourceLoc,
+                                "Cannot create index " + index.getIndexName() + " , enforced index "
+                                        + existingIndex.getIndexName() + " on field \""
+                                        + StringUtils.join(indexKeyFieldNames, ',')
+                                        + "\" is already defined with type \"" + existingIndexKeyFieldTypes + "\"");
                     }
                 }
             }
@@ -1552,13 +1712,15 @@
         if (datasetType == DatasetType.EXTERNAL && isSecondaryPrimaryIndex) {
             throw new CompilationException(ErrorCode.CANNOT_CREATE_SEC_PRIMARY_IDX_ON_EXT_DATASET);
         }
+        if (indexType != IndexType.BTREE && isSecondaryPrimaryIndex) {
+            throw new CompilationException(ErrorCode.COMPILATION_INCOMPATIBLE_INDEX_TYPE, sourceLoc,
+                    String.valueOf(indexType));
+        }
     }
 
-    protected void validateIndexKeyFields(CreateIndexStatement stmtCreateIndex, List<Integer> keySourceIndicators,
-            ARecordType aRecordType, ARecordType metaRecordType, List<List<String>> indexFields,
-            List<IAType> indexFieldTypes) throws AlgebricksException {
-        ValidateUtil.validateKeyFields(aRecordType, metaRecordType, indexFields, keySourceIndicators, indexFieldTypes,
-                stmtCreateIndex.getIndexType(), stmtCreateIndex.getSourceLocation());
+    protected void validateIndexFieldType(IndexType indexType, IAType fieldType, List<String> displayFieldName,
+            SourceLocation sourceLoc) throws AlgebricksException {
+        ValidateUtil.validateIndexFieldType(indexType, fieldType, displayFieldName, sourceLoc);
     }
 
     protected void handleCreateTypeStatement(MetadataProvider metadataProvider, Statement stmt) throws Exception {
@@ -1950,10 +2112,8 @@
                 // #. mark PendingDropOp on the existing index
                 MetadataManager.INSTANCE.dropIndex(mdTxnCtx, dataverseName, datasetName, indexName);
                 MetadataManager.INSTANCE.addIndex(mdTxnCtx,
-                        new Index(dataverseName, datasetName, indexName, index.getIndexType(), index.getKeyFieldNames(),
-                                index.getKeyFieldSourceIndicators(), index.getKeyFieldTypes(),
-                                index.isOverridingKeyFieldTypes(), index.isEnforced(), index.isPrimaryIndex(),
-                                MetadataUtil.PENDING_DROP_OP));
+                        new Index(dataverseName, datasetName, indexName, index.getIndexType(), index.getIndexDetails(),
+                                index.isEnforced(), index.isPrimaryIndex(), MetadataUtil.PENDING_DROP_OP));
 
                 // #. commit the existing transaction before calling runJob.
                 MetadataManager.INSTANCE.commitTransaction(mdTxnCtx);
@@ -2003,10 +2163,9 @@
                                     externalIndex.getIndexName());
                             MetadataManager.INSTANCE.addIndex(mdTxnCtx,
                                     new Index(dataverseName, datasetName, externalIndex.getIndexName(),
-                                            externalIndex.getIndexType(), externalIndex.getKeyFieldNames(),
-                                            externalIndex.getKeyFieldSourceIndicators(), index.getKeyFieldTypes(),
-                                            index.isOverridingKeyFieldTypes(), index.isEnforced(),
-                                            externalIndex.isPrimaryIndex(), MetadataUtil.PENDING_DROP_OP));
+                                            externalIndex.getIndexType(), externalIndex.getIndexDetails(),
+                                            externalIndex.isEnforced(), externalIndex.isPrimaryIndex(),
+                                            MetadataUtil.PENDING_DROP_OP));
                         }
                     }
                 }
@@ -2014,10 +2173,8 @@
                 // #. mark PendingDropOp on the existing index
                 MetadataManager.INSTANCE.dropIndex(mdTxnCtx, dataverseName, datasetName, indexName);
                 MetadataManager.INSTANCE.addIndex(mdTxnCtx,
-                        new Index(dataverseName, datasetName, indexName, index.getIndexType(), index.getKeyFieldNames(),
-                                index.getKeyFieldSourceIndicators(), index.getKeyFieldTypes(),
-                                index.isOverridingKeyFieldTypes(), index.isEnforced(), index.isPrimaryIndex(),
-                                MetadataUtil.PENDING_DROP_OP));
+                        new Index(dataverseName, datasetName, indexName, index.getIndexType(), index.getIndexDetails(),
+                                index.isEnforced(), index.isPrimaryIndex(), MetadataUtil.PENDING_DROP_OP));
 
                 // #. commit the existing transaction before calling runJob.
                 MetadataManager.INSTANCE.commitTransaction(mdTxnCtx);
diff --git a/asterixdb/asterix-app/src/test/java/org/apache/asterix/app/bootstrap/TestNodeController.java b/asterixdb/asterix-app/src/test/java/org/apache/asterix/app/bootstrap/TestNodeController.java
index 1040781..c668fb6 100644
--- a/asterixdb/asterix-app/src/test/java/org/apache/asterix/app/bootstrap/TestNodeController.java
+++ b/asterixdb/asterix-app/src/test/java/org/apache/asterix/app/bootstrap/TestNodeController.java
@@ -201,7 +201,8 @@
                     storageComponentProvider.getStorageManager(), secondaryIndexInfo.fileSplitProvider);
             IIndexDataflowHelperFactory primaryIndexHelperFactory = new IndexDataflowHelperFactory(
                     storageComponentProvider.getStorageManager(), primaryIndexInfo.getFileSplitProvider());
-            int[] fieldPermutation = new int[secondaryIndex.getKeyFieldNames().size()];
+            Index.ValueIndexDetails secondaryIndexDetails = (Index.ValueIndexDetails) secondaryIndex.getIndexDetails();
+            int[] fieldPermutation = new int[secondaryIndexDetails.getKeyFieldNames().size()];
             for (int i = 0; i < fieldPermutation.length; i++) {
                 fieldPermutation[i] = i;
             }
@@ -257,8 +258,10 @@
             // for the index, we will have to create an assign operator that extract the sk
             // then the secondary LSMInsertDeleteOperatorNodePushable
             if (secondaryIndex != null) {
-                List<List<String>> skNames = secondaryIndex.getKeyFieldNames();
-                List<Integer> indicators = secondaryIndex.getKeyFieldSourceIndicators();
+                Index.ValueIndexDetails secondaryIndexDetails =
+                        (Index.ValueIndexDetails) secondaryIndex.getIndexDetails();
+                List<List<String>> skNames = secondaryIndexDetails.getKeyFieldNames();
+                List<Integer> indicators = secondaryIndexDetails.getKeyFieldSourceIndicators();
                 IScalarEvaluatorFactory[] secondaryFieldAccessEvalFactories =
                         new IScalarEvaluatorFactory[skNames.size()];
                 for (int i = 0; i < skNames.size(); i++) {
@@ -266,14 +269,15 @@
                             ? indicators.get(i).intValue() == Index.RECORD_INDICATOR ? recordType : metaType
                             : recordType;
                     int pos = skNames.get(i).size() > 1 ? -1 : sourceType.getFieldIndex(skNames.get(i).get(0));
-                    secondaryFieldAccessEvalFactories[i] =
-                            mdProvider.getDataFormat().getFieldAccessEvaluatorFactory(mdProvider.getFunctionManager(),
-                                    sourceType, secondaryIndex.getKeyFieldNames().get(i), pos, null);
+                    secondaryFieldAccessEvalFactories[i] = mdProvider.getDataFormat().getFieldAccessEvaluatorFactory(
+                            mdProvider.getFunctionManager(), sourceType, skNames.get(i), pos, null);
                 }
                 // outColumns are computed inside the assign runtime
                 int[] outColumns = new int[skNames.size()];
                 // projection list include old and new (primary and secondary keys)
-                int[] projectionList = new int[skNames.size() + primaryIndexInfo.index.getKeyFieldNames().size()];
+                Index.ValueIndexDetails primaryIndexDetails =
+                        (Index.ValueIndexDetails) primaryIndexInfo.index.getIndexDetails();
+                int[] projectionList = new int[skNames.size() + primaryIndexDetails.getKeyFieldNames().size()];
                 for (int i = 0; i < secondaryFieldAccessEvalFactories.length; i++) {
                     outColumns[i] = primaryIndexInfo.rDesc.getFieldCount() + i;
                 }
@@ -281,7 +285,7 @@
                 for (int i = 0; i < secondaryFieldAccessEvalFactories.length; i++) {
                     projectionList[projCount++] = primaryIndexInfo.rDesc.getFieldCount() + i;
                 }
-                for (int i = 0; i < primaryIndexInfo.index.getKeyFieldNames().size(); i++) {
+                for (int i = 0; i < primaryIndexDetails.getKeyFieldNames().size(); i++) {
                     projectionList[projCount++] = i;
                 }
                 IPushRuntime assignOp =
@@ -354,8 +358,10 @@
             // for the index, we will have to create an assign operator that extract the sk
             // then the secondary LSMInsertDeleteOperatorNodePushable
             if (secondaryIndex != null) {
-                List<List<String>> skNames = secondaryIndex.getKeyFieldNames();
-                List<Integer> indicators = secondaryIndex.getKeyFieldSourceIndicators();
+                Index.ValueIndexDetails secondaryIndexDetails =
+                        (Index.ValueIndexDetails) secondaryIndex.getIndexDetails();
+                List<List<String>> skNames = secondaryIndexDetails.getKeyFieldNames();
+                List<Integer> indicators = secondaryIndexDetails.getKeyFieldSourceIndicators();
                 IScalarEvaluatorFactory[] secondaryFieldAccessEvalFactories =
                         new IScalarEvaluatorFactory[skNames.size()];
                 for (int i = 0; i < skNames.size(); i++) {
@@ -363,14 +369,15 @@
                             ? indicators.get(i).intValue() == Index.RECORD_INDICATOR ? recordType : metaType
                             : recordType;
                     int pos = skNames.get(i).size() > 1 ? -1 : sourceType.getFieldIndex(skNames.get(i).get(0));
-                    secondaryFieldAccessEvalFactories[i] =
-                            mdProvider.getDataFormat().getFieldAccessEvaluatorFactory(mdProvider.getFunctionManager(),
-                                    sourceType, secondaryIndex.getKeyFieldNames().get(i), pos, null);
+                    secondaryFieldAccessEvalFactories[i] = mdProvider.getDataFormat().getFieldAccessEvaluatorFactory(
+                            mdProvider.getFunctionManager(), sourceType, skNames.get(i), pos, null);
                 }
                 // outColumns are computed inside the assign runtime
                 int[] outColumns = new int[skNames.size()];
                 // projection list include old and new (primary and secondary keys)
-                int[] projectionList = new int[skNames.size() + primaryIndexInfo.index.getKeyFieldNames().size()];
+                Index.ValueIndexDetails primaryIndexDetails =
+                        (Index.ValueIndexDetails) primaryIndexInfo.index.getIndexDetails();
+                int[] projectionList = new int[skNames.size() + primaryIndexDetails.getKeyFieldNames().size()];
                 for (int i = 0; i < secondaryFieldAccessEvalFactories.length; i++) {
                     outColumns[i] = primaryIndexInfo.rDesc.getFieldCount() + i;
                 }
@@ -378,7 +385,7 @@
                 for (int i = 0; i < secondaryFieldAccessEvalFactories.length; i++) {
                     projectionList[projCount++] = primaryIndexInfo.rDesc.getFieldCount() + i;
                 }
-                for (int i = 0; i < primaryIndexInfo.index.getKeyFieldNames().size(); i++) {
+                for (int i = 0; i < primaryIndexDetails.getKeyFieldNames().size(); i++) {
                     projectionList[projCount++] = i;
                 }
                 IPushRuntime assignOp =
@@ -632,6 +639,7 @@
         public SecondaryIndexInfo(PrimaryIndexInfo primaryIndexInfo, Index secondaryIndex) {
             this.primaryIndexInfo = primaryIndexInfo;
             this.secondaryIndex = secondaryIndex;
+            Index.ValueIndexDetails secondaryIndexDetails = (Index.ValueIndexDetails) secondaryIndex.getIndexDetails();
             List<String> nodes = Collections.singletonList(ExecutionTestUtil.integrationUtil.ncs[0].getId());
             CcApplicationContext appCtx =
                     (CcApplicationContext) ExecutionTestUtil.integrationUtil.cc.getApplicationContext();
@@ -639,11 +647,11 @@
                     primaryIndexInfo.dataset, secondaryIndex.getIndexName(), nodes);
             fileSplitProvider = new ConstantFileSplitProvider(splits);
             secondaryIndexTypeTraits = createSecondaryIndexTypeTraits(primaryIndexInfo.recordType,
-                    primaryIndexInfo.metaType, primaryIndexInfo.primaryKeyTypes,
-                    secondaryIndex.getKeyFieldTypes().toArray(new IAType[secondaryIndex.getKeyFieldTypes().size()]));
+                    primaryIndexInfo.metaType, primaryIndexInfo.primaryKeyTypes, secondaryIndexDetails
+                            .getKeyFieldTypes().toArray(new IAType[secondaryIndexDetails.getKeyFieldTypes().size()]));
             secondaryIndexSerdes = createSecondaryIndexSerdes(primaryIndexInfo.recordType, primaryIndexInfo.metaType,
-                    primaryIndexInfo.primaryKeyTypes,
-                    secondaryIndex.getKeyFieldTypes().toArray(new IAType[secondaryIndex.getKeyFieldTypes().size()]));
+                    primaryIndexInfo.primaryKeyTypes, secondaryIndexDetails.getKeyFieldTypes()
+                            .toArray(new IAType[secondaryIndexDetails.getKeyFieldTypes().size()]));
             rDesc = new RecordDescriptor(secondaryIndexSerdes, secondaryIndexTypeTraits);
             insertFieldsPermutations = new int[secondaryIndexTypeTraits.length];
             for (int i = 0; i < insertFieldsPermutations.length; i++) {
@@ -651,7 +659,7 @@
             }
             primaryKeyIndexes = new int[primaryIndexInfo.primaryKeyIndexes.length];
             for (int i = 0; i < primaryKeyIndexes.length; i++) {
-                primaryKeyIndexes[i] = i + secondaryIndex.getKeyFieldNames().size();
+                primaryKeyIndexes[i] = i + secondaryIndexDetails.getKeyFieldNames().size();
             }
         }
 
@@ -711,7 +719,8 @@
                 keyFieldNames.add(Arrays.asList(fieldNames[primaryKeyIndexes[i]]));
             }
             index = new Index(dataset.getDataverseName(), dataset.getDatasetName(), dataset.getDatasetName(),
-                    IndexType.BTREE, keyFieldNames, primaryKeyIndicators, keyFieldTypes, false, false, true,
+                    IndexType.BTREE,
+                    new Index.ValueIndexDetails(keyFieldNames, primaryKeyIndicators, keyFieldTypes, false), false, true,
                     MetadataUtil.PENDING_NO_OP);
             List<String> nodes = Collections.singletonList(ExecutionTestUtil.integrationUtil.ncs[0].getId());
             CcApplicationContext appCtx =
diff --git a/asterixdb/asterix-app/src/test/java/org/apache/asterix/test/dataflow/TestDataset.java b/asterixdb/asterix-app/src/test/java/org/apache/asterix/test/dataflow/TestDataset.java
index f28e9bb..f87757c 100644
--- a/asterixdb/asterix-app/src/test/java/org/apache/asterix/test/dataflow/TestDataset.java
+++ b/asterixdb/asterix-app/src/test/java/org/apache/asterix/test/dataflow/TestDataset.java
@@ -55,14 +55,14 @@
     }
 
     @Override
-    public IPushRuntimeFactory getCommitRuntimeFactory(MetadataProvider metadataProvider,
-            int[] primaryKeyFieldPermutation, boolean isSink) throws AlgebricksException {
+    public IPushRuntimeFactory getCommitRuntimeFactory(MetadataProvider metadataProvider, int[] keyFieldPermutation,
+            boolean isSink) throws AlgebricksException {
         return new IPushRuntimeFactory() {
             @Override
             public IPushRuntime[] createPushRuntime(IHyracksTaskContext ctx) throws HyracksDataException {
-                return new IPushRuntime[] { new CommitRuntime(ctx, new TxnId(ctx.getJobletContext().getJobId().getId()),
-                        getDatasetId(), primaryKeyFieldPermutation, true,
-                        ctx.getTaskAttemptId().getTaskId().getPartition(), true) };
+                return new IPushRuntime[] {
+                        new CommitRuntime(ctx, new TxnId(ctx.getJobletContext().getJobId().getId()), getDatasetId(),
+                                keyFieldPermutation, true, ctx.getTaskAttemptId().getTaskId().getPartition(), true) };
             }
         };
     }
diff --git a/asterixdb/asterix-app/src/test/java/org/apache/asterix/test/dataflow/TestLsmBTreeResourceFactoryProvider.java b/asterixdb/asterix-app/src/test/java/org/apache/asterix/test/dataflow/TestLsmBTreeResourceFactoryProvider.java
index 532337d..9818538 100644
--- a/asterixdb/asterix-app/src/test/java/org/apache/asterix/test/dataflow/TestLsmBTreeResourceFactoryProvider.java
+++ b/asterixdb/asterix-app/src/test/java/org/apache/asterix/test/dataflow/TestLsmBTreeResourceFactoryProvider.java
@@ -98,20 +98,21 @@
                 && index.getIndexName().equals(IndexingConstants.getFilesIndexName(dataset.getDatasetName()))) {
             return FilesIndexDescription.EXTERNAL_FILE_INDEX_TYPE_TRAITS;
         }
+        Index.ValueIndexDetails indexDetails = (Index.ValueIndexDetails) index.getIndexDetails();
         int numPrimaryKeys = dataset.getPrimaryKeys().size();
-        int numSecondaryKeys = index.getKeyFieldNames().size();
+        int numSecondaryKeys = indexDetails.getKeyFieldNames().size();
         ITypeTraitProvider typeTraitProvider = metadataProvider.getStorageComponentProvider().getTypeTraitProvider();
         ITypeTraits[] secondaryTypeTraits = new ITypeTraits[numSecondaryKeys + numPrimaryKeys];
         for (int i = 0; i < numSecondaryKeys; i++) {
             ARecordType sourceType;
-            List<Integer> keySourceIndicators = index.getKeyFieldSourceIndicators();
+            List<Integer> keySourceIndicators = indexDetails.getKeyFieldSourceIndicators();
             if (keySourceIndicators == null || keySourceIndicators.get(i) == 0) {
                 sourceType = recordType;
             } else {
                 sourceType = metaType;
             }
-            Pair<IAType, Boolean> keyTypePair = Index.getNonNullableOpenFieldType(index.getKeyFieldTypes().get(i),
-                    index.getKeyFieldNames().get(i), sourceType);
+            Pair<IAType, Boolean> keyTypePair = Index.getNonNullableOpenFieldType(
+                    indexDetails.getKeyFieldTypes().get(i), indexDetails.getKeyFieldNames().get(i), sourceType);
             IAType keyType = keyTypePair.first;
             secondaryTypeTraits[i] = typeTraitProvider.getTypeTrait(keyType);
         }
@@ -132,22 +133,23 @@
                 && index.getIndexName().equals(IndexingConstants.getFilesIndexName(dataset.getDatasetName()))) {
             return FilesIndexDescription.FILES_INDEX_COMP_FACTORIES;
         }
+        Index.ValueIndexDetails indexDetails = (Index.ValueIndexDetails) index.getIndexDetails();
         int numPrimaryKeys = dataset.getPrimaryKeys().size();
-        int numSecondaryKeys = index.getKeyFieldNames().size();
+        int numSecondaryKeys = indexDetails.getKeyFieldNames().size();
         IBinaryComparatorFactoryProvider cmpFactoryProvider =
                 metadataProvider.getStorageComponentProvider().getComparatorFactoryProvider();
         IBinaryComparatorFactory[] secondaryCmpFactories =
                 new IBinaryComparatorFactory[numSecondaryKeys + numPrimaryKeys];
         for (int i = 0; i < numSecondaryKeys; i++) {
             ARecordType sourceType;
-            List<Integer> keySourceIndicators = index.getKeyFieldSourceIndicators();
+            List<Integer> keySourceIndicators = indexDetails.getKeyFieldSourceIndicators();
             if (keySourceIndicators == null || keySourceIndicators.get(i) == 0) {
                 sourceType = recordType;
             } else {
                 sourceType = metaType;
             }
-            Pair<IAType, Boolean> keyTypePair = Index.getNonNullableOpenFieldType(index.getKeyFieldTypes().get(i),
-                    index.getKeyFieldNames().get(i), sourceType);
+            Pair<IAType, Boolean> keyTypePair = Index.getNonNullableOpenFieldType(
+                    indexDetails.getKeyFieldTypes().get(i), indexDetails.getKeyFieldNames().get(i), sourceType);
             IAType keyType = keyTypePair.first;
             secondaryCmpFactories[i] = cmpFactoryProvider.getBinaryComparatorFactory(keyType, true);
         }
@@ -166,14 +168,14 @@
             if (index.getIndexName().equals(IndexingConstants.getFilesIndexName(dataset.getDatasetName()))) {
                 return FilesIndexDescription.BLOOM_FILTER_FIELDS;
             } else {
-                return new int[] { index.getKeyFieldNames().size() };
+                return new int[] { ((Index.ValueIndexDetails) index.getIndexDetails()).getKeyFieldNames().size() };
             }
         } else if (index.getIndexType() == IndexType.BTREE || index.getIndexType() == IndexType.RTREE) {
             // secondary btrees and rtrees do not have bloom filters
             return null;
         } else {
             // inverted indexes have bloom filters on deleted-key btrees
-            int numKeys = index.getKeyFieldNames().size();
+            int numKeys = ((Index.ValueIndexDetails) index.getIndexDetails()).getKeyFieldNames().size();
             int[] bloomFilterKeyFields = new int[numKeys];
             for (int i = 0; i < numKeys; i++) {
                 bloomFilterKeyFields[i] = i;
diff --git a/asterixdb/asterix-app/src/test/resources/metadata/testsuite.xml b/asterixdb/asterix-app/src/test/resources/metadata/testsuite.xml
index 8af4c67..a29dc48 100644
--- a/asterixdb/asterix-app/src/test/resources/metadata/testsuite.xml
+++ b/asterixdb/asterix-app/src/test/resources/metadata/testsuite.xml
@@ -524,37 +524,37 @@
     <test-case FilePath="exception">
       <compilation-unit name="issue_384_create_index_error_1">
         <output-dir compare="Text">none</output-dir>
-        <expected-error>ASX1079: Compilation error: The field "[loc]" which is of type point cannot be indexed using the BTree index. (in line 37, at column 1)</expected-error>
+        <expected-error>ASX1079: Compilation error: The field "[loc]" which is of type point cannot be indexed using the BTree index. (in line 37, at column 33)</expected-error>
       </compilation-unit>
     </test-case>
     <test-case FilePath="exception">
       <compilation-unit name="issue_384_create_index_error_2">
         <output-dir compare="Text">none</output-dir>
-        <expected-error>ASX1079: Compilation error: The field "[age]" which is of type integer cannot be indexed using the RTree index. (in line 37, at column 1)</expected-error>
+        <expected-error>ASX1079: Compilation error: The field "[age]" which is of type integer cannot be indexed using the RTree index. (in line 37, at column 33)</expected-error>
       </compilation-unit>
     </test-case>
     <test-case FilePath="exception">
       <compilation-unit name="issue_384_create_index_error_3">
         <output-dir compare="Text">none</output-dir>
-        <expected-error>ASX1079: Compilation error: The field "[loc]" which is of type point cannot be indexed using the Length Partitioned Keyword index. (in line 37, at column 1)</expected-error>
+        <expected-error>ASX1079: Compilation error: The field "[loc]" which is of type point cannot be indexed using the Length Partitioned Keyword index. (in line 37, at column 33)</expected-error>
       </compilation-unit>
     </test-case>
     <test-case FilePath="exception">
       <compilation-unit name="issue_384_create_index_error_4">
         <output-dir compare="Text">none</output-dir>
-        <expected-error>ASX1079: Compilation error: The field "[loc]" which is of type point cannot be indexed using the Length Partitioned Keyword index. (in line 37, at column 1)</expected-error>
+        <expected-error>ASX1079: Compilation error: The field "[loc]" which is of type point cannot be indexed using the Length Partitioned Keyword index. (in line 37, at column 33)</expected-error>
       </compilation-unit>
     </test-case>
     <test-case FilePath="exception">
       <compilation-unit name="issue_384_create_index_error_5">
         <output-dir compare="Text">none</output-dir>
-        <expected-error>ASX1079: Compilation error: The field "[loc]" which is of type point cannot be indexed using the Length Partitioned N-Gram index. (in line 37, at column 1)</expected-error>
+        <expected-error>ASX1079: Compilation error: The field "[loc]" which is of type point cannot be indexed using the Length Partitioned N-Gram index. (in line 37, at column 33)</expected-error>
       </compilation-unit>
     </test-case>
     <test-case FilePath="exception">
       <compilation-unit name="issue_384_create_index_error_6">
         <output-dir compare="Text">none</output-dir>
-        <expected-error>ASX1079: Compilation error: The field "[loc]" which is of type point cannot be indexed using the Length Partitioned N-Gram index. (in line 37, at column 1)</expected-error>
+        <expected-error>ASX1079: Compilation error: The field "[loc]" which is of type point cannot be indexed using the Length Partitioned N-Gram index. (in line 37, at column 33)</expected-error>
       </compilation-unit>
     </test-case>
   </test-group>
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/array-index/join-unnest-queries/use-case-1/query1.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/array-index/join-unnest-queries/use-case-1/query1.sqlpp
new file mode 100644
index 0000000..3960ee6
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/array-index/join-unnest-queries/use-case-1/query1.sqlpp
@@ -0,0 +1,44 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+SET `compiler.arrayindex` "true";
+
+DROP DATAVERSE TestYelp IF EXISTS;
+CREATE DATAVERSE TestYelp;
+USE TestYelp;
+
+CREATE TYPE CheckinType AS {
+    checkin_id: uuid,
+    business_id: string,
+    dates: [string]
+};
+CREATE TYPE CheckinDatesToMarkersType AS {
+    marker: string,
+    datetime: string,
+    date: string,
+    time: string
+};
+
+CREATE DATASET YelpCheckin(CheckinType) PRIMARY KEY checkin_id AUTOGENERATED;
+CREATE DATASET YelpCheckinDateMarkers (CheckinDatesToMarkersType) PRIMARY KEY marker;
+CREATE INDEX IdxYelpCheckinDates ON YelpCheckin (UNNEST dates);
+
+SELECT M.marker
+FROM YelpCheckinDateMarkers M, YelpCheckin C, C.dates D
+WHERE D /*+ indexnl */ = M.datetime
+      AND C.business_id = "--Ni3oJ4VOqfOEu7Sj2Vzg";
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/array-index/join-unnest-queries/use-case-1/query2.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/array-index/join-unnest-queries/use-case-1/query2.sqlpp
new file mode 100644
index 0000000..a231ac7
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/array-index/join-unnest-queries/use-case-1/query2.sqlpp
@@ -0,0 +1,44 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+SET `compiler.arrayindex` "true";
+
+DROP DATAVERSE TestYelp IF EXISTS;
+CREATE DATAVERSE TestYelp;
+USE TestYelp;
+
+CREATE TYPE CheckinType AS {
+    checkin_id: uuid,
+    business_id: string,
+    dates: [string]
+};
+CREATE TYPE CheckinDatesToMarkersType AS {
+    marker: string,
+    datetime: string,
+    date: string,
+    time: string
+};
+
+CREATE DATASET YelpCheckin(CheckinType) PRIMARY KEY checkin_id AUTOGENERATED;
+CREATE DATASET YelpCheckinDateMarkers (CheckinDatesToMarkersType) PRIMARY KEY marker;
+CREATE INDEX IdxYelpCheckinDates ON YelpCheckin (UNNEST dates);
+
+SELECT M.marker
+FROM YelpCheckinDateMarkers M, YelpCheckin C, C.dates D
+WHERE D /*+ indexnl */ = M.datetime
+      AND M.time = "19:49:16";
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/array-index/join-unnest-queries/use-case-1/query3.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/array-index/join-unnest-queries/use-case-1/query3.sqlpp
new file mode 100644
index 0000000..8fe3c15
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/array-index/join-unnest-queries/use-case-1/query3.sqlpp
@@ -0,0 +1,46 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+SET `compiler.arrayindex` "true";
+
+DROP DATAVERSE TestYelp IF EXISTS;
+CREATE DATAVERSE TestYelp;
+USE TestYelp;
+
+CREATE TYPE CheckinType AS {
+    checkin_id: uuid,
+    business_id: string,
+    dates: [string]
+};
+CREATE TYPE CheckinDatesToMarkersType AS {
+    marker: string,
+    datetime: string,
+    date: string,
+    time: string
+};
+
+CREATE DATASET YelpCheckin(CheckinType) PRIMARY KEY checkin_id AUTOGENERATED;
+CREATE DATASET YelpCheckinDateMarkers (CheckinDatesToMarkersType) PRIMARY KEY marker;
+CREATE INDEX IdxYelpCheckinDates ON YelpCheckin (UNNEST dates);
+
+SELECT COUNT(*)
+FROM YelpCheckinDateMarkers M
+INNER JOIN (
+    SELECT VALUE D
+    FROM YelpCheckin C, C.dates D
+) AS CD ON CD /*+ indexnl */ = M.datetime;
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/array-index/join-unnest-queries/use-case-2/query1.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/array-index/join-unnest-queries/use-case-2/query1.sqlpp
new file mode 100644
index 0000000..c10c2a9
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/array-index/join-unnest-queries/use-case-2/query1.sqlpp
@@ -0,0 +1,47 @@
+/*
+ * 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 TestYelp IF EXISTS;
+CREATE DATAVERSE TestYelp;
+USE TestYelp;
+
+CREATE TYPE CheckinType AS {
+    checkin_id: uuid,
+    business_id: string,
+    checkin_times: {
+        dates: [string],
+        times: [string]
+    }
+};
+CREATE TYPE CheckinDatesToMarkersType AS {
+    marker: string,
+    datetime: string,
+    date: string,
+    time: string
+};
+
+CREATE DATASET YelpCheckin(CheckinType) PRIMARY KEY checkin_id AUTOGENERATED;
+CREATE DATASET YelpCheckinDateMarkers (CheckinDatesToMarkersType) PRIMARY KEY marker;
+CREATE INDEX IdxYelpCheckinDates ON YelpCheckin (UNNEST checkin_times.dates);
+
+SELECT M.marker
+FROM YelpCheckinDateMarkers M, YelpCheckin C, C.checkin_times.dates D
+WHERE D /*+ indexnl */ = M.date
+      AND C.business_id = "--Ni3oJ4VOqfOEu7Sj2Vzg";
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/array-index/join-unnest-queries/use-case-2/query2.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/array-index/join-unnest-queries/use-case-2/query2.sqlpp
new file mode 100644
index 0000000..8e5cf1e
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/array-index/join-unnest-queries/use-case-2/query2.sqlpp
@@ -0,0 +1,47 @@
+/*
+ * 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 TestYelp IF EXISTS;
+CREATE DATAVERSE TestYelp;
+USE TestYelp;
+
+CREATE TYPE CheckinType AS {
+    checkin_id: uuid,
+    business_id: string,
+    checkin_times: {
+        dates: [string],
+        times: [string]
+    }
+};
+CREATE TYPE CheckinDatesToMarkersType AS {
+    marker: string,
+    datetime: string,
+    date: string,
+    time: string
+};
+
+CREATE DATASET YelpCheckin(CheckinType) PRIMARY KEY checkin_id AUTOGENERATED;
+CREATE DATASET YelpCheckinDateMarkers (CheckinDatesToMarkersType) PRIMARY KEY marker;
+CREATE INDEX IdxYelpCheckinDates ON YelpCheckin (UNNEST checkin_times.dates);
+
+SELECT M.marker
+FROM YelpCheckinDateMarkers M, YelpCheckin C, C.checkin_times.dates D
+WHERE D /*+ indexnl */ = M.date
+      AND M.time = "19:49:16";
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/array-index/join-unnest-queries/use-case-2/query3.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/array-index/join-unnest-queries/use-case-2/query3.sqlpp
new file mode 100644
index 0000000..b8e9b49
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/array-index/join-unnest-queries/use-case-2/query3.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 TestYelp IF EXISTS;
+CREATE DATAVERSE TestYelp;
+USE TestYelp;
+
+CREATE TYPE CheckinType AS {
+    checkin_id: uuid,
+    business_id: string,
+    checkin_times: {
+        dates: [string],
+        times: [string]
+    }
+};
+CREATE TYPE CheckinDatesToMarkersType AS {
+    marker: string,
+    datetime: string,
+    date: string,
+    time: string
+};
+
+CREATE DATASET YelpCheckin(CheckinType) PRIMARY KEY checkin_id AUTOGENERATED;
+CREATE DATASET YelpCheckinDateMarkers (CheckinDatesToMarkersType) PRIMARY KEY marker;
+CREATE INDEX IdxYelpCheckinDates ON YelpCheckin (UNNEST checkin_times.dates);
+
+SELECT COUNT(*)
+FROM YelpCheckinDateMarkers M
+INNER JOIN (
+    SELECT VALUE D
+    FROM YelpCheckin C, C.checkin_times.dates D
+) AS CD ON CD /*+ indexnl */ = M.date;
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/array-index/join-unnest-queries/use-case-3/query1.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/array-index/join-unnest-queries/use-case-3/query1.sqlpp
new file mode 100644
index 0000000..91f6d8a
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/array-index/join-unnest-queries/use-case-3/query1.sqlpp
@@ -0,0 +1,47 @@
+/*
+ * 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 TestYelp IF EXISTS;
+CREATE DATAVERSE TestYelp;
+USE TestYelp;
+
+CREATE TYPE CheckinType AS {
+    checkin_id: uuid,
+    business_id: string,
+    checkin_times: [{
+        date: string,
+        time: string
+    }]
+};
+CREATE TYPE CheckinDatesToMarkersType AS {
+    marker: string,
+    datetime: string,
+    date: string,
+    time: string
+};
+
+CREATE DATASET YelpCheckin(CheckinType) PRIMARY KEY checkin_id AUTOGENERATED;
+CREATE DATASET YelpCheckinDateMarkers (CheckinDatesToMarkersType) PRIMARY KEY marker;
+CREATE INDEX IdxYelpCheckinDates ON YelpCheckin (UNNEST checkin_times SELECT date);
+
+SELECT M.marker
+FROM YelpCheckinDateMarkers M, YelpCheckin C, C.checkin_times D
+WHERE D.date /*+ indexnl */ = M.date
+      AND C.business_id = "--Ni3oJ4VOqfOEu7Sj2Vzg";
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/array-index/join-unnest-queries/use-case-3/query2.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/array-index/join-unnest-queries/use-case-3/query2.sqlpp
new file mode 100644
index 0000000..d4742ce
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/array-index/join-unnest-queries/use-case-3/query2.sqlpp
@@ -0,0 +1,47 @@
+/*
+ * 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 TestYelp IF EXISTS;
+CREATE DATAVERSE TestYelp;
+USE TestYelp;
+
+CREATE TYPE CheckinType AS {
+    checkin_id: uuid,
+    business_id: string,
+    checkin_times: [{
+        date: string,
+        time: string
+    }]
+};
+CREATE TYPE CheckinDatesToMarkersType AS {
+    marker: string,
+    datetime: string,
+    date: string,
+    time: string
+};
+
+CREATE DATASET YelpCheckin(CheckinType) PRIMARY KEY checkin_id AUTOGENERATED;
+CREATE DATASET YelpCheckinDateMarkers (CheckinDatesToMarkersType) PRIMARY KEY marker;
+CREATE INDEX IdxYelpCheckinDates ON YelpCheckin (UNNEST checkin_times SELECT date);
+
+SELECT M.marker
+FROM YelpCheckinDateMarkers M, YelpCheckin C, C.checkin_times D
+WHERE D.date /*+ indexnl */ = M.date
+      AND D.time = "19:49:16";
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/array-index/join-unnest-queries/use-case-3/query3.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/array-index/join-unnest-queries/use-case-3/query3.sqlpp
new file mode 100644
index 0000000..e3afa1e
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/array-index/join-unnest-queries/use-case-3/query3.sqlpp
@@ -0,0 +1,47 @@
+/*
+ * 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 TestYelp IF EXISTS;
+CREATE DATAVERSE TestYelp;
+USE TestYelp;
+
+CREATE TYPE CheckinType AS {
+    checkin_id: uuid,
+    business_id: string,
+    checkin_times: [{
+        date: string,
+        time: string
+    }]
+};
+CREATE TYPE CheckinDatesToMarkersType AS {
+    marker: string,
+    datetime: string,
+    date: string,
+    time: string
+};
+
+CREATE DATASET YelpCheckin(CheckinType) PRIMARY KEY checkin_id AUTOGENERATED;
+CREATE DATASET YelpCheckinDateMarkers (CheckinDatesToMarkersType) PRIMARY KEY marker;
+CREATE INDEX IdxYelpCheckinDates ON YelpCheckin (UNNEST checkin_times SELECT date);
+
+SELECT M.marker
+FROM YelpCheckinDateMarkers M, YelpCheckin C, C.checkin_times D
+WHERE D.date /*+ indexnl */ = M.date
+      AND M.time = "19:49:16";
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/array-index/join-unnest-queries/use-case-3/query4.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/array-index/join-unnest-queries/use-case-3/query4.sqlpp
new file mode 100644
index 0000000..f5d3c52
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/array-index/join-unnest-queries/use-case-3/query4.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 TestYelp IF EXISTS;
+CREATE DATAVERSE TestYelp;
+USE TestYelp;
+
+CREATE TYPE CheckinType AS {
+    checkin_id: uuid,
+    business_id: string,
+    checkin_times: [{
+        date: string,
+        time: string
+    }]
+};
+CREATE TYPE CheckinDatesToMarkersType AS {
+    marker: string,
+    datetime: string,
+    date: string,
+    time: string
+};
+
+CREATE DATASET YelpCheckin(CheckinType) PRIMARY KEY checkin_id AUTOGENERATED;
+CREATE DATASET YelpCheckinDateMarkers (CheckinDatesToMarkersType) PRIMARY KEY marker;
+CREATE INDEX IdxYelpCheckinDates ON YelpCheckin (UNNEST checkin_times SELECT date);
+
+SELECT COUNT(*)
+FROM YelpCheckinDateMarkers M
+INNER JOIN (
+    SELECT VALUE D.date
+    FROM YelpCheckin C, C.checkin_times D
+) AS CD ON CD /*+ indexnl */ = M.date;
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/array-index/join-unnest-queries/use-case-4/query1.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/array-index/join-unnest-queries/use-case-4/query1.sqlpp
new file mode 100644
index 0000000..1afd508
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/array-index/join-unnest-queries/use-case-4/query1.sqlpp
@@ -0,0 +1,48 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+SET `compiler.arrayindex` "true";
+
+DROP DATAVERSE TestYelp IF EXISTS;
+CREATE DATAVERSE TestYelp;
+USE TestYelp;
+
+CREATE TYPE CheckinType AS {
+    checkin_id: uuid,
+    business_id: string,
+    checkin_times: [{
+        checkin_times_id: int,
+        dates: [string],
+        times: [string]
+    }]
+};
+CREATE TYPE CheckinDatesToMarkersType AS {
+    marker: string,
+    datetime: string,
+    date: string,
+    time: string
+};
+
+CREATE DATASET YelpCheckin(CheckinType) PRIMARY KEY checkin_id AUTOGENERATED;
+CREATE DATASET YelpCheckinDateMarkers (CheckinDatesToMarkersType) PRIMARY KEY marker;
+CREATE INDEX IdxYelpCheckinDates ON YelpCheckin (UNNEST checkin_times UNNEST dates);
+
+SELECT M.marker
+FROM YelpCheckinDateMarkers M, YelpCheckin C, C.checkin_times CT, CT.dates D
+WHERE D /*+ indexnl */ = M.date
+      AND C.business_id = "--Ni3oJ4VOqfOEu7Sj2Vzg";
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/array-index/join-unnest-queries/use-case-4/query2.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/array-index/join-unnest-queries/use-case-4/query2.sqlpp
new file mode 100644
index 0000000..b911a20
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/array-index/join-unnest-queries/use-case-4/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 TestYelp IF EXISTS;
+CREATE DATAVERSE TestYelp;
+USE TestYelp;
+
+CREATE TYPE CheckinType AS {
+    checkin_id: uuid,
+    business_id: string,
+    checkin_times: [{
+        checkin_times_id: int,
+        dates: [string],
+        times: [string]
+    }]
+};
+CREATE TYPE CheckinDatesToMarkersType AS {
+    marker: string,
+    datetime: string,
+    date: string,
+    time: string
+};
+
+CREATE DATASET YelpCheckin(CheckinType) PRIMARY KEY checkin_id AUTOGENERATED;
+CREATE DATASET YelpCheckinDateMarkers (CheckinDatesToMarkersType) PRIMARY KEY marker;
+CREATE INDEX IdxYelpCheckinDates ON YelpCheckin (UNNEST checkin_times UNNEST dates);
+
+SELECT M.marker
+FROM YelpCheckinDateMarkers M, YelpCheckin C, C.checkin_times CT, CT.dates D
+WHERE D /*+ indexnl */ = M.date
+      AND CT.checkin_times_id = 1
+      AND C.business_id = "--Ni3oJ4VOqfOEu7Sj2Vzg";
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/array-index/join-unnest-queries/use-case-4/query3.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/array-index/join-unnest-queries/use-case-4/query3.sqlpp
new file mode 100644
index 0000000..5a23cb4
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/array-index/join-unnest-queries/use-case-4/query3.sqlpp
@@ -0,0 +1,48 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+SET `compiler.arrayindex` "true";
+
+DROP DATAVERSE TestYelp IF EXISTS;
+CREATE DATAVERSE TestYelp;
+USE TestYelp;
+
+CREATE TYPE CheckinType AS {
+    checkin_id: uuid,
+    business_id: string,
+    checkin_times: [{
+        checkin_times_id: int,
+        dates: [string],
+        times: [string]
+    }]
+};
+CREATE TYPE CheckinDatesToMarkersType AS {
+    marker: string,
+    datetime: string,
+    date: string,
+    time: string
+};
+
+CREATE DATASET YelpCheckin(CheckinType) PRIMARY KEY checkin_id AUTOGENERATED;
+CREATE DATASET YelpCheckinDateMarkers (CheckinDatesToMarkersType) PRIMARY KEY marker;
+CREATE INDEX IdxYelpCheckinDates ON YelpCheckin (UNNEST checkin_times UNNEST dates);
+
+SELECT M.marker
+FROM YelpCheckinDateMarkers M, YelpCheckin C, C.checkin_times CT, CT.dates D
+WHERE D /*+ indexnl */ = M.date
+      AND M.time = "19:49:16";
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/array-index/join-unnest-queries/use-case-4/query4.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/array-index/join-unnest-queries/use-case-4/query4.sqlpp
new file mode 100644
index 0000000..c142d1a
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/array-index/join-unnest-queries/use-case-4/query4.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 TestYelp IF EXISTS;
+CREATE DATAVERSE TestYelp;
+USE TestYelp;
+
+CREATE TYPE CheckinType AS {
+    checkin_id: uuid,
+    business_id: string,
+    checkin_times: [{
+        checkin_times_id: int,
+        dates: [string],
+        times: [string]
+    }]
+};
+CREATE TYPE CheckinDatesToMarkersType AS {
+    marker: string,
+    datetime: string,
+    date: string,
+    time: string
+};
+
+CREATE DATASET YelpCheckin(CheckinType) PRIMARY KEY checkin_id AUTOGENERATED;
+CREATE DATASET YelpCheckinDateMarkers (CheckinDatesToMarkersType) PRIMARY KEY marker;
+CREATE INDEX IdxYelpCheckinDates ON YelpCheckin (UNNEST checkin_times UNNEST dates);
+
+SELECT COUNT(*)
+FROM YelpCheckinDateMarkers M
+INNER JOIN (
+    SELECT VALUE D
+    FROM YelpCheckin C, C.checkin_times CT, CT.dates D
+) AS CD ON CD /*+ indexnl */ = M.date;
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/array-index/select-quantified-queries/use-case-1/query1.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/array-index/select-quantified-queries/use-case-1/query1.sqlpp
new file mode 100644
index 0000000..f267701
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/array-index/select-quantified-queries/use-case-1/query1.sqlpp
@@ -0,0 +1,36 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+SET `compiler.arrayindex` "true";
+
+DROP DATAVERSE TestYelp IF EXISTS;
+CREATE DATAVERSE TestYelp;
+USE TestYelp;
+
+CREATE TYPE CheckinType AS {
+    checkin_id: uuid,
+    business_id: string,
+    dates: [string]
+};
+
+CREATE DATASET YelpCheckin(CheckinType) PRIMARY KEY checkin_id AUTOGENERATED;
+CREATE INDEX IdxYelpCheckinDates ON YelpCheckin (UNNEST dates);
+
+SELECT C.business_id
+FROM YelpCheckin C 
+WHERE "2016-04-26 19:49:16" IN C.dates;
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/array-index/select-quantified-queries/use-case-1/query2.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/array-index/select-quantified-queries/use-case-1/query2.sqlpp
new file mode 100644
index 0000000..6cbfdff
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/array-index/select-quantified-queries/use-case-1/query2.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 TestYelp IF EXISTS;
+CREATE DATAVERSE TestYelp;
+USE TestYelp;
+
+CREATE TYPE CheckinType AS {
+    checkin_id: uuid,
+    business_id: string,
+    dates: [string]
+};
+
+CREATE DATASET YelpCheckin(CheckinType) PRIMARY KEY checkin_id AUTOGENERATED;
+CREATE INDEX IdxYelpCheckinDates ON YelpCheckin (UNNEST dates);
+
+SELECT COUNT(*)
+FROM YelpCheckin C
+WHERE SOME D IN C.dates
+SATISFIES D > "2016" AND D < "2017";
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/array-index/select-quantified-queries/use-case-1/query3.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/array-index/select-quantified-queries/use-case-1/query3.sqlpp
new file mode 100644
index 0000000..d2dc752
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/array-index/select-quantified-queries/use-case-1/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 TestYelp IF EXISTS;
+CREATE DATAVERSE TestYelp;
+USE TestYelp;
+
+CREATE TYPE CheckinType AS {
+    checkin_id: uuid,
+    business_id: string,
+    dates: [string]
+};
+
+CREATE DATASET YelpCheckin(CheckinType) PRIMARY KEY checkin_id AUTOGENERATED;
+CREATE INDEX IdxYelpCheckinDates ON YelpCheckin (UNNEST dates);
+
+SELECT COUNT(*)
+FROM YelpCheckin C
+WHERE LEN(C.dates) > 0 AND
+      (EVERY D IN C.dates
+       SATISFIES D > "2016" AND D < "2017");
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/array-index/select-quantified-queries/use-case-2/query1.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/array-index/select-quantified-queries/use-case-2/query1.sqlpp
new file mode 100644
index 0000000..145f8f21
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/array-index/select-quantified-queries/use-case-2/query1.sqlpp
@@ -0,0 +1,39 @@
+/*
+ * 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 TestYelp IF EXISTS;
+CREATE DATAVERSE TestYelp;
+USE TestYelp;
+
+CREATE TYPE CheckinType AS {
+    checkin_id: uuid,
+    business_id: string,
+    checkin_times: {
+        dates: [string],
+        times: [string]
+    }
+};
+
+CREATE DATASET YelpCheckin(CheckinType) PRIMARY KEY checkin_id AUTOGENERATED;
+CREATE INDEX IdxYelpCheckinDates ON YelpCheckin (UNNEST checkin_times.dates);
+
+SELECT C.business_id
+FROM YelpCheckin C
+WHERE "2016-04-26" IN C.checkin_times.dates;
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/array-index/select-quantified-queries/use-case-2/query2.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/array-index/select-quantified-queries/use-case-2/query2.sqlpp
new file mode 100644
index 0000000..65e15fc
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/array-index/select-quantified-queries/use-case-2/query2.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 TestYelp IF EXISTS;
+CREATE DATAVERSE TestYelp;
+USE TestYelp;
+
+CREATE TYPE CheckinType AS {
+    checkin_id: uuid,
+    business_id: string,
+    checkin_times: {
+        dates: [string],
+        times: [string]
+    }
+};
+
+CREATE DATASET YelpCheckin(CheckinType) PRIMARY KEY checkin_id AUTOGENERATED;
+CREATE INDEX IdxYelpCheckinDates ON YelpCheckin (UNNEST checkin_times.dates);
+
+SELECT COUNT(*)
+FROM YelpCheckin C
+WHERE SOME D IN C.checkin_times.dates 
+SATISFIES D > "2016" AND D < "2017";
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/array-index/select-quantified-queries/use-case-2/query3.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/array-index/select-quantified-queries/use-case-2/query3.sqlpp
new file mode 100644
index 0000000..24ae2e4
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/array-index/select-quantified-queries/use-case-2/query3.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 TestYelp IF EXISTS;
+CREATE DATAVERSE TestYelp;
+USE TestYelp;
+
+CREATE TYPE CheckinType AS {
+    checkin_id: uuid,
+    business_id: string,
+    checkin_times: {
+        dates: [string],
+        times: [string]
+    }
+};
+
+CREATE DATASET YelpCheckin(CheckinType) PRIMARY KEY checkin_id AUTOGENERATED;
+CREATE INDEX IdxYelpCheckinDates ON YelpCheckin (UNNEST checkin_times.dates);
+
+SELECT COUNT(*)
+FROM YelpCheckin C
+WHERE LEN(C.checkin_times.dates) > 0 AND
+      (EVERY D IN C.checkin_times.dates
+       SATISFIES D > "2016" AND D < "2017");
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/array-index/select-quantified-queries/use-case-3/query1.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/array-index/select-quantified-queries/use-case-3/query1.sqlpp
new file mode 100644
index 0000000..3c07b9d
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/array-index/select-quantified-queries/use-case-3/query1.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 TestYelp IF EXISTS;
+CREATE DATAVERSE TestYelp;
+USE TestYelp;
+
+CREATE TYPE CheckinType AS {
+    checkin_id: uuid,
+    business_id: string,
+    checkin_times: [{
+        date: string,
+        time: string
+    }]
+};
+
+CREATE DATASET YelpCheckin(CheckinType) PRIMARY KEY checkin_id AUTOGENERATED;
+CREATE INDEX IdxYelpCheckinDates ON YelpCheckin (UNNEST checkin_times SELECT date);
+
+SELECT C.business_id
+FROM YelpCheckin C
+WHERE SOME D IN C.checkin_times 
+SATISFIES "2016-04-26" = D.date;
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/array-index/select-quantified-queries/use-case-3/query2.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/array-index/select-quantified-queries/use-case-3/query2.sqlpp
new file mode 100644
index 0000000..c8d5c23
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/array-index/select-quantified-queries/use-case-3/query2.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 TestYelp IF EXISTS;
+CREATE DATAVERSE TestYelp;
+USE TestYelp;
+
+CREATE TYPE CheckinType AS {
+    checkin_id: uuid,
+    business_id: string,
+    checkin_times: [{
+        date: string,
+        time: string
+    }]
+};
+
+CREATE DATASET YelpCheckin(CheckinType) PRIMARY KEY checkin_id AUTOGENERATED;
+CREATE INDEX IdxYelpCheckinDates ON YelpCheckin (UNNEST checkin_times SELECT date);
+
+SELECT C.business_id
+FROM YelpCheckin C
+WHERE SOME D IN C.checkin_times 
+SATISFIES D.date = "2016-04-26" AND D.time = "19:49:16";
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/array-index/select-quantified-queries/use-case-3/query3.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/array-index/select-quantified-queries/use-case-3/query3.sqlpp
new file mode 100644
index 0000000..52f533d
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/array-index/select-quantified-queries/use-case-3/query3.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 TestYelp IF EXISTS;
+CREATE DATAVERSE TestYelp;
+USE TestYelp;
+
+CREATE TYPE CheckinType AS {
+    checkin_id: uuid,
+    business_id: string,
+    checkin_times: [{
+        date: string,
+        time: string
+    }]
+};
+
+CREATE DATASET YelpCheckin(CheckinType) PRIMARY KEY checkin_id AUTOGENERATED;
+CREATE INDEX IdxYelpCheckinDates ON YelpCheckin (UNNEST checkin_times SELECT date);
+
+SELECT C.business_id
+FROM YelpCheckin C
+WHERE LEN(C.checkin_times) > 0 AND
+      (EVERY D IN C.checkin_times
+       SATISFIES D.date BETWEEN "2016" AND "2017");
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/array-index/select-quantified-queries/use-case-4/query1.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/array-index/select-quantified-queries/use-case-4/query1.sqlpp
new file mode 100644
index 0000000..1bd9564
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/array-index/select-quantified-queries/use-case-4/query1.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 TestYelp IF EXISTS;
+CREATE DATAVERSE TestYelp;
+USE TestYelp;
+
+CREATE TYPE CheckinType AS {
+    checkin_id: uuid,
+    business_id: string,
+    checkin_times: [{
+        dates: [string],
+        times: [string]
+    }]
+};
+
+CREATE DATASET YelpCheckin(CheckinType) PRIMARY KEY checkin_id AUTOGENERATED;
+CREATE INDEX IdxYelpCheckinDates ON YelpCheckin (UNNEST checkin_times UNNEST dates);
+
+SELECT C.business_id
+FROM YelpCheckin C
+WHERE SOME D IN C.checkin_times
+SATISFIES "2016-04-26" IN D.dates;
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/array-index/select-quantified-queries/use-case-4/query2.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/array-index/select-quantified-queries/use-case-4/query2.sqlpp
new file mode 100644
index 0000000..204e589
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/array-index/select-quantified-queries/use-case-4/query2.sqlpp
@@ -0,0 +1,43 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+SET `compiler.arrayindex` "true";
+
+DROP DATAVERSE TestYelp IF EXISTS;
+CREATE DATAVERSE TestYelp;
+USE TestYelp;
+
+CREATE TYPE CheckinType AS {
+    checkin_id: uuid,
+    business_id: string,
+    checkin_times: [{
+        dates: [string],
+        times: [string]
+    }]
+};
+
+CREATE DATASET YelpCheckin(CheckinType) PRIMARY KEY checkin_id AUTOGENERATED;
+CREATE INDEX IdxYelpCheckinDates ON YelpCheckin (UNNEST checkin_times UNNEST dates);
+
+SELECT C.business_id
+FROM YelpCheckin C
+WHERE SOME CT IN C.checkin_times
+SATISFIES (
+    SOME D IN CT.dates
+    SATISFIES D = "2016-04-26"
+);
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/array-index/select-quantified-queries/use-case-4/query3.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/array-index/select-quantified-queries/use-case-4/query3.sqlpp
new file mode 100644
index 0000000..8b3fd61
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/array-index/select-quantified-queries/use-case-4/query3.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 TestYelp IF EXISTS;
+CREATE DATAVERSE TestYelp;
+USE TestYelp;
+
+CREATE TYPE CheckinType AS {
+    checkin_id: uuid,
+    business_id: string,
+    checkin_times: [{
+        dates: [string],
+        times: [string]
+    }]
+};
+
+CREATE DATASET YelpCheckin(CheckinType) PRIMARY KEY checkin_id AUTOGENERATED;
+CREATE INDEX IdxYelpCheckinDates ON YelpCheckin (UNNEST checkin_times UNNEST dates);
+
+/* TODO (GLENN) Support checking for nested length clauses (i.e. EVERY, then EVERY). */
+SELECT C.business_id
+FROM YelpCheckin C
+WHERE LEN(C.checkin_times) > 0 AND
+      (EVERY CT IN C.checkin_times
+       SATISFIES (
+           SOME D IN CT.dates
+           SATISFIES "2019-06-07" = D
+      ));
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/array-index/select-unnest-queries/closed/use-case-1/query1.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/array-index/select-unnest-queries/closed/use-case-1/query1.sqlpp
new file mode 100644
index 0000000..d4bb5ff
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/array-index/select-unnest-queries/closed/use-case-1/query1.sqlpp
@@ -0,0 +1,36 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+SET `compiler.arrayindex` "true";
+
+DROP DATAVERSE TestYelp IF EXISTS;
+CREATE DATAVERSE TestYelp;
+USE TestYelp;
+
+CREATE TYPE CheckinType AS {
+    checkin_id: uuid,
+    business_id: string,
+    dates: [string]
+};
+
+CREATE DATASET YelpCheckin(CheckinType) PRIMARY KEY checkin_id AUTOGENERATED;
+CREATE INDEX IdxYelpCheckinDates ON YelpCheckin (UNNEST dates);
+
+SELECT C.business_id
+FROM YelpCheckin C, C.dates D
+WHERE "2016-04-26 19:49:16" = D;
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/array-index/select-unnest-queries/closed/use-case-1/query2.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/array-index/select-unnest-queries/closed/use-case-1/query2.sqlpp
new file mode 100644
index 0000000..ae997b3
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/array-index/select-unnest-queries/closed/use-case-1/query2.sqlpp
@@ -0,0 +1,36 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+SET `compiler.arrayindex` "true";
+
+DROP DATAVERSE TestYelp IF EXISTS;
+CREATE DATAVERSE TestYelp;
+USE TestYelp;
+
+CREATE TYPE CheckinType AS {
+    checkin_id: uuid,
+    business_id: string,
+    dates: [string]
+};
+
+CREATE DATASET YelpCheckin(CheckinType) PRIMARY KEY checkin_id AUTOGENERATED;
+CREATE INDEX IdxYelpCheckinDates ON YelpCheckin (UNNEST dates);
+
+SELECT COUNT(*)
+FROM YelpCheckin C, C.dates D
+WHERE D > "2016" AND D < "2017";
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/array-index/select-unnest-queries/closed/use-case-2/query1.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/array-index/select-unnest-queries/closed/use-case-2/query1.sqlpp
new file mode 100644
index 0000000..602e924
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/array-index/select-unnest-queries/closed/use-case-2/query1.sqlpp
@@ -0,0 +1,39 @@
+/*
+ * 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 TestYelp IF EXISTS;
+CREATE DATAVERSE TestYelp;
+USE TestYelp;
+
+CREATE TYPE CheckinType AS {
+    checkin_id: uuid,
+    business_id: string,
+    checkin_times: {
+        dates: [string],
+        times: [string]
+    }
+};
+
+CREATE DATASET YelpCheckin(CheckinType) PRIMARY KEY checkin_id AUTOGENERATED;
+CREATE INDEX IdxYelpCheckinDates ON YelpCheckin (UNNEST checkin_times.dates);
+
+SELECT C.business_id
+FROM YelpCheckin C, C.checkin_times.dates D
+WHERE "2016-04-26" = D;
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/array-index/select-unnest-queries/closed/use-case-2/query2.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/array-index/select-unnest-queries/closed/use-case-2/query2.sqlpp
new file mode 100644
index 0000000..0d94cce
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/array-index/select-unnest-queries/closed/use-case-2/query2.sqlpp
@@ -0,0 +1,39 @@
+/*
+ * 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 TestYelp IF EXISTS;
+CREATE DATAVERSE TestYelp;
+USE TestYelp;
+
+CREATE TYPE CheckinType AS {
+    checkin_id: uuid,
+    business_id: string,
+    checkin_times: {
+        dates: [string],
+        times: [string]
+    }
+};
+
+CREATE DATASET YelpCheckin(CheckinType) PRIMARY KEY checkin_id AUTOGENERATED;
+CREATE INDEX IdxYelpCheckinDates ON YelpCheckin (UNNEST checkin_times.dates);
+
+SELECT COUNT(*)
+FROM YelpCheckin C, C.checkin_times.dates D
+WHERE D > "2016" AND D < "2017";
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/array-index/select-unnest-queries/closed/use-case-3/query1.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/array-index/select-unnest-queries/closed/use-case-3/query1.sqlpp
new file mode 100644
index 0000000..52d826a
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/array-index/select-unnest-queries/closed/use-case-3/query1.sqlpp
@@ -0,0 +1,39 @@
+/*
+ * 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 TestYelp IF EXISTS;
+CREATE DATAVERSE TestYelp;
+USE TestYelp;
+
+CREATE TYPE CheckinType AS {
+    checkin_id: uuid,
+    business_id: string,
+    checkin_times: [{
+        date: string,
+        time: string
+    }]
+};
+
+CREATE DATASET YelpCheckin(CheckinType) PRIMARY KEY checkin_id AUTOGENERATED;
+CREATE INDEX IdxYelpCheckinDates ON YelpCheckin (UNNEST checkin_times SELECT date);
+
+SELECT C.business_id
+FROM YelpCheckin C, C.checkin_times D
+WHERE "2016-04-26" = D.date;
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/array-index/select-unnest-queries/closed/use-case-3/query2.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/array-index/select-unnest-queries/closed/use-case-3/query2.sqlpp
new file mode 100644
index 0000000..0755f55
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/array-index/select-unnest-queries/closed/use-case-3/query2.sqlpp
@@ -0,0 +1,39 @@
+/*
+ * 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 TestYelp IF EXISTS;
+CREATE DATAVERSE TestYelp;
+USE TestYelp;
+
+CREATE TYPE CheckinType AS {
+    checkin_id: uuid,
+    business_id: string,
+    checkin_times: [{
+        date: string,
+        time: string
+    }]
+};
+
+CREATE DATASET YelpCheckin(CheckinType) PRIMARY KEY checkin_id AUTOGENERATED;
+CREATE INDEX IdxYelpCheckinDates ON YelpCheckin (UNNEST checkin_times SELECT date);
+
+SELECT COUNT(*)
+FROM YelpCheckin C, C.checkin_times D
+WHERE D.date > "2016" AND D.date < "2017";
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/array-index/select-unnest-queries/closed/use-case-3/query3.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/array-index/select-unnest-queries/closed/use-case-3/query3.sqlpp
new file mode 100644
index 0000000..49949a0
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/array-index/select-unnest-queries/closed/use-case-3/query3.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 TestYelp IF EXISTS;
+CREATE DATAVERSE TestYelp;
+USE TestYelp;
+
+CREATE TYPE CheckinType AS {
+    checkin_id: uuid,
+    business_id: string,
+    checkin_times: [{
+        date: string,
+        time: string
+    }]
+};
+
+CREATE DATASET YelpCheckin(CheckinType) PRIMARY KEY checkin_id AUTOGENERATED;
+CREATE INDEX IdxYelpCheckinDates ON YelpCheckin (UNNEST checkin_times SELECT date);
+
+SELECT C.business_id
+FROM YelpCheckin C, C.checkin_times D
+WHERE "2016-04-26" = D.date AND 
+      D.time = "19:49:16"; 
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/array-index/select-unnest-queries/closed/use-case-4/query1.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/array-index/select-unnest-queries/closed/use-case-4/query1.sqlpp
new file mode 100644
index 0000000..2e3c0a8
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/array-index/select-unnest-queries/closed/use-case-4/query1.sqlpp
@@ -0,0 +1,39 @@
+/*
+ * 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 TestYelp IF EXISTS;
+CREATE DATAVERSE TestYelp;
+USE TestYelp;
+
+CREATE TYPE CheckinType AS {
+    checkin_id: uuid,
+    business_id: string,
+    checkin_times: [{
+        dates: [string],
+        times: [string]
+    }]
+};
+
+CREATE DATASET YelpCheckin(CheckinType) PRIMARY KEY checkin_id AUTOGENERATED;
+CREATE INDEX IdxYelpCheckinDates ON YelpCheckin (UNNEST checkin_times UNNEST dates);
+
+SELECT C.business_id
+FROM YelpCheckin C, C.checkin_times CT, CT.dates D
+WHERE "2016-04-26" = D;
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/array-index/select-unnest-queries/closed/use-case-4/query2.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/array-index/select-unnest-queries/closed/use-case-4/query2.sqlpp
new file mode 100644
index 0000000..89dc790
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/array-index/select-unnest-queries/closed/use-case-4/query2.sqlpp
@@ -0,0 +1,39 @@
+/*
+ * 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 TestYelp IF EXISTS;
+CREATE DATAVERSE TestYelp;
+USE TestYelp;
+
+CREATE TYPE CheckinType AS {
+    checkin_id: uuid,
+    business_id: string,
+    checkin_times: [{
+        dates: [string],
+        times: [string]
+    }]
+};
+
+CREATE DATASET YelpCheckin(CheckinType) PRIMARY KEY checkin_id AUTOGENERATED;
+CREATE INDEX IdxYelpCheckinDates ON YelpCheckin (UNNEST checkin_times UNNEST dates);
+
+SELECT COUNT(*)
+FROM YelpCheckin C, C.checkin_times CT, CT.dates D
+WHERE D > "2016" AND D < "2017";
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/array-index/select-unnest-queries/closed/with-3-level-record-path/query1.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/array-index/select-unnest-queries/closed/with-3-level-record-path/query1.sqlpp
new file mode 100644
index 0000000..660f62b
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/array-index/select-unnest-queries/closed/with-3-level-record-path/query1.sqlpp
@@ -0,0 +1,43 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+SET `compiler.arrayindex` "true";
+
+DROP DATAVERSE TestYelp IF EXISTS;
+CREATE DATAVERSE TestYelp;
+USE TestYelp;
+
+CREATE TYPE CheckinType AS {
+    checkin_id: uuid,
+    business_id: string,
+    checkin_data: {
+        checkin_temporal: {
+            checkin_times: {
+                dates: [string],
+                times: [string]
+            }
+        }
+    }
+};
+
+CREATE DATASET YelpCheckin(CheckinType) PRIMARY KEY checkin_id AUTOGENERATED;
+CREATE INDEX IdxYelpCheckinDates ON YelpCheckin (UNNEST checkin_data.checkin_temporal.checkin_times.dates);
+
+SELECT C.business_id
+FROM YelpCheckin C, C.checkin_data.checkin_temporal.checkin_times.dates D
+WHERE "2016-04-26" = D;
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/array-index/select-unnest-queries/closed/with-3-level-record-path/query2.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/array-index/select-unnest-queries/closed/with-3-level-record-path/query2.sqlpp
new file mode 100644
index 0000000..ce7fad8
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/array-index/select-unnest-queries/closed/with-3-level-record-path/query2.sqlpp
@@ -0,0 +1,43 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+SET `compiler.arrayindex` "true";
+
+DROP DATAVERSE TestYelp IF EXISTS;
+CREATE DATAVERSE TestYelp;
+USE TestYelp;
+
+CREATE TYPE CheckinType AS {
+    checkin_id: uuid,
+    business_id: string,
+    checkin_data: {
+        checkin_temporal: {
+            checkin_times: {
+                dates: [string],
+                times: [string]
+            }
+        }
+    }
+};
+
+CREATE DATASET YelpCheckin(CheckinType) PRIMARY KEY checkin_id AUTOGENERATED;
+CREATE INDEX IdxYelpCheckinDates ON YelpCheckin (UNNEST checkin_data.checkin_temporal.checkin_times.dates);
+
+SELECT COUNT(*)
+FROM YelpCheckin C, C.checkin_data.checkin_temporal.checkin_times.dates D
+WHERE D > "2016" AND D < "2017";
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/array-index/select-unnest-queries/closed/with-composite-pk/query1.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/array-index/select-unnest-queries/closed/with-composite-pk/query1.sqlpp
new file mode 100644
index 0000000..80a6bcc
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/array-index/select-unnest-queries/closed/with-composite-pk/query1.sqlpp
@@ -0,0 +1,36 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+SET `compiler.arrayindex` "true";
+
+DROP DATAVERSE TestYelp IF EXISTS;
+CREATE DATAVERSE TestYelp;
+USE TestYelp;
+
+CREATE TYPE CheckinType AS {
+    checkin_id: smallint,
+    business_id: string,
+    dates: [string]
+};
+
+CREATE DATASET YelpCheckin(CheckinType) PRIMARY KEY checkin_id, business_id;
+CREATE INDEX IdxYelpCheckinDates ON YelpCheckin (UNNEST dates);
+
+SELECT C.business_id
+FROM YelpCheckin C, C.dates D
+WHERE "2016-04-26 19:49:16" = D;
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/array-index/select-unnest-queries/closed/with-composite-pk/query2.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/array-index/select-unnest-queries/closed/with-composite-pk/query2.sqlpp
new file mode 100644
index 0000000..3ea8ac2
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/array-index/select-unnest-queries/closed/with-composite-pk/query2.sqlpp
@@ -0,0 +1,36 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+SET `compiler.arrayindex` "true";
+
+DROP DATAVERSE TestYelp IF EXISTS;
+CREATE DATAVERSE TestYelp;
+USE TestYelp;
+
+CREATE TYPE CheckinType AS {
+    checkin_id: smallint,
+    business_id: string,
+    dates: [string]
+};
+
+CREATE DATASET YelpCheckin(CheckinType) PRIMARY KEY checkin_id, business_id;
+CREATE INDEX IdxYelpCheckinDates ON YelpCheckin (UNNEST dates);
+
+SELECT COUNT(*)
+FROM YelpCheckin C, C.dates D
+WHERE D > "2016" AND D < "2017";
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/array-index/select-unnest-queries/closed/with-composite-sk/query1.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/array-index/select-unnest-queries/closed/with-composite-sk/query1.sqlpp
new file mode 100644
index 0000000..b9970c5
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/array-index/select-unnest-queries/closed/with-composite-sk/query1.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 TestYelp IF EXISTS;
+CREATE DATAVERSE TestYelp;
+USE TestYelp;
+
+CREATE TYPE CheckinType AS {
+    checkin_id: uuid,
+    business_id: string,
+    checkin_times: [{
+        date: string,
+        time: string
+    }]
+};
+
+CREATE DATASET YelpCheckin(CheckinType) PRIMARY KEY checkin_id AUTOGENERATED;
+CREATE INDEX IdxYelpCheckinDatesTimes ON YelpCheckin (UNNEST checkin_times SELECT date, time);
+
+SELECT C.business_id
+FROM YelpCheckin C, C.checkin_times D
+WHERE "2016-04-26" = D.date AND 
+      "19:49:16" = D.time AND
+      "--1UhMGODdWsrMastO9DZw" = C.business_id;
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/array-index/select-unnest-queries/closed/with-composite-sk/query2.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/array-index/select-unnest-queries/closed/with-composite-sk/query2.sqlpp
new file mode 100644
index 0000000..022d5ea
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/array-index/select-unnest-queries/closed/with-composite-sk/query2.sqlpp
@@ -0,0 +1,39 @@
+/*
+ * 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 TestYelp IF EXISTS;
+CREATE DATAVERSE TestYelp;
+USE TestYelp;
+
+CREATE TYPE CheckinType AS {
+    checkin_id: uuid,
+    business_id: string,
+    checkin_times: [{
+        date: string,
+        time: string
+    }]
+};
+
+CREATE DATASET YelpCheckin(CheckinType) PRIMARY KEY checkin_id AUTOGENERATED;
+CREATE INDEX IdxYelpCheckinDatesTimes ON YelpCheckin (UNNEST checkin_times SELECT date, time);
+
+SELECT COUNT(*)
+FROM YelpCheckin C, C.checkin_times D
+WHERE D.date > "2016" AND D.date < "2017";
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/array-index/select-unnest-queries/closed/with-filter-fields/query1.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/array-index/select-unnest-queries/closed/with-filter-fields/query1.sqlpp
new file mode 100644
index 0000000..9b36d62
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/array-index/select-unnest-queries/closed/with-filter-fields/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 TestYelp IF EXISTS;
+CREATE DATAVERSE TestYelp;
+USE TestYelp;
+
+CREATE TYPE CheckinType AS {
+    checkin_id: uuid,
+    business_id: string,
+    dates: [string]
+};
+
+CREATE DATASET YelpCheckin(CheckinType) PRIMARY KEY checkin_id AUTOGENERATED WITH FILTER ON business_id;
+CREATE INDEX IdxYelpCheckinDates ON YelpCheckin (UNNEST dates);
+
+SELECT C.business_id
+FROM YelpCheckin C, C.dates D
+WHERE "2016-04-26 19:49:16" = D AND 
+      C.business_id = "--1UhMGODdWsrMastO9DZw";
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/array-index/select-unnest-queries/closed/with-filter-fields/query2.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/array-index/select-unnest-queries/closed/with-filter-fields/query2.sqlpp
new file mode 100644
index 0000000..2cffd50
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/array-index/select-unnest-queries/closed/with-filter-fields/query2.sqlpp
@@ -0,0 +1,36 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+SET `compiler.arrayindex` "true";
+
+DROP DATAVERSE TestYelp IF EXISTS;
+CREATE DATAVERSE TestYelp;
+USE TestYelp;
+
+CREATE TYPE CheckinType AS {
+    checkin_id: uuid,
+    business_id: string,
+    dates: [string]
+};
+
+CREATE DATASET YelpCheckin(CheckinType) PRIMARY KEY checkin_id AUTOGENERATED WITH FILTER ON business_id;
+CREATE INDEX IdxYelpCheckinDates ON YelpCheckin (UNNEST dates);
+
+SELECT COUNT(*)
+FROM YelpCheckin C, C.dates D
+WHERE D > "2016" AND D < "2017";
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/array-index/select-unnest-queries/open/use-case-1/query1.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/array-index/select-unnest-queries/open/use-case-1/query1.sqlpp
new file mode 100644
index 0000000..71c910f
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/array-index/select-unnest-queries/open/use-case-1/query1.sqlpp
@@ -0,0 +1,34 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+SET `compiler.arrayindex` "true";
+
+DROP DATAVERSE TestYelp IF EXISTS;
+CREATE DATAVERSE TestYelp;
+USE TestYelp;
+
+CREATE TYPE CheckinType AS {
+    checkin_id: uuid
+};
+
+CREATE DATASET YelpCheckin(CheckinType) PRIMARY KEY checkin_id AUTOGENERATED;
+CREATE INDEX IdxYelpCheckinDates ON YelpCheckin (UNNEST dates : string ?) ;
+
+SELECT C.business_id
+FROM YelpCheckin C, C.dates D
+WHERE "2016-04-26 19:49:16" = D;
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/array-index/select-unnest-queries/open/use-case-1/query2.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/array-index/select-unnest-queries/open/use-case-1/query2.sqlpp
new file mode 100644
index 0000000..2d7e3a0
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/array-index/select-unnest-queries/open/use-case-1/query2.sqlpp
@@ -0,0 +1,34 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+SET `compiler.arrayindex` "true";
+
+DROP DATAVERSE TestYelp IF EXISTS;
+CREATE DATAVERSE TestYelp;
+USE TestYelp;
+
+CREATE TYPE CheckinType AS {
+    checkin_id: uuid
+};
+
+CREATE DATASET YelpCheckin(CheckinType) PRIMARY KEY checkin_id AUTOGENERATED;
+CREATE INDEX IdxYelpCheckinDates ON YelpCheckin (UNNEST dates : string ?) ;
+
+SELECT COUNT(*)
+FROM YelpCheckin C, C.dates D
+WHERE D > "2016" AND D < "2017";
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/array-index/select-unnest-queries/open/use-case-2/query1.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/array-index/select-unnest-queries/open/use-case-2/query1.sqlpp
new file mode 100644
index 0000000..c0f0377
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/array-index/select-unnest-queries/open/use-case-2/query1.sqlpp
@@ -0,0 +1,34 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+SET `compiler.arrayindex` "true";
+
+DROP DATAVERSE TestYelp IF EXISTS;
+CREATE DATAVERSE TestYelp;
+USE TestYelp;
+
+CREATE TYPE CheckinType AS {
+    checkin_id: uuid
+};
+
+CREATE DATASET YelpCheckin(CheckinType) PRIMARY KEY checkin_id AUTOGENERATED;
+CREATE INDEX IdxYelpCheckinDates ON YelpCheckin (UNNEST checkin_times.dates : string ?) ;
+
+SELECT C.business_id
+FROM YelpCheckin C, C.checkin_times.dates D
+WHERE "2016-04-26" = D;
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/array-index/select-unnest-queries/open/use-case-2/query2.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/array-index/select-unnest-queries/open/use-case-2/query2.sqlpp
new file mode 100644
index 0000000..e624637
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/array-index/select-unnest-queries/open/use-case-2/query2.sqlpp
@@ -0,0 +1,34 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+SET `compiler.arrayindex` "true";
+
+DROP DATAVERSE TestYelp IF EXISTS;
+CREATE DATAVERSE TestYelp;
+USE TestYelp;
+
+CREATE TYPE CheckinType AS {
+    checkin_id: uuid
+};
+
+CREATE DATASET YelpCheckin(CheckinType) PRIMARY KEY checkin_id AUTOGENERATED;
+CREATE INDEX IdxYelpCheckinDates ON YelpCheckin (UNNEST checkin_times.dates : string ?) ;
+
+SELECT COUNT(*)
+FROM YelpCheckin C, C.checkin_times.dates D
+WHERE D > "2016" AND D < "2017";
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/array-index/select-unnest-queries/open/use-case-3/query1.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/array-index/select-unnest-queries/open/use-case-3/query1.sqlpp
new file mode 100644
index 0000000..7054aed
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/array-index/select-unnest-queries/open/use-case-3/query1.sqlpp
@@ -0,0 +1,34 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+SET `compiler.arrayindex` "true";
+
+DROP DATAVERSE TestYelp IF EXISTS;
+CREATE DATAVERSE TestYelp;
+USE TestYelp;
+
+CREATE TYPE CheckinType AS {
+    checkin_id: uuid
+};
+
+CREATE DATASET YelpCheckin(CheckinType) PRIMARY KEY checkin_id AUTOGENERATED;
+CREATE INDEX IdxYelpCheckinDates ON YelpCheckin (UNNEST checkin_times SELECT date : string ? ) ;
+
+SELECT C.business_id
+FROM YelpCheckin C, C.checkin_times D
+WHERE "2016-04-26" = D.date;
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/array-index/select-unnest-queries/open/use-case-3/query2.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/array-index/select-unnest-queries/open/use-case-3/query2.sqlpp
new file mode 100644
index 0000000..ecfd0c5
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/array-index/select-unnest-queries/open/use-case-3/query2.sqlpp
@@ -0,0 +1,34 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+SET `compiler.arrayindex` "true";
+
+DROP DATAVERSE TestYelp IF EXISTS;
+CREATE DATAVERSE TestYelp;
+USE TestYelp;
+
+CREATE TYPE CheckinType AS {
+    checkin_id: uuid
+};
+
+CREATE DATASET YelpCheckin(CheckinType) PRIMARY KEY checkin_id AUTOGENERATED;
+CREATE INDEX IdxYelpCheckinDates ON YelpCheckin (UNNEST checkin_times SELECT date : string ?) ;
+
+SELECT COUNT(*)
+FROM YelpCheckin C, C.checkin_times D
+WHERE D.date > "2016" AND D.date < "2017";
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/array-index/select-unnest-queries/open/use-case-3/query3.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/array-index/select-unnest-queries/open/use-case-3/query3.sqlpp
new file mode 100644
index 0000000..97506d7
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/array-index/select-unnest-queries/open/use-case-3/query3.sqlpp
@@ -0,0 +1,35 @@
+/*
+ * 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 TestYelp IF EXISTS;
+CREATE DATAVERSE TestYelp;
+USE TestYelp;
+
+CREATE TYPE CheckinType AS {
+    checkin_id: uuid
+};
+
+CREATE DATASET YelpCheckin(CheckinType) PRIMARY KEY checkin_id AUTOGENERATED;
+CREATE INDEX IdxYelpCheckinDates ON YelpCheckin (UNNEST checkin_times SELECT date : string ? ) ;
+
+SELECT C.business_id
+FROM YelpCheckin C, C.checkin_times D
+WHERE "2016-04-26" = D.date AND 
+      D.time = "19:49:16"; 
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/array-index/select-unnest-queries/open/use-case-4/query1.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/array-index/select-unnest-queries/open/use-case-4/query1.sqlpp
new file mode 100644
index 0000000..464705e
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/array-index/select-unnest-queries/open/use-case-4/query1.sqlpp
@@ -0,0 +1,34 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+SET `compiler.arrayindex` "true";
+
+DROP DATAVERSE TestYelp IF EXISTS;
+CREATE DATAVERSE TestYelp;
+USE TestYelp;
+
+CREATE TYPE CheckinType AS {
+    checkin_id: uuid
+};
+
+CREATE DATASET YelpCheckin(CheckinType) PRIMARY KEY checkin_id AUTOGENERATED;
+CREATE INDEX IdxYelpCheckinDates ON YelpCheckin (UNNEST checkin_times UNNEST dates : string ?) ;
+
+SELECT C.business_id
+FROM YelpCheckin C, C.checkin_times CT, CT.dates D
+WHERE "2016-04-26" = D;
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/array-index/select-unnest-queries/open/use-case-4/query2.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/array-index/select-unnest-queries/open/use-case-4/query2.sqlpp
new file mode 100644
index 0000000..26fd6e8
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/array-index/select-unnest-queries/open/use-case-4/query2.sqlpp
@@ -0,0 +1,34 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+SET `compiler.arrayindex` "true";
+
+DROP DATAVERSE TestYelp IF EXISTS;
+CREATE DATAVERSE TestYelp;
+USE TestYelp;
+
+CREATE TYPE CheckinType AS {
+    checkin_id: uuid
+};
+
+CREATE DATASET YelpCheckin(CheckinType) PRIMARY KEY checkin_id AUTOGENERATED;
+CREATE INDEX IdxYelpCheckinDates ON YelpCheckin (UNNEST checkin_times UNNEST dates : string ?) ;
+
+SELECT COUNT(*)
+FROM YelpCheckin C, C.checkin_times CT, CT.dates D
+WHERE D > "2016" AND D < "2017";
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/array-index/select-unnest-queries/open/with-3-level-record-path/query1.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/array-index/select-unnest-queries/open/with-3-level-record-path/query1.sqlpp
new file mode 100644
index 0000000..913d010
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/array-index/select-unnest-queries/open/with-3-level-record-path/query1.sqlpp
@@ -0,0 +1,34 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+SET `compiler.arrayindex` "true";
+
+DROP DATAVERSE TestYelp IF EXISTS;
+CREATE DATAVERSE TestYelp;
+USE TestYelp;
+
+CREATE TYPE CheckinType AS {
+    checkin_id: uuid
+};
+
+CREATE DATASET YelpCheckin(CheckinType) PRIMARY KEY checkin_id AUTOGENERATED;
+CREATE INDEX IdxYelpCheckinDates ON YelpCheckin (UNNEST checkin_data.checkin_temporal.checkin_times.dates : string ?) ;
+
+SELECT C.business_id
+FROM YelpCheckin C, C.checkin_data.checkin_temporal.checkin_times.dates D
+WHERE "2016-04-26" = D;
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/array-index/select-unnest-queries/open/with-3-level-record-path/query2.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/array-index/select-unnest-queries/open/with-3-level-record-path/query2.sqlpp
new file mode 100644
index 0000000..8d56541
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/array-index/select-unnest-queries/open/with-3-level-record-path/query2.sqlpp
@@ -0,0 +1,34 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+SET `compiler.arrayindex` "true";
+
+DROP DATAVERSE TestYelp IF EXISTS;
+CREATE DATAVERSE TestYelp;
+USE TestYelp;
+
+CREATE TYPE CheckinType AS {
+    checkin_id: uuid
+};
+
+CREATE DATASET YelpCheckin(CheckinType) PRIMARY KEY checkin_id AUTOGENERATED;
+CREATE INDEX IdxYelpCheckinDates ON YelpCheckin (UNNEST checkin_data.checkin_temporal.checkin_times.dates : string ?) ;
+
+SELECT COUNT(*)
+FROM YelpCheckin C, C.checkin_data.checkin_temporal.checkin_times.dates D
+WHERE D > "2016" AND D < "2017";
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/array-index/select-unnest-queries/open/with-composite-sk/query1.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/array-index/select-unnest-queries/open/with-composite-sk/query1.sqlpp
new file mode 100644
index 0000000..3dcc3a2
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/array-index/select-unnest-queries/open/with-composite-sk/query1.sqlpp
@@ -0,0 +1,36 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+SET `compiler.arrayindex` "true";
+
+DROP DATAVERSE TestYelp IF EXISTS;
+CREATE DATAVERSE TestYelp;
+USE TestYelp;
+
+CREATE TYPE CheckinType AS {
+    checkin_id: uuid
+};
+
+CREATE DATASET YelpCheckin(CheckinType) PRIMARY KEY checkin_id AUTOGENERATED;
+CREATE INDEX IdxYelpCheckinDatesTimes ON YelpCheckin (UNNEST checkin_times SELECT date : string ?, time : string ?);
+
+SELECT C.business_id
+FROM YelpCheckin C, C.checkin_times D
+WHERE "2016-04-26" = D.date AND 
+      "19:49:16" = D.time AND
+      "--1UhMGODdWsrMastO9DZw" = C.business_id;
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/array-index/join-unnest-queries/use-case-1/query1.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/array-index/join-unnest-queries/use-case-1/query1.plan
new file mode 100644
index 0000000..e75ce49
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/array-index/join-unnest-queries/use-case-1/query1.plan
@@ -0,0 +1,29 @@
+-- DISTRIBUTE_RESULT  |PARTITIONED|
+  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+    -- STREAM_PROJECT  |PARTITIONED|
+      -- ASSIGN  |PARTITIONED|
+        -- STREAM_PROJECT  |PARTITIONED|
+          -- STREAM_SELECT  |PARTITIONED|
+            -- STREAM_PROJECT  |PARTITIONED|
+              -- UNNEST  |PARTITIONED|
+                -- STREAM_PROJECT  |PARTITIONED|
+                  -- STREAM_SELECT  |PARTITIONED|
+                    -- ASSIGN  |PARTITIONED|
+                      -- STREAM_PROJECT  |PARTITIONED|
+                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                          -- BTREE_SEARCH (TestYelp.YelpCheckin.YelpCheckin)  |PARTITIONED|
+                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                              -- PRE_SORTED_DISTINCT_BY  |PARTITIONED|
+                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                  -- STABLE_SORT [$$49(ASC), $$42(ASC)]  |PARTITIONED|
+                                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                      -- STREAM_PROJECT  |PARTITIONED|
+                                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                          -- BTREE_SEARCH (TestYelp.YelpCheckin.IdxYelpCheckinDates)  |PARTITIONED|
+                                            -- BROADCAST_EXCHANGE  |PARTITIONED|
+                                              -- STREAM_PROJECT  |PARTITIONED|
+                                                -- ASSIGN  |PARTITIONED|
+                                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                    -- DATASOURCE_SCAN (TestYelp.YelpCheckinDateMarkers)  |PARTITIONED|
+                                                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                        -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/array-index/join-unnest-queries/use-case-1/query2.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/array-index/join-unnest-queries/use-case-1/query2.plan
new file mode 100644
index 0000000..2c83436
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/array-index/join-unnest-queries/use-case-1/query2.plan
@@ -0,0 +1,29 @@
+-- DISTRIBUTE_RESULT  |PARTITIONED|
+  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+    -- STREAM_PROJECT  |PARTITIONED|
+      -- ASSIGN  |PARTITIONED|
+        -- STREAM_PROJECT  |PARTITIONED|
+          -- STREAM_SELECT  |PARTITIONED|
+            -- STREAM_PROJECT  |PARTITIONED|
+              -- UNNEST  |PARTITIONED|
+                -- STREAM_PROJECT  |PARTITIONED|
+                  -- ASSIGN  |PARTITIONED|
+                    -- STREAM_PROJECT  |PARTITIONED|
+                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                        -- BTREE_SEARCH (TestYelp.YelpCheckin.YelpCheckin)  |PARTITIONED|
+                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                            -- PRE_SORTED_DISTINCT_BY  |PARTITIONED|
+                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                -- STABLE_SORT [$$49(ASC), $$42(ASC)]  |PARTITIONED|
+                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                    -- STREAM_PROJECT  |PARTITIONED|
+                                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                        -- BTREE_SEARCH (TestYelp.YelpCheckin.IdxYelpCheckinDates)  |PARTITIONED|
+                                          -- BROADCAST_EXCHANGE  |PARTITIONED|
+                                            -- STREAM_PROJECT  |PARTITIONED|
+                                              -- STREAM_SELECT  |PARTITIONED|
+                                                -- ASSIGN  |PARTITIONED|
+                                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                    -- DATASOURCE_SCAN (TestYelp.YelpCheckinDateMarkers)  |PARTITIONED|
+                                                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                        -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/array-index/join-unnest-queries/use-case-1/query3.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/array-index/join-unnest-queries/use-case-1/query3.plan
new file mode 100644
index 0000000..56db4c9
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/array-index/join-unnest-queries/use-case-1/query3.plan
@@ -0,0 +1,32 @@
+-- DISTRIBUTE_RESULT  |UNPARTITIONED|
+  -- ONE_TO_ONE_EXCHANGE  |UNPARTITIONED|
+    -- STREAM_PROJECT  |UNPARTITIONED|
+      -- ASSIGN  |UNPARTITIONED|
+        -- AGGREGATE  |UNPARTITIONED|
+          -- RANDOM_MERGE_EXCHANGE  |PARTITIONED|
+            -- AGGREGATE  |PARTITIONED|
+              -- STREAM_SELECT  |PARTITIONED|
+                -- STREAM_PROJECT  |PARTITIONED|
+                  -- UNNEST  |PARTITIONED|
+                    -- STREAM_PROJECT  |PARTITIONED|
+                      -- ASSIGN  |PARTITIONED|
+                        -- STREAM_PROJECT  |PARTITIONED|
+                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                            -- BTREE_SEARCH (TestYelp.YelpCheckin.YelpCheckin)  |PARTITIONED|
+                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                -- STREAM_PROJECT  |PARTITIONED|
+                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                    -- PRE_SORTED_DISTINCT_BY  |PARTITIONED|
+                                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                        -- STABLE_SORT [$$75(ASC), $$68(ASC)]  |PARTITIONED|
+                                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                            -- STREAM_PROJECT  |PARTITIONED|
+                                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                -- BTREE_SEARCH (TestYelp.YelpCheckin.IdxYelpCheckinDates)  |PARTITIONED|
+                                                  -- BROADCAST_EXCHANGE  |PARTITIONED|
+                                                    -- STREAM_PROJECT  |PARTITIONED|
+                                                      -- ASSIGN  |PARTITIONED|
+                                                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                          -- DATASOURCE_SCAN (TestYelp.YelpCheckinDateMarkers)  |PARTITIONED|
+                                                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                              -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/array-index/join-unnest-queries/use-case-2/query1.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/array-index/join-unnest-queries/use-case-2/query1.plan
new file mode 100644
index 0000000..99cd582
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/array-index/join-unnest-queries/use-case-2/query1.plan
@@ -0,0 +1,29 @@
+-- DISTRIBUTE_RESULT  |PARTITIONED|
+  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+    -- STREAM_PROJECT  |PARTITIONED|
+      -- ASSIGN  |PARTITIONED|
+        -- STREAM_PROJECT  |PARTITIONED|
+          -- STREAM_SELECT  |PARTITIONED|
+            -- STREAM_PROJECT  |PARTITIONED|
+              -- UNNEST  |PARTITIONED|
+                -- STREAM_PROJECT  |PARTITIONED|
+                  -- ASSIGN  |PARTITIONED|
+                    -- STREAM_SELECT  |PARTITIONED|
+                      -- STREAM_PROJECT  |PARTITIONED|
+                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                          -- BTREE_SEARCH (TestYelp.YelpCheckin.YelpCheckin)  |PARTITIONED|
+                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                              -- PRE_SORTED_DISTINCT_BY  |PARTITIONED|
+                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                  -- STABLE_SORT [$$51(ASC), $$43(ASC)]  |PARTITIONED|
+                                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                      -- STREAM_PROJECT  |PARTITIONED|
+                                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                          -- BTREE_SEARCH (TestYelp.YelpCheckin.IdxYelpCheckinDates)  |PARTITIONED|
+                                            -- BROADCAST_EXCHANGE  |PARTITIONED|
+                                              -- STREAM_PROJECT  |PARTITIONED|
+                                                -- ASSIGN  |PARTITIONED|
+                                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                    -- DATASOURCE_SCAN (TestYelp.YelpCheckinDateMarkers)  |PARTITIONED|
+                                                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                        -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/array-index/join-unnest-queries/use-case-2/query2.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/array-index/join-unnest-queries/use-case-2/query2.plan
new file mode 100644
index 0000000..6864a95
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/array-index/join-unnest-queries/use-case-2/query2.plan
@@ -0,0 +1,29 @@
+-- DISTRIBUTE_RESULT  |PARTITIONED|
+  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+    -- STREAM_PROJECT  |PARTITIONED|
+      -- ASSIGN  |PARTITIONED|
+        -- STREAM_PROJECT  |PARTITIONED|
+          -- STREAM_SELECT  |PARTITIONED|
+            -- STREAM_PROJECT  |PARTITIONED|
+              -- UNNEST  |PARTITIONED|
+                -- STREAM_PROJECT  |PARTITIONED|
+                  -- ASSIGN  |PARTITIONED|
+                    -- STREAM_PROJECT  |PARTITIONED|
+                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                        -- BTREE_SEARCH (TestYelp.YelpCheckin.YelpCheckin)  |PARTITIONED|
+                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                            -- PRE_SORTED_DISTINCT_BY  |PARTITIONED|
+                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                -- STABLE_SORT [$$51(ASC), $$43(ASC)]  |PARTITIONED|
+                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                    -- STREAM_PROJECT  |PARTITIONED|
+                                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                        -- BTREE_SEARCH (TestYelp.YelpCheckin.IdxYelpCheckinDates)  |PARTITIONED|
+                                          -- BROADCAST_EXCHANGE  |PARTITIONED|
+                                            -- STREAM_PROJECT  |PARTITIONED|
+                                              -- STREAM_SELECT  |PARTITIONED|
+                                                -- ASSIGN  |PARTITIONED|
+                                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                    -- DATASOURCE_SCAN (TestYelp.YelpCheckinDateMarkers)  |PARTITIONED|
+                                                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                        -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/array-index/join-unnest-queries/use-case-2/query3.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/array-index/join-unnest-queries/use-case-2/query3.plan
new file mode 100644
index 0000000..b4ee1b0
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/array-index/join-unnest-queries/use-case-2/query3.plan
@@ -0,0 +1,32 @@
+-- DISTRIBUTE_RESULT  |UNPARTITIONED|
+  -- ONE_TO_ONE_EXCHANGE  |UNPARTITIONED|
+    -- STREAM_PROJECT  |UNPARTITIONED|
+      -- ASSIGN  |UNPARTITIONED|
+        -- AGGREGATE  |UNPARTITIONED|
+          -- RANDOM_MERGE_EXCHANGE  |PARTITIONED|
+            -- AGGREGATE  |PARTITIONED|
+              -- STREAM_SELECT  |PARTITIONED|
+                -- STREAM_PROJECT  |PARTITIONED|
+                  -- UNNEST  |PARTITIONED|
+                    -- STREAM_PROJECT  |PARTITIONED|
+                      -- ASSIGN  |PARTITIONED|
+                        -- STREAM_PROJECT  |PARTITIONED|
+                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                            -- BTREE_SEARCH (TestYelp.YelpCheckin.YelpCheckin)  |PARTITIONED|
+                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                -- STREAM_PROJECT  |PARTITIONED|
+                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                    -- PRE_SORTED_DISTINCT_BY  |PARTITIONED|
+                                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                        -- STABLE_SORT [$$77(ASC), $$69(ASC)]  |PARTITIONED|
+                                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                            -- STREAM_PROJECT  |PARTITIONED|
+                                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                -- BTREE_SEARCH (TestYelp.YelpCheckin.IdxYelpCheckinDates)  |PARTITIONED|
+                                                  -- BROADCAST_EXCHANGE  |PARTITIONED|
+                                                    -- STREAM_PROJECT  |PARTITIONED|
+                                                      -- ASSIGN  |PARTITIONED|
+                                                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                          -- DATASOURCE_SCAN (TestYelp.YelpCheckinDateMarkers)  |PARTITIONED|
+                                                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                              -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/array-index/join-unnest-queries/use-case-3/query1.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/array-index/join-unnest-queries/use-case-3/query1.plan
new file mode 100644
index 0000000..54e1f18
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/array-index/join-unnest-queries/use-case-3/query1.plan
@@ -0,0 +1,29 @@
+-- DISTRIBUTE_RESULT  |PARTITIONED|
+  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+    -- STREAM_PROJECT  |PARTITIONED|
+      -- ASSIGN  |PARTITIONED|
+        -- STREAM_PROJECT  |PARTITIONED|
+          -- STREAM_SELECT  |PARTITIONED|
+            -- STREAM_PROJECT  |PARTITIONED|
+              -- UNNEST  |PARTITIONED|
+                -- STREAM_PROJECT  |PARTITIONED|
+                  -- STREAM_SELECT  |PARTITIONED|
+                    -- ASSIGN  |PARTITIONED|
+                      -- STREAM_PROJECT  |PARTITIONED|
+                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                          -- BTREE_SEARCH (TestYelp.YelpCheckin.YelpCheckin)  |PARTITIONED|
+                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                              -- PRE_SORTED_DISTINCT_BY  |PARTITIONED|
+                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                  -- STABLE_SORT [$$51(ASC), $$43(ASC)]  |PARTITIONED|
+                                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                      -- STREAM_PROJECT  |PARTITIONED|
+                                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                          -- BTREE_SEARCH (TestYelp.YelpCheckin.IdxYelpCheckinDates)  |PARTITIONED|
+                                            -- BROADCAST_EXCHANGE  |PARTITIONED|
+                                              -- STREAM_PROJECT  |PARTITIONED|
+                                                -- ASSIGN  |PARTITIONED|
+                                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                    -- DATASOURCE_SCAN (TestYelp.YelpCheckinDateMarkers)  |PARTITIONED|
+                                                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                        -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/array-index/join-unnest-queries/use-case-3/query2.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/array-index/join-unnest-queries/use-case-3/query2.plan
new file mode 100644
index 0000000..0bc4272
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/array-index/join-unnest-queries/use-case-3/query2.plan
@@ -0,0 +1,29 @@
+-- DISTRIBUTE_RESULT  |PARTITIONED|
+  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+    -- STREAM_PROJECT  |PARTITIONED|
+      -- ASSIGN  |PARTITIONED|
+        -- STREAM_PROJECT  |PARTITIONED|
+          -- STREAM_SELECT  |PARTITIONED|
+            -- STREAM_SELECT  |PARTITIONED|
+              -- STREAM_PROJECT  |PARTITIONED|
+                -- UNNEST  |PARTITIONED|
+                  -- STREAM_PROJECT  |PARTITIONED|
+                    -- ASSIGN  |PARTITIONED|
+                      -- STREAM_PROJECT  |PARTITIONED|
+                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                          -- BTREE_SEARCH (TestYelp.YelpCheckin.YelpCheckin)  |PARTITIONED|
+                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                              -- PRE_SORTED_DISTINCT_BY  |PARTITIONED|
+                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                  -- STABLE_SORT [$$51(ASC), $$43(ASC)]  |PARTITIONED|
+                                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                      -- STREAM_PROJECT  |PARTITIONED|
+                                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                          -- BTREE_SEARCH (TestYelp.YelpCheckin.IdxYelpCheckinDates)  |PARTITIONED|
+                                            -- BROADCAST_EXCHANGE  |PARTITIONED|
+                                              -- STREAM_PROJECT  |PARTITIONED|
+                                                -- ASSIGN  |PARTITIONED|
+                                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                    -- DATASOURCE_SCAN (TestYelp.YelpCheckinDateMarkers)  |PARTITIONED|
+                                                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                        -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/array-index/join-unnest-queries/use-case-3/query3.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/array-index/join-unnest-queries/use-case-3/query3.plan
new file mode 100644
index 0000000..6864a95
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/array-index/join-unnest-queries/use-case-3/query3.plan
@@ -0,0 +1,29 @@
+-- DISTRIBUTE_RESULT  |PARTITIONED|
+  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+    -- STREAM_PROJECT  |PARTITIONED|
+      -- ASSIGN  |PARTITIONED|
+        -- STREAM_PROJECT  |PARTITIONED|
+          -- STREAM_SELECT  |PARTITIONED|
+            -- STREAM_PROJECT  |PARTITIONED|
+              -- UNNEST  |PARTITIONED|
+                -- STREAM_PROJECT  |PARTITIONED|
+                  -- ASSIGN  |PARTITIONED|
+                    -- STREAM_PROJECT  |PARTITIONED|
+                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                        -- BTREE_SEARCH (TestYelp.YelpCheckin.YelpCheckin)  |PARTITIONED|
+                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                            -- PRE_SORTED_DISTINCT_BY  |PARTITIONED|
+                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                -- STABLE_SORT [$$51(ASC), $$43(ASC)]  |PARTITIONED|
+                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                    -- STREAM_PROJECT  |PARTITIONED|
+                                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                        -- BTREE_SEARCH (TestYelp.YelpCheckin.IdxYelpCheckinDates)  |PARTITIONED|
+                                          -- BROADCAST_EXCHANGE  |PARTITIONED|
+                                            -- STREAM_PROJECT  |PARTITIONED|
+                                              -- STREAM_SELECT  |PARTITIONED|
+                                                -- ASSIGN  |PARTITIONED|
+                                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                    -- DATASOURCE_SCAN (TestYelp.YelpCheckinDateMarkers)  |PARTITIONED|
+                                                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                        -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/array-index/join-unnest-queries/use-case-3/query4.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/array-index/join-unnest-queries/use-case-3/query4.plan
new file mode 100644
index 0000000..f251b77
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/array-index/join-unnest-queries/use-case-3/query4.plan
@@ -0,0 +1,32 @@
+-- DISTRIBUTE_RESULT  |UNPARTITIONED|
+  -- ONE_TO_ONE_EXCHANGE  |UNPARTITIONED|
+    -- STREAM_PROJECT  |UNPARTITIONED|
+      -- ASSIGN  |UNPARTITIONED|
+        -- AGGREGATE  |UNPARTITIONED|
+          -- RANDOM_MERGE_EXCHANGE  |PARTITIONED|
+            -- AGGREGATE  |PARTITIONED|
+              -- STREAM_SELECT  |PARTITIONED|
+                -- STREAM_PROJECT  |PARTITIONED|
+                  -- UNNEST  |PARTITIONED|
+                    -- STREAM_PROJECT  |PARTITIONED|
+                      -- ASSIGN  |PARTITIONED|
+                        -- STREAM_PROJECT  |PARTITIONED|
+                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                            -- BTREE_SEARCH (TestYelp.YelpCheckin.YelpCheckin)  |PARTITIONED|
+                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                -- STREAM_PROJECT  |PARTITIONED|
+                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                    -- PRE_SORTED_DISTINCT_BY  |PARTITIONED|
+                                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                        -- STABLE_SORT [$$77(ASC), $$70(ASC)]  |PARTITIONED|
+                                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                            -- STREAM_PROJECT  |PARTITIONED|
+                                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                -- BTREE_SEARCH (TestYelp.YelpCheckin.IdxYelpCheckinDates)  |PARTITIONED|
+                                                  -- BROADCAST_EXCHANGE  |PARTITIONED|
+                                                    -- STREAM_PROJECT  |PARTITIONED|
+                                                      -- ASSIGN  |PARTITIONED|
+                                                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                          -- DATASOURCE_SCAN (TestYelp.YelpCheckinDateMarkers)  |PARTITIONED|
+                                                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                              -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/array-index/join-unnest-queries/use-case-4/query1.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/array-index/join-unnest-queries/use-case-4/query1.plan
new file mode 100644
index 0000000..778f88d
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/array-index/join-unnest-queries/use-case-4/query1.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|
+              -- UNNEST  |PARTITIONED|
+                -- STREAM_PROJECT  |PARTITIONED|
+                  -- ASSIGN  |PARTITIONED|
+                    -- STREAM_PROJECT  |PARTITIONED|
+                      -- UNNEST  |PARTITIONED|
+                        -- STREAM_PROJECT  |PARTITIONED|
+                          -- STREAM_SELECT  |PARTITIONED|
+                            -- ASSIGN  |PARTITIONED|
+                              -- STREAM_PROJECT  |PARTITIONED|
+                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                  -- BTREE_SEARCH (TestYelp.YelpCheckin.YelpCheckin)  |PARTITIONED|
+                                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                      -- PRE_SORTED_DISTINCT_BY  |PARTITIONED|
+                                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                          -- STABLE_SORT [$$61(ASC), $$53(ASC)]  |PARTITIONED|
+                                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                              -- STREAM_PROJECT  |PARTITIONED|
+                                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                  -- BTREE_SEARCH (TestYelp.YelpCheckin.IdxYelpCheckinDates)  |PARTITIONED|
+                                                    -- BROADCAST_EXCHANGE  |PARTITIONED|
+                                                      -- STREAM_PROJECT  |PARTITIONED|
+                                                        -- ASSIGN  |PARTITIONED|
+                                                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                            -- DATASOURCE_SCAN (TestYelp.YelpCheckinDateMarkers)  |PARTITIONED|
+                                                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/array-index/join-unnest-queries/use-case-4/query2.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/array-index/join-unnest-queries/use-case-4/query2.plan
new file mode 100644
index 0000000..8db17af
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/array-index/join-unnest-queries/use-case-4/query2.plan
@@ -0,0 +1,34 @@
+-- DISTRIBUTE_RESULT  |PARTITIONED|
+  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+    -- STREAM_PROJECT  |PARTITIONED|
+      -- ASSIGN  |PARTITIONED|
+        -- STREAM_PROJECT  |PARTITIONED|
+          -- STREAM_SELECT  |PARTITIONED|
+            -- STREAM_PROJECT  |PARTITIONED|
+              -- UNNEST  |PARTITIONED|
+                -- STREAM_PROJECT  |PARTITIONED|
+                  -- STREAM_SELECT  |PARTITIONED|
+                    -- ASSIGN  |PARTITIONED|
+                      -- STREAM_PROJECT  |PARTITIONED|
+                        -- UNNEST  |PARTITIONED|
+                          -- STREAM_PROJECT  |PARTITIONED|
+                            -- STREAM_SELECT  |PARTITIONED|
+                              -- ASSIGN  |PARTITIONED|
+                                -- STREAM_PROJECT  |PARTITIONED|
+                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                    -- BTREE_SEARCH (TestYelp.YelpCheckin.YelpCheckin)  |PARTITIONED|
+                                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                        -- PRE_SORTED_DISTINCT_BY  |PARTITIONED|
+                                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                            -- STABLE_SORT [$$64(ASC), $$55(ASC)]  |PARTITIONED|
+                                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                -- STREAM_PROJECT  |PARTITIONED|
+                                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                    -- BTREE_SEARCH (TestYelp.YelpCheckin.IdxYelpCheckinDates)  |PARTITIONED|
+                                                      -- BROADCAST_EXCHANGE  |PARTITIONED|
+                                                        -- STREAM_PROJECT  |PARTITIONED|
+                                                          -- ASSIGN  |PARTITIONED|
+                                                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                              -- DATASOURCE_SCAN (TestYelp.YelpCheckinDateMarkers)  |PARTITIONED|
+                                                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                  -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/array-index/join-unnest-queries/use-case-4/query3.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/array-index/join-unnest-queries/use-case-4/query3.plan
new file mode 100644
index 0000000..d28ee6d
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/array-index/join-unnest-queries/use-case-4/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|
+              -- UNNEST  |PARTITIONED|
+                -- STREAM_PROJECT  |PARTITIONED|
+                  -- ASSIGN  |PARTITIONED|
+                    -- STREAM_PROJECT  |PARTITIONED|
+                      -- UNNEST  |PARTITIONED|
+                        -- STREAM_PROJECT  |PARTITIONED|
+                          -- ASSIGN  |PARTITIONED|
+                            -- STREAM_PROJECT  |PARTITIONED|
+                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                -- BTREE_SEARCH (TestYelp.YelpCheckin.YelpCheckin)  |PARTITIONED|
+                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                    -- PRE_SORTED_DISTINCT_BY  |PARTITIONED|
+                                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                        -- STABLE_SORT [$$61(ASC), $$53(ASC)]  |PARTITIONED|
+                                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                            -- STREAM_PROJECT  |PARTITIONED|
+                                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                -- BTREE_SEARCH (TestYelp.YelpCheckin.IdxYelpCheckinDates)  |PARTITIONED|
+                                                  -- BROADCAST_EXCHANGE  |PARTITIONED|
+                                                    -- STREAM_PROJECT  |PARTITIONED|
+                                                      -- STREAM_SELECT  |PARTITIONED|
+                                                        -- ASSIGN  |PARTITIONED|
+                                                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                            -- DATASOURCE_SCAN (TestYelp.YelpCheckinDateMarkers)  |PARTITIONED|
+                                                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/array-index/join-unnest-queries/use-case-4/query4.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/array-index/join-unnest-queries/use-case-4/query4.plan
new file mode 100644
index 0000000..22b9b1c
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/array-index/join-unnest-queries/use-case-4/query4.plan
@@ -0,0 +1,36 @@
+-- DISTRIBUTE_RESULT  |UNPARTITIONED|
+  -- ONE_TO_ONE_EXCHANGE  |UNPARTITIONED|
+    -- STREAM_PROJECT  |UNPARTITIONED|
+      -- ASSIGN  |UNPARTITIONED|
+        -- AGGREGATE  |UNPARTITIONED|
+          -- RANDOM_MERGE_EXCHANGE  |PARTITIONED|
+            -- AGGREGATE  |PARTITIONED|
+              -- STREAM_SELECT  |PARTITIONED|
+                -- STREAM_PROJECT  |PARTITIONED|
+                  -- UNNEST  |PARTITIONED|
+                    -- STREAM_PROJECT  |PARTITIONED|
+                      -- ASSIGN  |PARTITIONED|
+                        -- STREAM_PROJECT  |PARTITIONED|
+                          -- UNNEST  |PARTITIONED|
+                            -- STREAM_PROJECT  |PARTITIONED|
+                              -- ASSIGN  |PARTITIONED|
+                                -- STREAM_PROJECT  |PARTITIONED|
+                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                    -- BTREE_SEARCH (TestYelp.YelpCheckin.YelpCheckin)  |PARTITIONED|
+                                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                        -- STREAM_PROJECT  |PARTITIONED|
+                                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                            -- PRE_SORTED_DISTINCT_BY  |PARTITIONED|
+                                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                -- STABLE_SORT [$$87(ASC), $$79(ASC)]  |PARTITIONED|
+                                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                    -- STREAM_PROJECT  |PARTITIONED|
+                                                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                        -- BTREE_SEARCH (TestYelp.YelpCheckin.IdxYelpCheckinDates)  |PARTITIONED|
+                                                          -- BROADCAST_EXCHANGE  |PARTITIONED|
+                                                            -- STREAM_PROJECT  |PARTITIONED|
+                                                              -- ASSIGN  |PARTITIONED|
+                                                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                  -- DATASOURCE_SCAN (TestYelp.YelpCheckinDateMarkers)  |PARTITIONED|
+                                                                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                      -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/array-index/select-quantified-queries/use-case-1/query1.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/array-index/select-quantified-queries/use-case-1/query1.plan
new file mode 100644
index 0000000..2c5b278
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/array-index/select-quantified-queries/use-case-1/query1.plan
@@ -0,0 +1,30 @@
+-- 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|
+                              -- NESTED_TUPLE_SOURCE  |LOCAL|
+                      }
+                -- STREAM_PROJECT  |PARTITIONED|
+                  -- ASSIGN  |PARTITIONED|
+                    -- STREAM_PROJECT  |PARTITIONED|
+                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                        -- BTREE_SEARCH (TestYelp.YelpCheckin.YelpCheckin)  |PARTITIONED|
+                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                            -- PRE_SORTED_DISTINCT_BY  |PARTITIONED|
+                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                -- STABLE_SORT [$$26(ASC)]  |PARTITIONED|
+                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                    -- STREAM_PROJECT  |PARTITIONED|
+                                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                        -- BTREE_SEARCH (TestYelp.YelpCheckin.IdxYelpCheckinDates)  |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/use-case-1/query2.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/array-index/select-quantified-queries/use-case-1/query2.plan
new file mode 100644
index 0000000..761ed03
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/array-index/select-quantified-queries/use-case-1/query2.plan
@@ -0,0 +1,32 @@
+-- DISTRIBUTE_RESULT  |UNPARTITIONED|
+  -- ONE_TO_ONE_EXCHANGE  |UNPARTITIONED|
+    -- STREAM_PROJECT  |UNPARTITIONED|
+      -- ASSIGN  |UNPARTITIONED|
+        -- AGGREGATE  |UNPARTITIONED|
+          -- RANDOM_MERGE_EXCHANGE  |PARTITIONED|
+            -- AGGREGATE  |PARTITIONED|
+              -- STREAM_SELECT  |PARTITIONED|
+                -- STREAM_PROJECT  |PARTITIONED|
+                  -- SUBPLAN  |PARTITIONED|
+                          {
+                            -- AGGREGATE  |LOCAL|
+                              -- STREAM_SELECT  |LOCAL|
+                                -- UNNEST  |LOCAL|
+                                  -- NESTED_TUPLE_SOURCE  |LOCAL|
+                          }
+                    -- STREAM_PROJECT  |PARTITIONED|
+                      -- ASSIGN  |PARTITIONED|
+                        -- STREAM_PROJECT  |PARTITIONED|
+                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                            -- BTREE_SEARCH (TestYelp.YelpCheckin.YelpCheckin)  |PARTITIONED|
+                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                -- PRE_SORTED_DISTINCT_BY  |PARTITIONED|
+                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                    -- STABLE_SORT [$$55(ASC)]  |PARTITIONED|
+                                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                        -- STREAM_PROJECT  |PARTITIONED|
+                                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                            -- BTREE_SEARCH (TestYelp.YelpCheckin.IdxYelpCheckinDates)  |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/use-case-1/query3.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/array-index/select-quantified-queries/use-case-1/query3.plan
new file mode 100644
index 0000000..f2819ea
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/array-index/select-quantified-queries/use-case-1/query3.plan
@@ -0,0 +1,33 @@
+-- DISTRIBUTE_RESULT  |UNPARTITIONED|
+  -- ONE_TO_ONE_EXCHANGE  |UNPARTITIONED|
+    -- STREAM_PROJECT  |UNPARTITIONED|
+      -- ASSIGN  |UNPARTITIONED|
+        -- AGGREGATE  |UNPARTITIONED|
+          -- RANDOM_MERGE_EXCHANGE  |PARTITIONED|
+            -- AGGREGATE  |PARTITIONED|
+              -- STREAM_SELECT  |PARTITIONED|
+                -- STREAM_PROJECT  |PARTITIONED|
+                  -- SUBPLAN  |PARTITIONED|
+                          {
+                            -- AGGREGATE  |LOCAL|
+                              -- STREAM_SELECT  |LOCAL|
+                                -- UNNEST  |LOCAL|
+                                  -- NESTED_TUPLE_SOURCE  |LOCAL|
+                          }
+                    -- STREAM_SELECT  |PARTITIONED|
+                      -- STREAM_PROJECT  |PARTITIONED|
+                        -- ASSIGN  |PARTITIONED|
+                          -- STREAM_PROJECT  |PARTITIONED|
+                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                              -- BTREE_SEARCH (TestYelp.YelpCheckin.YelpCheckin)  |PARTITIONED|
+                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                  -- PRE_SORTED_DISTINCT_BY  |PARTITIONED|
+                                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                      -- STABLE_SORT [$$60(ASC)]  |PARTITIONED|
+                                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                          -- STREAM_PROJECT  |PARTITIONED|
+                                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                              -- BTREE_SEARCH (TestYelp.YelpCheckin.IdxYelpCheckinDates)  |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/use-case-2/query1.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/array-index/select-quantified-queries/use-case-2/query1.plan
new file mode 100644
index 0000000..e29b93c
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/array-index/select-quantified-queries/use-case-2/query1.plan
@@ -0,0 +1,30 @@
+-- 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|
+                              -- NESTED_TUPLE_SOURCE  |LOCAL|
+                      }
+                -- STREAM_PROJECT  |PARTITIONED|
+                  -- ASSIGN  |PARTITIONED|
+                    -- STREAM_PROJECT  |PARTITIONED|
+                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                        -- BTREE_SEARCH (TestYelp.YelpCheckin.YelpCheckin)  |PARTITIONED|
+                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                            -- PRE_SORTED_DISTINCT_BY  |PARTITIONED|
+                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                -- STABLE_SORT [$$28(ASC)]  |PARTITIONED|
+                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                    -- STREAM_PROJECT  |PARTITIONED|
+                                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                        -- BTREE_SEARCH (TestYelp.YelpCheckin.IdxYelpCheckinDates)  |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/use-case-2/query2.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/array-index/select-quantified-queries/use-case-2/query2.plan
new file mode 100644
index 0000000..9843256
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/array-index/select-quantified-queries/use-case-2/query2.plan
@@ -0,0 +1,32 @@
+-- DISTRIBUTE_RESULT  |UNPARTITIONED|
+  -- ONE_TO_ONE_EXCHANGE  |UNPARTITIONED|
+    -- STREAM_PROJECT  |UNPARTITIONED|
+      -- ASSIGN  |UNPARTITIONED|
+        -- AGGREGATE  |UNPARTITIONED|
+          -- RANDOM_MERGE_EXCHANGE  |PARTITIONED|
+            -- AGGREGATE  |PARTITIONED|
+              -- STREAM_SELECT  |PARTITIONED|
+                -- STREAM_PROJECT  |PARTITIONED|
+                  -- SUBPLAN  |PARTITIONED|
+                          {
+                            -- AGGREGATE  |LOCAL|
+                              -- STREAM_SELECT  |LOCAL|
+                                -- UNNEST  |LOCAL|
+                                  -- NESTED_TUPLE_SOURCE  |LOCAL|
+                          }
+                    -- STREAM_PROJECT  |PARTITIONED|
+                      -- ASSIGN  |PARTITIONED|
+                        -- STREAM_PROJECT  |PARTITIONED|
+                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                            -- BTREE_SEARCH (TestYelp.YelpCheckin.YelpCheckin)  |PARTITIONED|
+                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                -- PRE_SORTED_DISTINCT_BY  |PARTITIONED|
+                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                    -- STABLE_SORT [$$57(ASC)]  |PARTITIONED|
+                                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                        -- STREAM_PROJECT  |PARTITIONED|
+                                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                            -- BTREE_SEARCH (TestYelp.YelpCheckin.IdxYelpCheckinDates)  |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/use-case-2/query3.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/array-index/select-quantified-queries/use-case-2/query3.plan
new file mode 100644
index 0000000..b9e4c18
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/array-index/select-quantified-queries/use-case-2/query3.plan
@@ -0,0 +1,33 @@
+-- DISTRIBUTE_RESULT  |UNPARTITIONED|
+  -- ONE_TO_ONE_EXCHANGE  |UNPARTITIONED|
+    -- STREAM_PROJECT  |UNPARTITIONED|
+      -- ASSIGN  |UNPARTITIONED|
+        -- AGGREGATE  |UNPARTITIONED|
+          -- RANDOM_MERGE_EXCHANGE  |PARTITIONED|
+            -- AGGREGATE  |PARTITIONED|
+              -- STREAM_SELECT  |PARTITIONED|
+                -- STREAM_PROJECT  |PARTITIONED|
+                  -- SUBPLAN  |PARTITIONED|
+                          {
+                            -- AGGREGATE  |LOCAL|
+                              -- STREAM_SELECT  |LOCAL|
+                                -- UNNEST  |LOCAL|
+                                  -- NESTED_TUPLE_SOURCE  |LOCAL|
+                          }
+                    -- STREAM_SELECT  |PARTITIONED|
+                      -- STREAM_PROJECT  |PARTITIONED|
+                        -- ASSIGN  |PARTITIONED|
+                          -- STREAM_PROJECT  |PARTITIONED|
+                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                              -- BTREE_SEARCH (TestYelp.YelpCheckin.YelpCheckin)  |PARTITIONED|
+                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                  -- PRE_SORTED_DISTINCT_BY  |PARTITIONED|
+                                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                      -- STABLE_SORT [$$63(ASC)]  |PARTITIONED|
+                                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                          -- STREAM_PROJECT  |PARTITIONED|
+                                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                              -- BTREE_SEARCH (TestYelp.YelpCheckin.IdxYelpCheckinDates)  |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/use-case-3/query1.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/array-index/select-quantified-queries/use-case-3/query1.plan
new file mode 100644
index 0000000..1d92106
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/array-index/select-quantified-queries/use-case-3/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|
+                            -- ASSIGN  |LOCAL|
+                              -- UNNEST  |LOCAL|
+                                -- NESTED_TUPLE_SOURCE  |LOCAL|
+                      }
+                -- STREAM_PROJECT  |PARTITIONED|
+                  -- ASSIGN  |PARTITIONED|
+                    -- STREAM_PROJECT  |PARTITIONED|
+                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                        -- BTREE_SEARCH (TestYelp.YelpCheckin.YelpCheckin)  |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 (TestYelp.YelpCheckin.IdxYelpCheckinDates)  |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/use-case-3/query2.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/array-index/select-quantified-queries/use-case-3/query2.plan
new file mode 100644
index 0000000..45549a5
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/array-index/select-quantified-queries/use-case-3/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|
+                            -- ASSIGN  |LOCAL|
+                              -- UNNEST  |LOCAL|
+                                -- NESTED_TUPLE_SOURCE  |LOCAL|
+                      }
+                -- STREAM_PROJECT  |PARTITIONED|
+                  -- ASSIGN  |PARTITIONED|
+                    -- STREAM_PROJECT  |PARTITIONED|
+                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                        -- BTREE_SEARCH (TestYelp.YelpCheckin.YelpCheckin)  |PARTITIONED|
+                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                            -- PRE_SORTED_DISTINCT_BY  |PARTITIONED|
+                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                -- STABLE_SORT [$$40(ASC)]  |PARTITIONED|
+                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                    -- STREAM_PROJECT  |PARTITIONED|
+                                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                        -- BTREE_SEARCH (TestYelp.YelpCheckin.IdxYelpCheckinDates)  |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/use-case-3/query3.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/array-index/select-quantified-queries/use-case-3/query3.plan
new file mode 100644
index 0000000..f8aad44
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/array-index/select-quantified-queries/use-case-3/query3.plan
@@ -0,0 +1,32 @@
+-- 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|
+                                -- NESTED_TUPLE_SOURCE  |LOCAL|
+                      }
+                -- STREAM_SELECT  |PARTITIONED|
+                  -- STREAM_PROJECT  |PARTITIONED|
+                    -- ASSIGN  |PARTITIONED|
+                      -- STREAM_PROJECT  |PARTITIONED|
+                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                          -- BTREE_SEARCH (TestYelp.YelpCheckin.YelpCheckin)  |PARTITIONED|
+                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                              -- PRE_SORTED_DISTINCT_BY  |PARTITIONED|
+                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                  -- STABLE_SORT [$$44(ASC)]  |PARTITIONED|
+                                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                      -- STREAM_PROJECT  |PARTITIONED|
+                                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                          -- BTREE_SEARCH (TestYelp.YelpCheckin.IdxYelpCheckinDates)  |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/use-case-4/query1.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/array-index/select-quantified-queries/use-case-4/query1.plan
new file mode 100644
index 0000000..84dc8c7
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/array-index/select-quantified-queries/use-case-4/query1.plan
@@ -0,0 +1,38 @@
+-- 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|
+                                          -- UNNEST  |LOCAL|
+                                            -- NESTED_TUPLE_SOURCE  |LOCAL|
+                                    }
+                              -- ASSIGN  |LOCAL|
+                                -- UNNEST  |LOCAL|
+                                  -- NESTED_TUPLE_SOURCE  |LOCAL|
+                      }
+                -- STREAM_PROJECT  |PARTITIONED|
+                  -- ASSIGN  |PARTITIONED|
+                    -- STREAM_PROJECT  |PARTITIONED|
+                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                        -- BTREE_SEARCH (TestYelp.YelpCheckin.YelpCheckin)  |PARTITIONED|
+                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                            -- PRE_SORTED_DISTINCT_BY  |PARTITIONED|
+                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                -- STABLE_SORT [$$41(ASC)]  |PARTITIONED|
+                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                    -- STREAM_PROJECT  |PARTITIONED|
+                                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                        -- BTREE_SEARCH (TestYelp.YelpCheckin.IdxYelpCheckinDates)  |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/use-case-4/query2.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/array-index/select-quantified-queries/use-case-4/query2.plan
new file mode 100644
index 0000000..a689c35
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/array-index/select-quantified-queries/use-case-4/query2.plan
@@ -0,0 +1,38 @@
+-- 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|
+                                          -- UNNEST  |LOCAL|
+                                            -- NESTED_TUPLE_SOURCE  |LOCAL|
+                                    }
+                              -- ASSIGN  |LOCAL|
+                                -- UNNEST  |LOCAL|
+                                  -- NESTED_TUPLE_SOURCE  |LOCAL|
+                      }
+                -- STREAM_PROJECT  |PARTITIONED|
+                  -- ASSIGN  |PARTITIONED|
+                    -- STREAM_PROJECT  |PARTITIONED|
+                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                        -- BTREE_SEARCH (TestYelp.YelpCheckin.YelpCheckin)  |PARTITIONED|
+                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                            -- PRE_SORTED_DISTINCT_BY  |PARTITIONED|
+                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                -- STABLE_SORT [$$49(ASC)]  |PARTITIONED|
+                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                    -- STREAM_PROJECT  |PARTITIONED|
+                                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                        -- BTREE_SEARCH (TestYelp.YelpCheckin.IdxYelpCheckinDates)  |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/use-case-4/query3.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/array-index/select-quantified-queries/use-case-4/query3.plan
new file mode 100644
index 0000000..bfdc30a
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/array-index/select-quantified-queries/use-case-4/query3.plan
@@ -0,0 +1,39 @@
+-- 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|
+                                          -- UNNEST  |LOCAL|
+                                            -- NESTED_TUPLE_SOURCE  |LOCAL|
+                                    }
+                              -- ASSIGN  |LOCAL|
+                                -- UNNEST  |LOCAL|
+                                  -- NESTED_TUPLE_SOURCE  |LOCAL|
+                      }
+                -- STREAM_SELECT  |PARTITIONED|
+                  -- STREAM_PROJECT  |PARTITIONED|
+                    -- ASSIGN  |PARTITIONED|
+                      -- STREAM_PROJECT  |PARTITIONED|
+                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                          -- BTREE_SEARCH (TestYelp.YelpCheckin.YelpCheckin)  |PARTITIONED|
+                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                              -- PRE_SORTED_DISTINCT_BY  |PARTITIONED|
+                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                  -- STABLE_SORT [$$54(ASC)]  |PARTITIONED|
+                                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                      -- STREAM_PROJECT  |PARTITIONED|
+                                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                          -- BTREE_SEARCH (TestYelp.YelpCheckin.IdxYelpCheckinDates)  |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/closed/use-case-1/query1.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/array-index/select-unnest-queries/closed/use-case-1/query1.plan
new file mode 100644
index 0000000..85a80b7
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/array-index/select-unnest-queries/closed/use-case-1/query1.plan
@@ -0,0 +1,23 @@
+-- DISTRIBUTE_RESULT  |PARTITIONED|
+  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+    -- STREAM_PROJECT  |PARTITIONED|
+      -- ASSIGN  |PARTITIONED|
+        -- STREAM_PROJECT  |PARTITIONED|
+          -- STREAM_SELECT  |PARTITIONED|
+            -- STREAM_PROJECT  |PARTITIONED|
+              -- UNNEST  |PARTITIONED|
+                -- ASSIGN  |PARTITIONED|
+                  -- STREAM_PROJECT  |PARTITIONED|
+                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                      -- BTREE_SEARCH (TestYelp.YelpCheckin.YelpCheckin)  |PARTITIONED|
+                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                          -- PRE_SORTED_DISTINCT_BY  |PARTITIONED|
+                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                              -- STABLE_SORT [$$33(ASC)]  |PARTITIONED|
+                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                  -- STREAM_PROJECT  |PARTITIONED|
+                                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                      -- BTREE_SEARCH (TestYelp.YelpCheckin.IdxYelpCheckinDates)  |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/closed/use-case-1/query2.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/array-index/select-unnest-queries/closed/use-case-1/query2.plan
new file mode 100644
index 0000000..a65a2c1
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/array-index/select-unnest-queries/closed/use-case-1/query2.plan
@@ -0,0 +1,26 @@
+-- DISTRIBUTE_RESULT  |UNPARTITIONED|
+  -- ONE_TO_ONE_EXCHANGE  |UNPARTITIONED|
+    -- STREAM_PROJECT  |UNPARTITIONED|
+      -- ASSIGN  |UNPARTITIONED|
+        -- AGGREGATE  |UNPARTITIONED|
+          -- RANDOM_MERGE_EXCHANGE  |PARTITIONED|
+            -- AGGREGATE  |PARTITIONED|
+              -- STREAM_SELECT  |PARTITIONED|
+                -- STREAM_PROJECT  |PARTITIONED|
+                  -- UNNEST  |PARTITIONED|
+                    -- STREAM_PROJECT  |PARTITIONED|
+                      -- ASSIGN  |PARTITIONED|
+                        -- STREAM_PROJECT  |PARTITIONED|
+                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                            -- BTREE_SEARCH (TestYelp.YelpCheckin.YelpCheckin)  |PARTITIONED|
+                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                -- PRE_SORTED_DISTINCT_BY  |PARTITIONED|
+                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                    -- STABLE_SORT [$$54(ASC)]  |PARTITIONED|
+                                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                        -- STREAM_PROJECT  |PARTITIONED|
+                                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                            -- BTREE_SEARCH (TestYelp.YelpCheckin.IdxYelpCheckinDates)  |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/closed/use-case-2/query1.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/array-index/select-unnest-queries/closed/use-case-2/query1.plan
new file mode 100644
index 0000000..3f9647b
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/array-index/select-unnest-queries/closed/use-case-2/query1.plan
@@ -0,0 +1,23 @@
+-- DISTRIBUTE_RESULT  |PARTITIONED|
+  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+    -- STREAM_PROJECT  |PARTITIONED|
+      -- ASSIGN  |PARTITIONED|
+        -- STREAM_PROJECT  |PARTITIONED|
+          -- STREAM_SELECT  |PARTITIONED|
+            -- STREAM_PROJECT  |PARTITIONED|
+              -- UNNEST  |PARTITIONED|
+                -- ASSIGN  |PARTITIONED|
+                  -- STREAM_PROJECT  |PARTITIONED|
+                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                      -- BTREE_SEARCH (TestYelp.YelpCheckin.YelpCheckin)  |PARTITIONED|
+                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                          -- PRE_SORTED_DISTINCT_BY  |PARTITIONED|
+                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                              -- STABLE_SORT [$$35(ASC)]  |PARTITIONED|
+                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                  -- STREAM_PROJECT  |PARTITIONED|
+                                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                      -- BTREE_SEARCH (TestYelp.YelpCheckin.IdxYelpCheckinDates)  |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/closed/use-case-2/query2.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/array-index/select-unnest-queries/closed/use-case-2/query2.plan
new file mode 100644
index 0000000..9af45a6
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/array-index/select-unnest-queries/closed/use-case-2/query2.plan
@@ -0,0 +1,26 @@
+-- DISTRIBUTE_RESULT  |UNPARTITIONED|
+  -- ONE_TO_ONE_EXCHANGE  |UNPARTITIONED|
+    -- STREAM_PROJECT  |UNPARTITIONED|
+      -- ASSIGN  |UNPARTITIONED|
+        -- AGGREGATE  |UNPARTITIONED|
+          -- RANDOM_MERGE_EXCHANGE  |PARTITIONED|
+            -- AGGREGATE  |PARTITIONED|
+              -- STREAM_SELECT  |PARTITIONED|
+                -- STREAM_PROJECT  |PARTITIONED|
+                  -- UNNEST  |PARTITIONED|
+                    -- STREAM_PROJECT  |PARTITIONED|
+                      -- ASSIGN  |PARTITIONED|
+                        -- STREAM_PROJECT  |PARTITIONED|
+                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                            -- BTREE_SEARCH (TestYelp.YelpCheckin.YelpCheckin)  |PARTITIONED|
+                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                -- PRE_SORTED_DISTINCT_BY  |PARTITIONED|
+                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                    -- STABLE_SORT [$$56(ASC)]  |PARTITIONED|
+                                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                        -- STREAM_PROJECT  |PARTITIONED|
+                                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                            -- BTREE_SEARCH (TestYelp.YelpCheckin.IdxYelpCheckinDates)  |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/closed/use-case-3/query1.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/array-index/select-unnest-queries/closed/use-case-3/query1.plan
new file mode 100644
index 0000000..3f9647b
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/array-index/select-unnest-queries/closed/use-case-3/query1.plan
@@ -0,0 +1,23 @@
+-- DISTRIBUTE_RESULT  |PARTITIONED|
+  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+    -- STREAM_PROJECT  |PARTITIONED|
+      -- ASSIGN  |PARTITIONED|
+        -- STREAM_PROJECT  |PARTITIONED|
+          -- STREAM_SELECT  |PARTITIONED|
+            -- STREAM_PROJECT  |PARTITIONED|
+              -- UNNEST  |PARTITIONED|
+                -- ASSIGN  |PARTITIONED|
+                  -- STREAM_PROJECT  |PARTITIONED|
+                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                      -- BTREE_SEARCH (TestYelp.YelpCheckin.YelpCheckin)  |PARTITIONED|
+                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                          -- PRE_SORTED_DISTINCT_BY  |PARTITIONED|
+                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                              -- STABLE_SORT [$$35(ASC)]  |PARTITIONED|
+                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                  -- STREAM_PROJECT  |PARTITIONED|
+                                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                      -- BTREE_SEARCH (TestYelp.YelpCheckin.IdxYelpCheckinDates)  |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/closed/use-case-3/query2.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/array-index/select-unnest-queries/closed/use-case-3/query2.plan
new file mode 100644
index 0000000..bb72552
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/array-index/select-unnest-queries/closed/use-case-3/query2.plan
@@ -0,0 +1,28 @@
+-- DISTRIBUTE_RESULT  |UNPARTITIONED|
+  -- ONE_TO_ONE_EXCHANGE  |UNPARTITIONED|
+    -- STREAM_PROJECT  |UNPARTITIONED|
+      -- ASSIGN  |UNPARTITIONED|
+        -- AGGREGATE  |UNPARTITIONED|
+          -- RANDOM_MERGE_EXCHANGE  |PARTITIONED|
+            -- AGGREGATE  |PARTITIONED|
+              -- STREAM_SELECT  |PARTITIONED|
+                -- STREAM_PROJECT  |PARTITIONED|
+                  -- ASSIGN  |PARTITIONED|
+                    -- STREAM_PROJECT  |PARTITIONED|
+                      -- UNNEST  |PARTITIONED|
+                        -- STREAM_PROJECT  |PARTITIONED|
+                          -- ASSIGN  |PARTITIONED|
+                            -- STREAM_PROJECT  |PARTITIONED|
+                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                -- BTREE_SEARCH (TestYelp.YelpCheckin.YelpCheckin)  |PARTITIONED|
+                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                    -- PRE_SORTED_DISTINCT_BY  |PARTITIONED|
+                                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                        -- STABLE_SORT [$$57(ASC)]  |PARTITIONED|
+                                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                            -- STREAM_PROJECT  |PARTITIONED|
+                                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                -- BTREE_SEARCH (TestYelp.YelpCheckin.IdxYelpCheckinDates)  |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/closed/use-case-3/query3.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/array-index/select-unnest-queries/closed/use-case-3/query3.plan
new file mode 100644
index 0000000..0202127
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/array-index/select-unnest-queries/closed/use-case-3/query3.plan
@@ -0,0 +1,23 @@
+-- DISTRIBUTE_RESULT  |PARTITIONED|
+  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+    -- STREAM_PROJECT  |PARTITIONED|
+      -- ASSIGN  |PARTITIONED|
+        -- STREAM_PROJECT  |PARTITIONED|
+          -- STREAM_SELECT  |PARTITIONED|
+            -- STREAM_PROJECT  |PARTITIONED|
+              -- UNNEST  |PARTITIONED|
+                -- ASSIGN  |PARTITIONED|
+                  -- STREAM_PROJECT  |PARTITIONED|
+                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                      -- BTREE_SEARCH (TestYelp.YelpCheckin.YelpCheckin)  |PARTITIONED|
+                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                          -- PRE_SORTED_DISTINCT_BY  |PARTITIONED|
+                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                              -- STABLE_SORT [$$39(ASC)]  |PARTITIONED|
+                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                  -- STREAM_PROJECT  |PARTITIONED|
+                                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                      -- BTREE_SEARCH (TestYelp.YelpCheckin.IdxYelpCheckinDates)  |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/closed/use-case-4/query1.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/array-index/select-unnest-queries/closed/use-case-4/query1.plan
new file mode 100644
index 0000000..4cf8dfe
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/array-index/select-unnest-queries/closed/use-case-4/query1.plan
@@ -0,0 +1,27 @@
+-- DISTRIBUTE_RESULT  |PARTITIONED|
+  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+    -- STREAM_PROJECT  |PARTITIONED|
+      -- ASSIGN  |PARTITIONED|
+        -- STREAM_PROJECT  |PARTITIONED|
+          -- STREAM_SELECT  |PARTITIONED|
+            -- STREAM_PROJECT  |PARTITIONED|
+              -- UNNEST  |PARTITIONED|
+                -- STREAM_PROJECT  |PARTITIONED|
+                  -- ASSIGN  |PARTITIONED|
+                    -- STREAM_PROJECT  |PARTITIONED|
+                      -- UNNEST  |PARTITIONED|
+                        -- ASSIGN  |PARTITIONED|
+                          -- STREAM_PROJECT  |PARTITIONED|
+                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                              -- BTREE_SEARCH (TestYelp.YelpCheckin.YelpCheckin)  |PARTITIONED|
+                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                  -- PRE_SORTED_DISTINCT_BY  |PARTITIONED|
+                                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                      -- STABLE_SORT [$$45(ASC)]  |PARTITIONED|
+                                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                          -- STREAM_PROJECT  |PARTITIONED|
+                                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                              -- BTREE_SEARCH (TestYelp.YelpCheckin.IdxYelpCheckinDates)  |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/closed/use-case-4/query2.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/array-index/select-unnest-queries/closed/use-case-4/query2.plan
new file mode 100644
index 0000000..4cb1efa
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/array-index/select-unnest-queries/closed/use-case-4/query2.plan
@@ -0,0 +1,30 @@
+-- DISTRIBUTE_RESULT  |UNPARTITIONED|
+  -- ONE_TO_ONE_EXCHANGE  |UNPARTITIONED|
+    -- STREAM_PROJECT  |UNPARTITIONED|
+      -- ASSIGN  |UNPARTITIONED|
+        -- AGGREGATE  |UNPARTITIONED|
+          -- RANDOM_MERGE_EXCHANGE  |PARTITIONED|
+            -- AGGREGATE  |PARTITIONED|
+              -- STREAM_SELECT  |PARTITIONED|
+                -- STREAM_PROJECT  |PARTITIONED|
+                  -- UNNEST  |PARTITIONED|
+                    -- STREAM_PROJECT  |PARTITIONED|
+                      -- ASSIGN  |PARTITIONED|
+                        -- STREAM_PROJECT  |PARTITIONED|
+                          -- UNNEST  |PARTITIONED|
+                            -- STREAM_PROJECT  |PARTITIONED|
+                              -- ASSIGN  |PARTITIONED|
+                                -- STREAM_PROJECT  |PARTITIONED|
+                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                    -- BTREE_SEARCH (TestYelp.YelpCheckin.YelpCheckin)  |PARTITIONED|
+                                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                        -- PRE_SORTED_DISTINCT_BY  |PARTITIONED|
+                                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                            -- STABLE_SORT [$$66(ASC)]  |PARTITIONED|
+                                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                -- STREAM_PROJECT  |PARTITIONED|
+                                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                    -- BTREE_SEARCH (TestYelp.YelpCheckin.IdxYelpCheckinDates)  |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/closed/with-3-level-record-path/query1.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/array-index/select-unnest-queries/closed/with-3-level-record-path/query1.plan
new file mode 100644
index 0000000..0202127
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/array-index/select-unnest-queries/closed/with-3-level-record-path/query1.plan
@@ -0,0 +1,23 @@
+-- DISTRIBUTE_RESULT  |PARTITIONED|
+  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+    -- STREAM_PROJECT  |PARTITIONED|
+      -- ASSIGN  |PARTITIONED|
+        -- STREAM_PROJECT  |PARTITIONED|
+          -- STREAM_SELECT  |PARTITIONED|
+            -- STREAM_PROJECT  |PARTITIONED|
+              -- UNNEST  |PARTITIONED|
+                -- ASSIGN  |PARTITIONED|
+                  -- STREAM_PROJECT  |PARTITIONED|
+                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                      -- BTREE_SEARCH (TestYelp.YelpCheckin.YelpCheckin)  |PARTITIONED|
+                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                          -- PRE_SORTED_DISTINCT_BY  |PARTITIONED|
+                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                              -- STABLE_SORT [$$39(ASC)]  |PARTITIONED|
+                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                  -- STREAM_PROJECT  |PARTITIONED|
+                                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                      -- BTREE_SEARCH (TestYelp.YelpCheckin.IdxYelpCheckinDates)  |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/closed/with-3-level-record-path/query2.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/array-index/select-unnest-queries/closed/with-3-level-record-path/query2.plan
new file mode 100644
index 0000000..9177af3
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/array-index/select-unnest-queries/closed/with-3-level-record-path/query2.plan
@@ -0,0 +1,26 @@
+-- DISTRIBUTE_RESULT  |UNPARTITIONED|
+  -- ONE_TO_ONE_EXCHANGE  |UNPARTITIONED|
+    -- STREAM_PROJECT  |UNPARTITIONED|
+      -- ASSIGN  |UNPARTITIONED|
+        -- AGGREGATE  |UNPARTITIONED|
+          -- RANDOM_MERGE_EXCHANGE  |PARTITIONED|
+            -- AGGREGATE  |PARTITIONED|
+              -- STREAM_SELECT  |PARTITIONED|
+                -- STREAM_PROJECT  |PARTITIONED|
+                  -- UNNEST  |PARTITIONED|
+                    -- STREAM_PROJECT  |PARTITIONED|
+                      -- ASSIGN  |PARTITIONED|
+                        -- STREAM_PROJECT  |PARTITIONED|
+                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                            -- BTREE_SEARCH (TestYelp.YelpCheckin.YelpCheckin)  |PARTITIONED|
+                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                -- PRE_SORTED_DISTINCT_BY  |PARTITIONED|
+                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                    -- STABLE_SORT [$$60(ASC)]  |PARTITIONED|
+                                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                        -- STREAM_PROJECT  |PARTITIONED|
+                                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                            -- BTREE_SEARCH (TestYelp.YelpCheckin.IdxYelpCheckinDates)  |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/closed/with-composite-pk/query1.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/array-index/select-unnest-queries/closed/with-composite-pk/query1.plan
new file mode 100644
index 0000000..883fa06
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/array-index/select-unnest-queries/closed/with-composite-pk/query1.plan
@@ -0,0 +1,24 @@
+-- DISTRIBUTE_RESULT  |PARTITIONED|
+  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+    -- STREAM_PROJECT  |PARTITIONED|
+      -- ASSIGN  |PARTITIONED|
+        -- STREAM_PROJECT  |PARTITIONED|
+          -- STREAM_SELECT  |PARTITIONED|
+            -- STREAM_PROJECT  |PARTITIONED|
+              -- UNNEST  |PARTITIONED|
+                -- STREAM_PROJECT  |PARTITIONED|
+                  -- ASSIGN  |PARTITIONED|
+                    -- STREAM_PROJECT  |PARTITIONED|
+                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                        -- BTREE_SEARCH (TestYelp.YelpCheckin.YelpCheckin)  |PARTITIONED|
+                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                            -- PRE_SORTED_DISTINCT_BY  |PARTITIONED|
+                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                -- STABLE_SORT [$$34(ASC), $$35(ASC)]  |PARTITIONED|
+                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                    -- STREAM_PROJECT  |PARTITIONED|
+                                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                        -- BTREE_SEARCH (TestYelp.YelpCheckin.IdxYelpCheckinDates)  |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/closed/with-composite-pk/query2.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/array-index/select-unnest-queries/closed/with-composite-pk/query2.plan
new file mode 100644
index 0000000..e4d4667
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/array-index/select-unnest-queries/closed/with-composite-pk/query2.plan
@@ -0,0 +1,26 @@
+-- DISTRIBUTE_RESULT  |UNPARTITIONED|
+  -- ONE_TO_ONE_EXCHANGE  |UNPARTITIONED|
+    -- STREAM_PROJECT  |UNPARTITIONED|
+      -- ASSIGN  |UNPARTITIONED|
+        -- AGGREGATE  |UNPARTITIONED|
+          -- RANDOM_MERGE_EXCHANGE  |PARTITIONED|
+            -- AGGREGATE  |PARTITIONED|
+              -- STREAM_SELECT  |PARTITIONED|
+                -- STREAM_PROJECT  |PARTITIONED|
+                  -- UNNEST  |PARTITIONED|
+                    -- STREAM_PROJECT  |PARTITIONED|
+                      -- ASSIGN  |PARTITIONED|
+                        -- STREAM_PROJECT  |PARTITIONED|
+                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                            -- BTREE_SEARCH (TestYelp.YelpCheckin.YelpCheckin)  |PARTITIONED|
+                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                -- PRE_SORTED_DISTINCT_BY  |PARTITIONED|
+                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                    -- STABLE_SORT [$$55(ASC), $$56(ASC)]  |PARTITIONED|
+                                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                        -- STREAM_PROJECT  |PARTITIONED|
+                                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                            -- BTREE_SEARCH (TestYelp.YelpCheckin.IdxYelpCheckinDates)  |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/closed/with-composite-sk/query1.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/array-index/select-unnest-queries/closed/with-composite-sk/query1.plan
new file mode 100644
index 0000000..2ad5dca
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/array-index/select-unnest-queries/closed/with-composite-sk/query1.plan
@@ -0,0 +1,25 @@
+-- DISTRIBUTE_RESULT  |PARTITIONED|
+  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+    -- STREAM_PROJECT  |PARTITIONED|
+      -- ASSIGN  |PARTITIONED|
+        -- STREAM_PROJECT  |PARTITIONED|
+          -- STREAM_SELECT  |PARTITIONED|
+            -- STREAM_PROJECT  |PARTITIONED|
+              -- UNNEST  |PARTITIONED|
+                -- STREAM_SELECT  |PARTITIONED|
+                  -- STREAM_PROJECT  |PARTITIONED|
+                    -- ASSIGN  |PARTITIONED|
+                      -- STREAM_PROJECT  |PARTITIONED|
+                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                          -- BTREE_SEARCH (TestYelp.YelpCheckin.YelpCheckin)  |PARTITIONED|
+                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                              -- PRE_SORTED_DISTINCT_BY  |PARTITIONED|
+                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                  -- STABLE_SORT [$$42(ASC)]  |PARTITIONED|
+                                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                      -- STREAM_PROJECT  |PARTITIONED|
+                                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                          -- BTREE_SEARCH (TestYelp.YelpCheckin.IdxYelpCheckinDatesTimes)  |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/closed/with-composite-sk/query2.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/array-index/select-unnest-queries/closed/with-composite-sk/query2.plan
new file mode 100644
index 0000000..afcd3a0
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/array-index/select-unnest-queries/closed/with-composite-sk/query2.plan
@@ -0,0 +1,28 @@
+-- DISTRIBUTE_RESULT  |UNPARTITIONED|
+  -- ONE_TO_ONE_EXCHANGE  |UNPARTITIONED|
+    -- STREAM_PROJECT  |UNPARTITIONED|
+      -- ASSIGN  |UNPARTITIONED|
+        -- AGGREGATE  |UNPARTITIONED|
+          -- RANDOM_MERGE_EXCHANGE  |PARTITIONED|
+            -- AGGREGATE  |PARTITIONED|
+              -- STREAM_SELECT  |PARTITIONED|
+                -- STREAM_PROJECT  |PARTITIONED|
+                  -- ASSIGN  |PARTITIONED|
+                    -- STREAM_PROJECT  |PARTITIONED|
+                      -- UNNEST  |PARTITIONED|
+                        -- STREAM_PROJECT  |PARTITIONED|
+                          -- ASSIGN  |PARTITIONED|
+                            -- STREAM_PROJECT  |PARTITIONED|
+                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                -- BTREE_SEARCH (TestYelp.YelpCheckin.YelpCheckin)  |PARTITIONED|
+                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                    -- PRE_SORTED_DISTINCT_BY  |PARTITIONED|
+                                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                        -- STABLE_SORT [$$58(ASC)]  |PARTITIONED|
+                                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                            -- STREAM_PROJECT  |PARTITIONED|
+                                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                -- BTREE_SEARCH (TestYelp.YelpCheckin.IdxYelpCheckinDatesTimes)  |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/closed/with-filter-fields/query1.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/array-index/select-unnest-queries/closed/with-filter-fields/query1.plan
new file mode 100644
index 0000000..3a14007
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/array-index/select-unnest-queries/closed/with-filter-fields/query1.plan
@@ -0,0 +1,25 @@
+-- DISTRIBUTE_RESULT  |PARTITIONED|
+  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+    -- STREAM_PROJECT  |PARTITIONED|
+      -- ASSIGN  |PARTITIONED|
+        -- STREAM_PROJECT  |PARTITIONED|
+          -- STREAM_SELECT  |PARTITIONED|
+            -- STREAM_PROJECT  |PARTITIONED|
+              -- UNNEST  |PARTITIONED|
+                -- STREAM_SELECT  |PARTITIONED|
+                  -- STREAM_PROJECT  |PARTITIONED|
+                    -- ASSIGN  |PARTITIONED|
+                      -- STREAM_PROJECT  |PARTITIONED|
+                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                          -- BTREE_SEARCH (TestYelp.YelpCheckin.YelpCheckin)  |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 (TestYelp.YelpCheckin.IdxYelpCheckinDates)  |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/closed/with-filter-fields/query2.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/array-index/select-unnest-queries/closed/with-filter-fields/query2.plan
new file mode 100644
index 0000000..a65a2c1
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/array-index/select-unnest-queries/closed/with-filter-fields/query2.plan
@@ -0,0 +1,26 @@
+-- DISTRIBUTE_RESULT  |UNPARTITIONED|
+  -- ONE_TO_ONE_EXCHANGE  |UNPARTITIONED|
+    -- STREAM_PROJECT  |UNPARTITIONED|
+      -- ASSIGN  |UNPARTITIONED|
+        -- AGGREGATE  |UNPARTITIONED|
+          -- RANDOM_MERGE_EXCHANGE  |PARTITIONED|
+            -- AGGREGATE  |PARTITIONED|
+              -- STREAM_SELECT  |PARTITIONED|
+                -- STREAM_PROJECT  |PARTITIONED|
+                  -- UNNEST  |PARTITIONED|
+                    -- STREAM_PROJECT  |PARTITIONED|
+                      -- ASSIGN  |PARTITIONED|
+                        -- STREAM_PROJECT  |PARTITIONED|
+                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                            -- BTREE_SEARCH (TestYelp.YelpCheckin.YelpCheckin)  |PARTITIONED|
+                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                -- PRE_SORTED_DISTINCT_BY  |PARTITIONED|
+                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                    -- STABLE_SORT [$$54(ASC)]  |PARTITIONED|
+                                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                        -- STREAM_PROJECT  |PARTITIONED|
+                                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                            -- BTREE_SEARCH (TestYelp.YelpCheckin.IdxYelpCheckinDates)  |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/use-case-1/query1.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/array-index/select-unnest-queries/open/use-case-1/query1.plan
new file mode 100644
index 0000000..85a80b7
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/array-index/select-unnest-queries/open/use-case-1/query1.plan
@@ -0,0 +1,23 @@
+-- DISTRIBUTE_RESULT  |PARTITIONED|
+  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+    -- STREAM_PROJECT  |PARTITIONED|
+      -- ASSIGN  |PARTITIONED|
+        -- STREAM_PROJECT  |PARTITIONED|
+          -- STREAM_SELECT  |PARTITIONED|
+            -- STREAM_PROJECT  |PARTITIONED|
+              -- UNNEST  |PARTITIONED|
+                -- ASSIGN  |PARTITIONED|
+                  -- STREAM_PROJECT  |PARTITIONED|
+                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                      -- BTREE_SEARCH (TestYelp.YelpCheckin.YelpCheckin)  |PARTITIONED|
+                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                          -- PRE_SORTED_DISTINCT_BY  |PARTITIONED|
+                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                              -- STABLE_SORT [$$33(ASC)]  |PARTITIONED|
+                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                  -- STREAM_PROJECT  |PARTITIONED|
+                                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                      -- BTREE_SEARCH (TestYelp.YelpCheckin.IdxYelpCheckinDates)  |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/use-case-1/query2.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/array-index/select-unnest-queries/open/use-case-1/query2.plan
new file mode 100644
index 0000000..a65a2c1
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/array-index/select-unnest-queries/open/use-case-1/query2.plan
@@ -0,0 +1,26 @@
+-- DISTRIBUTE_RESULT  |UNPARTITIONED|
+  -- ONE_TO_ONE_EXCHANGE  |UNPARTITIONED|
+    -- STREAM_PROJECT  |UNPARTITIONED|
+      -- ASSIGN  |UNPARTITIONED|
+        -- AGGREGATE  |UNPARTITIONED|
+          -- RANDOM_MERGE_EXCHANGE  |PARTITIONED|
+            -- AGGREGATE  |PARTITIONED|
+              -- STREAM_SELECT  |PARTITIONED|
+                -- STREAM_PROJECT  |PARTITIONED|
+                  -- UNNEST  |PARTITIONED|
+                    -- STREAM_PROJECT  |PARTITIONED|
+                      -- ASSIGN  |PARTITIONED|
+                        -- STREAM_PROJECT  |PARTITIONED|
+                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                            -- BTREE_SEARCH (TestYelp.YelpCheckin.YelpCheckin)  |PARTITIONED|
+                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                -- PRE_SORTED_DISTINCT_BY  |PARTITIONED|
+                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                    -- STABLE_SORT [$$54(ASC)]  |PARTITIONED|
+                                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                        -- STREAM_PROJECT  |PARTITIONED|
+                                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                            -- BTREE_SEARCH (TestYelp.YelpCheckin.IdxYelpCheckinDates)  |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/use-case-2/query1.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/array-index/select-unnest-queries/open/use-case-2/query1.plan
new file mode 100644
index 0000000..3f9647b
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/array-index/select-unnest-queries/open/use-case-2/query1.plan
@@ -0,0 +1,23 @@
+-- DISTRIBUTE_RESULT  |PARTITIONED|
+  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+    -- STREAM_PROJECT  |PARTITIONED|
+      -- ASSIGN  |PARTITIONED|
+        -- STREAM_PROJECT  |PARTITIONED|
+          -- STREAM_SELECT  |PARTITIONED|
+            -- STREAM_PROJECT  |PARTITIONED|
+              -- UNNEST  |PARTITIONED|
+                -- ASSIGN  |PARTITIONED|
+                  -- STREAM_PROJECT  |PARTITIONED|
+                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                      -- BTREE_SEARCH (TestYelp.YelpCheckin.YelpCheckin)  |PARTITIONED|
+                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                          -- PRE_SORTED_DISTINCT_BY  |PARTITIONED|
+                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                              -- STABLE_SORT [$$35(ASC)]  |PARTITIONED|
+                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                  -- STREAM_PROJECT  |PARTITIONED|
+                                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                      -- BTREE_SEARCH (TestYelp.YelpCheckin.IdxYelpCheckinDates)  |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/use-case-2/query2.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/array-index/select-unnest-queries/open/use-case-2/query2.plan
new file mode 100644
index 0000000..9af45a6
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/array-index/select-unnest-queries/open/use-case-2/query2.plan
@@ -0,0 +1,26 @@
+-- DISTRIBUTE_RESULT  |UNPARTITIONED|
+  -- ONE_TO_ONE_EXCHANGE  |UNPARTITIONED|
+    -- STREAM_PROJECT  |UNPARTITIONED|
+      -- ASSIGN  |UNPARTITIONED|
+        -- AGGREGATE  |UNPARTITIONED|
+          -- RANDOM_MERGE_EXCHANGE  |PARTITIONED|
+            -- AGGREGATE  |PARTITIONED|
+              -- STREAM_SELECT  |PARTITIONED|
+                -- STREAM_PROJECT  |PARTITIONED|
+                  -- UNNEST  |PARTITIONED|
+                    -- STREAM_PROJECT  |PARTITIONED|
+                      -- ASSIGN  |PARTITIONED|
+                        -- STREAM_PROJECT  |PARTITIONED|
+                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                            -- BTREE_SEARCH (TestYelp.YelpCheckin.YelpCheckin)  |PARTITIONED|
+                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                -- PRE_SORTED_DISTINCT_BY  |PARTITIONED|
+                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                    -- STABLE_SORT [$$56(ASC)]  |PARTITIONED|
+                                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                        -- STREAM_PROJECT  |PARTITIONED|
+                                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                            -- BTREE_SEARCH (TestYelp.YelpCheckin.IdxYelpCheckinDates)  |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/use-case-3/query1.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/array-index/select-unnest-queries/open/use-case-3/query1.plan
new file mode 100644
index 0000000..3f9647b
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/array-index/select-unnest-queries/open/use-case-3/query1.plan
@@ -0,0 +1,23 @@
+-- DISTRIBUTE_RESULT  |PARTITIONED|
+  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+    -- STREAM_PROJECT  |PARTITIONED|
+      -- ASSIGN  |PARTITIONED|
+        -- STREAM_PROJECT  |PARTITIONED|
+          -- STREAM_SELECT  |PARTITIONED|
+            -- STREAM_PROJECT  |PARTITIONED|
+              -- UNNEST  |PARTITIONED|
+                -- ASSIGN  |PARTITIONED|
+                  -- STREAM_PROJECT  |PARTITIONED|
+                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                      -- BTREE_SEARCH (TestYelp.YelpCheckin.YelpCheckin)  |PARTITIONED|
+                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                          -- PRE_SORTED_DISTINCT_BY  |PARTITIONED|
+                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                              -- STABLE_SORT [$$35(ASC)]  |PARTITIONED|
+                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                  -- STREAM_PROJECT  |PARTITIONED|
+                                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                      -- BTREE_SEARCH (TestYelp.YelpCheckin.IdxYelpCheckinDates)  |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/use-case-3/query2.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/array-index/select-unnest-queries/open/use-case-3/query2.plan
new file mode 100644
index 0000000..bb72552
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/array-index/select-unnest-queries/open/use-case-3/query2.plan
@@ -0,0 +1,28 @@
+-- DISTRIBUTE_RESULT  |UNPARTITIONED|
+  -- ONE_TO_ONE_EXCHANGE  |UNPARTITIONED|
+    -- STREAM_PROJECT  |UNPARTITIONED|
+      -- ASSIGN  |UNPARTITIONED|
+        -- AGGREGATE  |UNPARTITIONED|
+          -- RANDOM_MERGE_EXCHANGE  |PARTITIONED|
+            -- AGGREGATE  |PARTITIONED|
+              -- STREAM_SELECT  |PARTITIONED|
+                -- STREAM_PROJECT  |PARTITIONED|
+                  -- ASSIGN  |PARTITIONED|
+                    -- STREAM_PROJECT  |PARTITIONED|
+                      -- UNNEST  |PARTITIONED|
+                        -- STREAM_PROJECT  |PARTITIONED|
+                          -- ASSIGN  |PARTITIONED|
+                            -- STREAM_PROJECT  |PARTITIONED|
+                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                -- BTREE_SEARCH (TestYelp.YelpCheckin.YelpCheckin)  |PARTITIONED|
+                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                    -- PRE_SORTED_DISTINCT_BY  |PARTITIONED|
+                                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                        -- STABLE_SORT [$$57(ASC)]  |PARTITIONED|
+                                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                            -- STREAM_PROJECT  |PARTITIONED|
+                                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                -- BTREE_SEARCH (TestYelp.YelpCheckin.IdxYelpCheckinDates)  |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/use-case-3/query3.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/array-index/select-unnest-queries/open/use-case-3/query3.plan
new file mode 100644
index 0000000..0202127
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/array-index/select-unnest-queries/open/use-case-3/query3.plan
@@ -0,0 +1,23 @@
+-- DISTRIBUTE_RESULT  |PARTITIONED|
+  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+    -- STREAM_PROJECT  |PARTITIONED|
+      -- ASSIGN  |PARTITIONED|
+        -- STREAM_PROJECT  |PARTITIONED|
+          -- STREAM_SELECT  |PARTITIONED|
+            -- STREAM_PROJECT  |PARTITIONED|
+              -- UNNEST  |PARTITIONED|
+                -- ASSIGN  |PARTITIONED|
+                  -- STREAM_PROJECT  |PARTITIONED|
+                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                      -- BTREE_SEARCH (TestYelp.YelpCheckin.YelpCheckin)  |PARTITIONED|
+                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                          -- PRE_SORTED_DISTINCT_BY  |PARTITIONED|
+                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                              -- STABLE_SORT [$$39(ASC)]  |PARTITIONED|
+                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                  -- STREAM_PROJECT  |PARTITIONED|
+                                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                      -- BTREE_SEARCH (TestYelp.YelpCheckin.IdxYelpCheckinDates)  |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/use-case-4/query1.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/array-index/select-unnest-queries/open/use-case-4/query1.plan
new file mode 100644
index 0000000..4cf8dfe
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/array-index/select-unnest-queries/open/use-case-4/query1.plan
@@ -0,0 +1,27 @@
+-- DISTRIBUTE_RESULT  |PARTITIONED|
+  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+    -- STREAM_PROJECT  |PARTITIONED|
+      -- ASSIGN  |PARTITIONED|
+        -- STREAM_PROJECT  |PARTITIONED|
+          -- STREAM_SELECT  |PARTITIONED|
+            -- STREAM_PROJECT  |PARTITIONED|
+              -- UNNEST  |PARTITIONED|
+                -- STREAM_PROJECT  |PARTITIONED|
+                  -- ASSIGN  |PARTITIONED|
+                    -- STREAM_PROJECT  |PARTITIONED|
+                      -- UNNEST  |PARTITIONED|
+                        -- ASSIGN  |PARTITIONED|
+                          -- STREAM_PROJECT  |PARTITIONED|
+                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                              -- BTREE_SEARCH (TestYelp.YelpCheckin.YelpCheckin)  |PARTITIONED|
+                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                  -- PRE_SORTED_DISTINCT_BY  |PARTITIONED|
+                                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                      -- STABLE_SORT [$$45(ASC)]  |PARTITIONED|
+                                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                          -- STREAM_PROJECT  |PARTITIONED|
+                                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                              -- BTREE_SEARCH (TestYelp.YelpCheckin.IdxYelpCheckinDates)  |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/use-case-4/query2.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/array-index/select-unnest-queries/open/use-case-4/query2.plan
new file mode 100644
index 0000000..4cb1efa
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/array-index/select-unnest-queries/open/use-case-4/query2.plan
@@ -0,0 +1,30 @@
+-- DISTRIBUTE_RESULT  |UNPARTITIONED|
+  -- ONE_TO_ONE_EXCHANGE  |UNPARTITIONED|
+    -- STREAM_PROJECT  |UNPARTITIONED|
+      -- ASSIGN  |UNPARTITIONED|
+        -- AGGREGATE  |UNPARTITIONED|
+          -- RANDOM_MERGE_EXCHANGE  |PARTITIONED|
+            -- AGGREGATE  |PARTITIONED|
+              -- STREAM_SELECT  |PARTITIONED|
+                -- STREAM_PROJECT  |PARTITIONED|
+                  -- UNNEST  |PARTITIONED|
+                    -- STREAM_PROJECT  |PARTITIONED|
+                      -- ASSIGN  |PARTITIONED|
+                        -- STREAM_PROJECT  |PARTITIONED|
+                          -- UNNEST  |PARTITIONED|
+                            -- STREAM_PROJECT  |PARTITIONED|
+                              -- ASSIGN  |PARTITIONED|
+                                -- STREAM_PROJECT  |PARTITIONED|
+                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                    -- BTREE_SEARCH (TestYelp.YelpCheckin.YelpCheckin)  |PARTITIONED|
+                                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                        -- PRE_SORTED_DISTINCT_BY  |PARTITIONED|
+                                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                            -- STABLE_SORT [$$66(ASC)]  |PARTITIONED|
+                                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                -- STREAM_PROJECT  |PARTITIONED|
+                                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                    -- BTREE_SEARCH (TestYelp.YelpCheckin.IdxYelpCheckinDates)  |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/with-3-level-record-path/query1.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/array-index/select-unnest-queries/open/with-3-level-record-path/query1.plan
new file mode 100644
index 0000000..0202127
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/array-index/select-unnest-queries/open/with-3-level-record-path/query1.plan
@@ -0,0 +1,23 @@
+-- DISTRIBUTE_RESULT  |PARTITIONED|
+  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+    -- STREAM_PROJECT  |PARTITIONED|
+      -- ASSIGN  |PARTITIONED|
+        -- STREAM_PROJECT  |PARTITIONED|
+          -- STREAM_SELECT  |PARTITIONED|
+            -- STREAM_PROJECT  |PARTITIONED|
+              -- UNNEST  |PARTITIONED|
+                -- ASSIGN  |PARTITIONED|
+                  -- STREAM_PROJECT  |PARTITIONED|
+                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                      -- BTREE_SEARCH (TestYelp.YelpCheckin.YelpCheckin)  |PARTITIONED|
+                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                          -- PRE_SORTED_DISTINCT_BY  |PARTITIONED|
+                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                              -- STABLE_SORT [$$39(ASC)]  |PARTITIONED|
+                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                  -- STREAM_PROJECT  |PARTITIONED|
+                                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                      -- BTREE_SEARCH (TestYelp.YelpCheckin.IdxYelpCheckinDates)  |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/with-3-level-record-path/query2.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/array-index/select-unnest-queries/open/with-3-level-record-path/query2.plan
new file mode 100644
index 0000000..9177af3
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/array-index/select-unnest-queries/open/with-3-level-record-path/query2.plan
@@ -0,0 +1,26 @@
+-- DISTRIBUTE_RESULT  |UNPARTITIONED|
+  -- ONE_TO_ONE_EXCHANGE  |UNPARTITIONED|
+    -- STREAM_PROJECT  |UNPARTITIONED|
+      -- ASSIGN  |UNPARTITIONED|
+        -- AGGREGATE  |UNPARTITIONED|
+          -- RANDOM_MERGE_EXCHANGE  |PARTITIONED|
+            -- AGGREGATE  |PARTITIONED|
+              -- STREAM_SELECT  |PARTITIONED|
+                -- STREAM_PROJECT  |PARTITIONED|
+                  -- UNNEST  |PARTITIONED|
+                    -- STREAM_PROJECT  |PARTITIONED|
+                      -- ASSIGN  |PARTITIONED|
+                        -- STREAM_PROJECT  |PARTITIONED|
+                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                            -- BTREE_SEARCH (TestYelp.YelpCheckin.YelpCheckin)  |PARTITIONED|
+                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                -- PRE_SORTED_DISTINCT_BY  |PARTITIONED|
+                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                    -- STABLE_SORT [$$60(ASC)]  |PARTITIONED|
+                                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                        -- STREAM_PROJECT  |PARTITIONED|
+                                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                            -- BTREE_SEARCH (TestYelp.YelpCheckin.IdxYelpCheckinDates)  |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/with-composite-sk/query1.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/array-index/select-unnest-queries/open/with-composite-sk/query1.plan
new file mode 100644
index 0000000..8ae90bf
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/array-index/select-unnest-queries/open/with-composite-sk/query1.plan
@@ -0,0 +1,25 @@
+-- DISTRIBUTE_RESULT  |PARTITIONED|
+  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+    -- STREAM_PROJECT  |PARTITIONED|
+      -- ASSIGN  |PARTITIONED|
+        -- STREAM_PROJECT  |PARTITIONED|
+          -- STREAM_SELECT  |PARTITIONED|
+            -- STREAM_PROJECT  |PARTITIONED|
+              -- UNNEST  |PARTITIONED|
+                -- STREAM_SELECT  |PARTITIONED|
+                  -- STREAM_PROJECT  |PARTITIONED|
+                    -- ASSIGN  |PARTITIONED|
+                      -- STREAM_PROJECT  |PARTITIONED|
+                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                          -- BTREE_SEARCH (TestYelp.YelpCheckin.YelpCheckin)  |PARTITIONED|
+                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                              -- PRE_SORTED_DISTINCT_BY  |PARTITIONED|
+                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                  -- STABLE_SORT [$$44(ASC)]  |PARTITIONED|
+                                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                      -- STREAM_PROJECT  |PARTITIONED|
+                                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                          -- BTREE_SEARCH (TestYelp.YelpCheckin.IdxYelpCheckinDatesTimes)  |PARTITIONED|
+                                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                              -- ASSIGN  |PARTITIONED|
+                                                -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/bulk-loading/after-index-creation/use-case-1/use-case-1.1.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/bulk-loading/after-index-creation/use-case-1/use-case-1.1.ddl.sqlpp
new file mode 100644
index 0000000..3e2eeda
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/bulk-loading/after-index-creation/use-case-1/use-case-1.1.ddl.sqlpp
@@ -0,0 +1,38 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+/*
+ * Description: Verify the bulk-loading operation for an array->atomic index.
+ */
+
+DROP DATAVERSE TestYelp IF EXISTS;
+CREATE DATAVERSE TestYelp;
+USE TestYelp;
+
+CREATE TYPE CheckinType AS {
+    checkin_id: uuid,
+    business_id: string,
+    dates: [string]
+};
+
+CREATE DATASET YelpCheckin(CheckinType) PRIMARY KEY checkin_id AUTOGENERATED;
+CREATE INDEX IdxYelpCheckinDates ON YelpCheckin (UNNEST dates);
+// CREATE INDEX IdxYelpCheckinDatesBusinessID ON YelpCheckin (UNNEST dates, business_id);
+// CREATE INDEX IdxYelpCheckinBusinessIDDates ON YelpCheckin (business_id, UNNEST dates);
+CREATE INDEX IdxYelpCheckinBusinessID ON YelpCheckin (business_id);
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/bulk-loading/after-index-creation/use-case-1/use-case-1.2.update.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/bulk-loading/after-index-creation/use-case-1/use-case-1.2.update.sqlpp
new file mode 100644
index 0000000..f911125
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/bulk-loading/after-index-creation/use-case-1/use-case-1.2.update.sqlpp
@@ -0,0 +1,24 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+USE TestYelp;
+
+LOAD DATASET YelpCheckin 
+USING localfs (("path"="asterix_nc1://data/yelp-checkin/use-case-1.json"),
+                ("format"="json"));
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/bulk-loading/after-index-creation/use-case-1/use-case-1.3.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/bulk-loading/after-index-creation/use-case-1/use-case-1.3.query.sqlpp
new file mode 100644
index 0000000..ec79bc6
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/bulk-loading/after-index-creation/use-case-1/use-case-1.3.query.sqlpp
@@ -0,0 +1,25 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+SET `compiler.arrayindex` "true";
+
+USE TestYelp;
+
+SELECT C.business_id
+FROM YelpCheckin C, C.dates D
+WHERE "2016-04-26 19:49:16" = D;
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/bulk-loading/after-index-creation/use-case-2/use-case-2.1.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/bulk-loading/after-index-creation/use-case-2/use-case-2.1.ddl.sqlpp
new file mode 100644
index 0000000..ecef50a
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/bulk-loading/after-index-creation/use-case-2/use-case-2.1.ddl.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.
+ */
+
+/*
+ * Description: Verify the bulk-loading operation for a record->array->atomic index.
+ */
+
+DROP DATAVERSE TestYelp IF EXISTS;
+CREATE DATAVERSE TestYelp;
+USE TestYelp;
+
+CREATE TYPE CheckinType AS {
+    checkin_id: uuid,
+    business_id: string,
+    checkin_times: {
+        dates: [string],
+        times: [string]
+    }
+};
+
+CREATE DATASET YelpCheckin(CheckinType) PRIMARY KEY checkin_id AUTOGENERATED;
+CREATE INDEX IdxYelpCheckinDates ON YelpCheckin (UNNEST checkin_times.dates);
+// CREATE INDEX IdxYelpCheckinDatesBusiness ON YelpCheckin (UNNEST checkin_times.dates, (business_id));
+// CREATE INDEX IdxYelpCheckinBusinessDates ON YelpCheckin (business_id, UNNEST checkin_times.dates);
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/bulk-loading/after-index-creation/use-case-2/use-case-2.2.update.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/bulk-loading/after-index-creation/use-case-2/use-case-2.2.update.sqlpp
new file mode 100644
index 0000000..3fd8b82
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/bulk-loading/after-index-creation/use-case-2/use-case-2.2.update.sqlpp
@@ -0,0 +1,24 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+USE TestYelp;
+
+LOAD DATASET YelpCheckin 
+USING localfs (("path"="asterix_nc1://data/yelp-checkin/use-case-2.json"),
+                ("format"="json"));
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/bulk-loading/after-index-creation/use-case-2/use-case-2.3.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/bulk-loading/after-index-creation/use-case-2/use-case-2.3.query.sqlpp
new file mode 100644
index 0000000..0f23cd4
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/bulk-loading/after-index-creation/use-case-2/use-case-2.3.query.sqlpp
@@ -0,0 +1,25 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+SET `compiler.arrayindex` "true";
+
+USE TestYelp;
+
+SELECT C.business_id
+FROM YelpCheckin C, C.checkin_times.dates D
+WHERE "2016-04-26" = D;
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/bulk-loading/after-index-creation/use-case-3/use-case-3.1.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/bulk-loading/after-index-creation/use-case-3/use-case-3.1.ddl.sqlpp
new file mode 100644
index 0000000..abd1c6c
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/bulk-loading/after-index-creation/use-case-3/use-case-3.1.ddl.sqlpp
@@ -0,0 +1,42 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+/*
+ * Description: Verify the bulk-loading operation for an array->record->atomic index, as well as a composite index on both fields. 
+ */
+
+DROP DATAVERSE TestYelp IF EXISTS;
+CREATE DATAVERSE TestYelp;
+USE TestYelp;
+
+CREATE TYPE CheckinType AS {
+    checkin_id: uuid,
+    business_id: string,
+    checkin_times: [{
+        date: string,
+        time: string
+    }]
+};
+
+CREATE DATASET YelpCheckin(CheckinType) PRIMARY KEY checkin_id AUTOGENERATED;
+CREATE INDEX IdxYelpCheckinDates ON YelpCheckin (UNNEST checkin_times SELECT date);
+CREATE INDEX IdxYelpCheckinDatesTimes ON YelpCheckin (UNNEST checkin_times SELECT date, time);
+CREATE INDEX IdxYelpCheckinBusinessID ON YelpCheckin (business_id);
+// CREATE INDEX IdxYelpCheckinDatesTimesBusiness ON YelpCheckin ((UNNEST checkin_times SELECT date, time), (business_id));
+// CREATE INDEX IdxYelpCheckinBusinessDatesTimes ON YelpCheckin (business_id, UNNEST checkin_times SELECT date, time);
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/bulk-loading/after-index-creation/use-case-3/use-case-3.2.update.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/bulk-loading/after-index-creation/use-case-3/use-case-3.2.update.sqlpp
new file mode 100644
index 0000000..371bc27
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/bulk-loading/after-index-creation/use-case-3/use-case-3.2.update.sqlpp
@@ -0,0 +1,24 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+USE TestYelp;
+
+LOAD DATASET YelpCheckin 
+USING localfs (("path"="asterix_nc1://data/yelp-checkin/use-case-3.json"),
+                ("format"="json"));
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/bulk-loading/after-index-creation/use-case-3/use-case-3.3.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/bulk-loading/after-index-creation/use-case-3/use-case-3.3.query.sqlpp
new file mode 100644
index 0000000..27f01aa
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/bulk-loading/after-index-creation/use-case-3/use-case-3.3.query.sqlpp
@@ -0,0 +1,25 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+SET `compiler.arrayindex` "true";
+
+USE TestYelp;
+
+SELECT C.business_id
+FROM YelpCheckin C, C.checkin_times CT
+WHERE "2016-04-26" = CT.date;
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/bulk-loading/after-index-creation/use-case-4/use-case-4.1.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/bulk-loading/after-index-creation/use-case-4/use-case-4.1.ddl.sqlpp
new file mode 100644
index 0000000..626e797
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/bulk-loading/after-index-creation/use-case-4/use-case-4.1.ddl.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.
+ */
+
+/*
+ * Description: Verify the bulk-loading operation for an array->record->array->atomic index.
+ */
+
+DROP DATAVERSE TestYelp IF EXISTS;
+CREATE DATAVERSE TestYelp;
+USE TestYelp;
+
+CREATE TYPE CheckinType AS {
+    checkin_id: uuid,
+    business_id: string,
+    checkin_times: [{
+        dates: [string],
+        times: [string]
+    }]
+};
+
+CREATE DATASET YelpCheckin(CheckinType) PRIMARY KEY checkin_id AUTOGENERATED;
+CREATE INDEX IdxYelpCheckinDates ON YelpCheckin (UNNEST checkin_times UNNEST dates);
+// CREATE INDEX IdxYelpCheckinDatesBusiness ON YelpCheckin ((UNNEST checkin_times UNNEST dates), business_id);
+// CREATE INDEX IdxYelpCheckinBusinessDates ON YelpCheckin (business_id, UNNEST checkin_times UNNEST dates);
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/bulk-loading/after-index-creation/use-case-4/use-case-4.2.update.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/bulk-loading/after-index-creation/use-case-4/use-case-4.2.update.sqlpp
new file mode 100644
index 0000000..5f7417c
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/bulk-loading/after-index-creation/use-case-4/use-case-4.2.update.sqlpp
@@ -0,0 +1,24 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+USE TestYelp;
+
+LOAD DATASET YelpCheckin 
+USING localfs (("path"="asterix_nc1://data/yelp-checkin/use-case-4.json"),
+                ("format"="json"));
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/bulk-loading/after-index-creation/use-case-4/use-case-4.3.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/bulk-loading/after-index-creation/use-case-4/use-case-4.3.query.sqlpp
new file mode 100644
index 0000000..8ba650f
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/bulk-loading/after-index-creation/use-case-4/use-case-4.3.query.sqlpp
@@ -0,0 +1,25 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+SET `compiler.arrayindex` "true";
+
+USE TestYelp;
+
+SELECT C.business_id
+FROM YelpCheckin C, C.checkin_times CT, CT.dates D
+WHERE "2016-04-26" = D;
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/bulk-loading/after-index-creation/with-3-level-record-path/with-3-level-record-path.1.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/bulk-loading/after-index-creation/with-3-level-record-path/with-3-level-record-path.1.ddl.sqlpp
new file mode 100644
index 0000000..47b4d5b
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/bulk-loading/after-index-creation/with-3-level-record-path/with-3-level-record-path.1.ddl.sqlpp
@@ -0,0 +1,43 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+/*
+ * Description: Verify the bulk-loading operation for a record->record->record->array->atomic index.
+ */
+
+DROP DATAVERSE TestYelp IF EXISTS;
+CREATE DATAVERSE TestYelp;
+USE TestYelp;
+
+CREATE TYPE CheckinType AS {
+    checkin_id: uuid,
+    business_id: string,
+    checkin_data: {
+        checkin_temporal: {
+            checkin_times: {
+                dates: [string],
+                times: [string]
+            }
+        }
+    }
+
+};
+
+CREATE DATASET YelpCheckin(CheckinType) PRIMARY KEY checkin_id AUTOGENERATED;
+CREATE INDEX IdxYelpCheckinDates ON YelpCheckin (UNNEST checkin_data.checkin_temporal.checkin_times.dates);
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/bulk-loading/after-index-creation/with-3-level-record-path/with-3-level-record-path.2.update.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/bulk-loading/after-index-creation/with-3-level-record-path/with-3-level-record-path.2.update.sqlpp
new file mode 100644
index 0000000..277d30b
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/bulk-loading/after-index-creation/with-3-level-record-path/with-3-level-record-path.2.update.sqlpp
@@ -0,0 +1,24 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+USE TestYelp;
+
+LOAD DATASET YelpCheckin 
+USING localfs (("path"="asterix_nc1://data/yelp-checkin/with-3-level-record-path.json"),
+                ("format"="json"));
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/bulk-loading/after-index-creation/with-3-level-record-path/with-3-level-record-path.3.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/bulk-loading/after-index-creation/with-3-level-record-path/with-3-level-record-path.3.query.sqlpp
new file mode 100644
index 0000000..cee1887
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/bulk-loading/after-index-creation/with-3-level-record-path/with-3-level-record-path.3.query.sqlpp
@@ -0,0 +1,25 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+SET `compiler.arrayindex` "true";
+
+USE TestYelp;
+
+SELECT C.business_id
+FROM YelpCheckin C, C.checkin_data.checkin_temporal.checkin_times.dates D
+WHERE "2016-04-26" = D;
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/bulk-loading/after-index-creation/with-composite-pk/with-composite-pk.1.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/bulk-loading/after-index-creation/with-composite-pk/with-composite-pk.1.ddl.sqlpp
new file mode 100644
index 0000000..415054c
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/bulk-loading/after-index-creation/with-composite-pk/with-composite-pk.1.ddl.sqlpp
@@ -0,0 +1,35 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+/*
+ * Description: Verify the bulk-loading operation on array indexes with more than one PK.
+ */
+
+DROP DATAVERSE TestYelp IF EXISTS;
+CREATE DATAVERSE TestYelp;
+USE TestYelp;
+
+CREATE TYPE CheckinType AS {
+    checkin_id: smallint,
+    business_id: string,
+    dates: [string]
+};
+
+CREATE DATASET YelpCheckin(CheckinType) PRIMARY KEY checkin_id, business_id;
+CREATE INDEX IdxYelpCheckinDates ON YelpCheckin (UNNEST dates);
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/bulk-loading/after-index-creation/with-composite-pk/with-composite-pk.2.update.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/bulk-loading/after-index-creation/with-composite-pk/with-composite-pk.2.update.sqlpp
new file mode 100644
index 0000000..38b50e6
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/bulk-loading/after-index-creation/with-composite-pk/with-composite-pk.2.update.sqlpp
@@ -0,0 +1,24 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+USE TestYelp;
+
+LOAD DATASET YelpCheckin 
+USING localfs (("path"="asterix_nc1://data/yelp-checkin/with-composite-pk.json"),
+                ("format"="json"));
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/bulk-loading/after-index-creation/with-composite-pk/with-composite-pk.3.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/bulk-loading/after-index-creation/with-composite-pk/with-composite-pk.3.query.sqlpp
new file mode 100644
index 0000000..ec79bc6
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/bulk-loading/after-index-creation/with-composite-pk/with-composite-pk.3.query.sqlpp
@@ -0,0 +1,25 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+SET `compiler.arrayindex` "true";
+
+USE TestYelp;
+
+SELECT C.business_id
+FROM YelpCheckin C, C.dates D
+WHERE "2016-04-26 19:49:16" = D;
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/bulk-loading/after-index-creation/with-filter-fields/with-filter-fields.1.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/bulk-loading/after-index-creation/with-filter-fields/with-filter-fields.1.ddl.sqlpp
new file mode 100644
index 0000000..1de5989
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/bulk-loading/after-index-creation/with-filter-fields/with-filter-fields.1.ddl.sqlpp
@@ -0,0 +1,35 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+/*
+ * Description: Verify the bulk-loading operation for an array index with filter fields.
+ */
+
+DROP DATAVERSE TestYelp IF EXISTS;
+CREATE DATAVERSE TestYelp;
+USE TestYelp;
+
+CREATE TYPE CheckinType AS {
+    checkin_id: smallint,
+    business_id: string,
+    dates: [string]
+};
+
+CREATE DATASET YelpCheckin(CheckinType) PRIMARY KEY checkin_id WITH FILTER ON business_id;
+CREATE INDEX IdxYelpCheckinDates ON YelpCheckin (UNNEST dates);
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/bulk-loading/after-index-creation/with-filter-fields/with-filter-fields.2.update.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/bulk-loading/after-index-creation/with-filter-fields/with-filter-fields.2.update.sqlpp
new file mode 100644
index 0000000..38b50e6
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/bulk-loading/after-index-creation/with-filter-fields/with-filter-fields.2.update.sqlpp
@@ -0,0 +1,24 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+USE TestYelp;
+
+LOAD DATASET YelpCheckin 
+USING localfs (("path"="asterix_nc1://data/yelp-checkin/with-composite-pk.json"),
+                ("format"="json"));
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/bulk-loading/after-index-creation/with-filter-fields/with-filter-fields.3.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/bulk-loading/after-index-creation/with-filter-fields/with-filter-fields.3.query.sqlpp
new file mode 100644
index 0000000..ec79bc6
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/bulk-loading/after-index-creation/with-filter-fields/with-filter-fields.3.query.sqlpp
@@ -0,0 +1,25 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+SET `compiler.arrayindex` "true";
+
+USE TestYelp;
+
+SELECT C.business_id
+FROM YelpCheckin C, C.dates D
+WHERE "2016-04-26 19:49:16" = D;
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/bulk-loading/after-index-creation/with-open-index/with-open-index.1.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/bulk-loading/after-index-creation/with-open-index/with-open-index.1.ddl.sqlpp
new file mode 100644
index 0000000..95d4be1
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/bulk-loading/after-index-creation/with-open-index/with-open-index.1.ddl.sqlpp
@@ -0,0 +1,33 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+/*
+ * Description: Verify the bulk-loading operation for an array->atomic index.
+ */
+
+DROP DATAVERSE TestYelp IF EXISTS;
+CREATE DATAVERSE TestYelp;
+USE TestYelp;
+
+CREATE TYPE CheckinType AS {
+    checkin_id: uuid
+};
+
+CREATE DATASET YelpCheckin(CheckinType) PRIMARY KEY checkin_id AUTOGENERATED;
+CREATE INDEX IdxYelpCheckinDates ON YelpCheckin (UNNEST dates : string ?) ;
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/bulk-loading/after-index-creation/with-open-index/with-open-index.2.update.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/bulk-loading/after-index-creation/with-open-index/with-open-index.2.update.sqlpp
new file mode 100644
index 0000000..f911125
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/bulk-loading/after-index-creation/with-open-index/with-open-index.2.update.sqlpp
@@ -0,0 +1,24 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+USE TestYelp;
+
+LOAD DATASET YelpCheckin 
+USING localfs (("path"="asterix_nc1://data/yelp-checkin/use-case-1.json"),
+                ("format"="json"));
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/bulk-loading/after-index-creation/with-open-index/with-open-index.3.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/bulk-loading/after-index-creation/with-open-index/with-open-index.3.query.sqlpp
new file mode 100644
index 0000000..ec79bc6
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/bulk-loading/after-index-creation/with-open-index/with-open-index.3.query.sqlpp
@@ -0,0 +1,25 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+SET `compiler.arrayindex` "true";
+
+USE TestYelp;
+
+SELECT C.business_id
+FROM YelpCheckin C, C.dates D
+WHERE "2016-04-26 19:49:16" = D;
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/bulk-loading/on-index-creation/closed/use-case-1/use-case-1.1.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/bulk-loading/on-index-creation/closed/use-case-1/use-case-1.1.ddl.sqlpp
new file mode 100644
index 0000000..5241534
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/bulk-loading/on-index-creation/closed/use-case-1/use-case-1.1.ddl.sqlpp
@@ -0,0 +1,34 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+/*
+ * Description: Verify the bulk-loading operation for an array->atomic index.
+ */
+
+DROP DATAVERSE TestYelp IF EXISTS;
+CREATE DATAVERSE TestYelp;
+USE TestYelp;
+
+CREATE TYPE CheckinType AS {
+    checkin_id: uuid,
+    business_id: string,
+    dates: [string]
+};
+
+CREATE DATASET YelpCheckin(CheckinType) PRIMARY KEY checkin_id AUTOGENERATED;
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/bulk-loading/on-index-creation/closed/use-case-1/use-case-1.2.update.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/bulk-loading/on-index-creation/closed/use-case-1/use-case-1.2.update.sqlpp
new file mode 100644
index 0000000..b75bf1b
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/bulk-loading/on-index-creation/closed/use-case-1/use-case-1.2.update.sqlpp
@@ -0,0 +1,276 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+USE TestYelp;
+
+INSERT INTO YelpCheckin (
+    [
+        {
+            "business_id": "--1UhMGODdWsrMastO9DZw",
+            "dates": [
+                "2016-04-26 19:49:16",
+                "2016-08-30 18:36:57",
+                "2016-10-15 02:45:18",
+                "2016-11-18 01:54:50",
+                "2017-04-20 18:39:06",
+                "2017-05-03 17:58:02",
+                "2019-03-19 22:04:48"
+            ]
+        },
+        {
+            "business_id": "--EF5N7P70J_UYBTPypYlA",
+            "dates": [
+                "2018-05-25 19:52:07",
+                "2018-09-18 16:09:44",
+                "2019-10-18 21:29:09"
+            ]
+        },
+        {
+            "business_id": "--Ni3oJ4VOqfOEu7Sj2Vzg",
+            "dates": [
+                "2019-06-07 17:54:58"
+            ]
+        },
+        {
+            "business_id": "--Y1Adl1YUWfYIRSd8vkmA",
+            "dates": [
+                "2011-05-03 20:54:05",
+                "2011-08-23 20:49:45",
+                "2014-12-04 06:13:01",
+                "2016-11-16 19:25:55"
+            ]
+        },
+        {
+            "business_id": "--YPwqIlRJrhHkJcjY3eiA",
+            "dates": [
+                "2016-06-18 21:35:45",
+                "2016-10-15 18:17:51"
+            ]
+        },
+        {
+            "business_id": "--e8PjCNhEz32pprnPhCwQ",
+            "dates": [
+                "2015-04-02 21:45:17"
+            ]
+        },
+        {
+            "business_id": "--kinfHwmtdjz03g8B8z8Q",
+            "dates": [
+                "2014-08-27 17:49:18",
+                "2015-12-19 21:30:31",
+                "2018-11-27 15:53:50"
+            ]
+        },
+        {
+            "business_id": "--q6datkI-f0EoVheXNEeQ",
+            "dates": [
+                "2014-01-28 20:56:04",
+                "2014-11-16 16:11:58",
+                "2015-11-15 19:21:53",
+                "2015-11-15 19:33:39"
+            ]
+        },
+        {
+            "business_id": "--qvQS4MigHPykD2GV0-zw",
+            "dates": [
+                "2019-04-11 18:30:12"
+            ]
+        },
+        {
+            "business_id": "--wIGbLEhlpl_UeAIyDmZQ",
+            "dates": [
+                "2015-06-06 20:01:06",
+                "2019-03-14 22:01:52"
+            ]
+        },
+        {
+            "business_id": "-0FA-Qdi3SPYIoJz9UQw-A",
+            "dates": [
+                "2018-09-29 18:55:17",
+                "2018-10-20 16:48:05",
+                "2018-10-20 22:20:24"
+            ]
+        },
+        {
+            "business_id": "-0Hj1hb_XW6ybWq2M7QhGA",
+            "dates": [
+                "2011-04-23 21:11:22",
+                "2014-05-04 19:42:48",
+                "2014-05-11 19:16:08",
+                "2014-06-04 19:14:18",
+                "2015-12-05 19:22:42",
+                "2017-05-15 23:19:00"
+            ]
+        },
+        {
+            "business_id": "-0KMvRFwDWdVBeTpT11iHw",
+            "dates": [
+                "2012-07-13 21:43:57",
+                "2016-12-24 02:27:31",
+                "2017-08-31 00:35:26"
+            ]
+        },
+        {
+            "business_id": "-0LPtgJC31FWMrMv317p0Q",
+            "dates": [
+                "2013-04-13 12:35:33",
+                "2013-08-19 23:35:49",
+                "2013-10-04 19:14:56"
+            ]
+        },
+        {
+            "business_id": "-0M3o2uWBnQZwd3hmfEwuw",
+            "dates": [
+                "2016-09-10 19:26:19",
+                "2018-09-08 14:15:37",
+                "2019-09-13 22:47:25"
+            ]
+        },
+        {
+            "business_id": "-0RRiWDtfnS16AKCtfvBZg",
+            "dates": [
+                "2017-05-19 14:30:16",
+                "2017-05-19 14:30:25",
+                "2017-08-28 15:49:37",
+                "2017-09-20 20:19:51",
+                "2017-10-01 16:31:05",
+                "2017-10-01 16:56:27",
+                "2017-12-27 23:33:20"
+            ]
+        },
+        {
+            "business_id": "-0Soj75v-XoRcf2ERr8Bmg",
+            "dates": [
+                "2019-06-05 18:22:49"
+            ]
+        },
+        {
+            "business_id": "-0ZumLlFjMh4ZW1z2nXGug",
+            "dates": [
+                "2011-09-24 21:37:32",
+                "2014-03-10 20:20:07",
+                "2015-05-27 00:40:24",
+                "2015-08-29 17:58:15",
+                "2018-03-16 15:03:26"
+            ]
+        },
+        {
+            "business_id": "-0aOudcaAyac0VJbMX-L1g",
+            "dates": [
+                "2015-03-16 23:51:16",
+                "2015-12-21 04:48:01",
+                "2016-10-28 20:22:42",
+                "2016-10-28 20:23:00"
+            ]
+        },
+        {
+            "business_id": "-0b86isaXMY0v4g-V8GZ9Q",
+            "dates": [
+                "2013-10-22 16:49:21",
+                "2014-11-21 17:39:24"
+            ]
+        },
+        {
+            "business_id": "-0d-BfFSU0bwLcnMaGRxYw",
+            "dates": [
+                "2014-08-07 18:30:48",
+                "2014-09-16 20:41:45",
+                "2014-10-12 23:22:27",
+                "2015-07-21 20:43:56",
+                "2015-07-21 20:45:07"
+            ]
+        },
+        {
+            "business_id": "-0jz6c3C6i7RG7Ag22K-Pg",
+            "dates": [
+                "2015-05-02 19:49:05",
+                "2015-05-06 03:52:18",
+                "2015-09-26 01:13:19"
+            ]
+        },
+        {
+            "business_id": "-0y3MZU2oYP8r1ruDP1bfQ",
+            "dates": [
+                "2015-04-11 13:14:14",
+                "2015-11-21 16:05:56",
+                "2016-05-06 14:10:04",
+                "2017-08-09 15:15:10",
+                "2017-10-21 15:12:56"
+            ]
+        },
+        {
+            "business_id": "-1BPe8UjF2_l3nVk-DFUjA",
+            "dates": [
+                "2015-12-03 18:44:00",
+                "2016-03-17 18:19:21",
+                "2016-11-02 15:58:38"
+            ]
+        },
+        {
+            "business_id": "-1E2CQu_38mkghvmZgCCRw",
+            "dates": [
+                "2019-04-04 22:02:37"
+            ]
+        },
+        {
+            "business_id": "-1wzk43IZ5D9Ysu6kzb5xA",
+            "dates": [
+                "2019-02-27 14:03:08"
+            ]
+        },
+        {
+            "business_id": "-23R9P2eG7VTc6DVLjFKzA",
+            "dates": [
+                "2011-12-21 19:02:51",
+                "2012-04-15 04:21:39",
+                "2012-04-15 14:23:56",
+                "2013-06-30 22:39:51",
+                "2013-10-04 20:34:13",
+                "2014-07-16 02:28:40"
+            ]
+        },
+        {
+            "business_id": "-26MGfikhJiTfCI-GqmzhQ",
+            "dates": [
+                "2018-06-13 20:16:07"
+            ]
+        },
+        {
+            "business_id": "-2bLuJsMZ0WhI9daurVQNQ",
+            "dates": [
+                "2015-05-29 16:46:17",
+                "2015-06-01 15:03:53"
+            ]
+        },
+        {
+            "business_id": "-2hDBMaza_ldqnZdiU06LQ",
+            "dates": [
+                "2011-10-08 12:02:23",
+                "2014-08-18 02:11:11",
+                "2016-01-07 05:27:51",
+                "2016-10-21 20:15:55",
+                "2016-12-01 03:57:10",
+                "2016-12-29 01:54:42",
+                "2018-07-22 19:55:31",
+                "2018-09-07 01:42:54",
+                "2019-03-08 03:41:06"
+            ]
+        }
+    ]
+);
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/bulk-loading/on-index-creation/closed/use-case-1/use-case-1.3.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/bulk-loading/on-index-creation/closed/use-case-1/use-case-1.3.ddl.sqlpp
new file mode 100644
index 0000000..dbf055c
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/bulk-loading/on-index-creation/closed/use-case-1/use-case-1.3.ddl.sqlpp
@@ -0,0 +1,24 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+USE TestYelp;
+
+CREATE INDEX IdxYelpCheckinDates ON YelpCheckin (UNNEST dates);
+// CREATE INDEX IdxYelpCheckinDatesBusinessID ON YelpCheckin (UNNEST dates, business_id);
+// CREATE INDEX IdxYelpCheckinBusinessID ON YelpCheckin (business_id, UNNEST dates);
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/bulk-loading/on-index-creation/closed/use-case-1/use-case-1.4.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/bulk-loading/on-index-creation/closed/use-case-1/use-case-1.4.query.sqlpp
new file mode 100644
index 0000000..ec79bc6
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/bulk-loading/on-index-creation/closed/use-case-1/use-case-1.4.query.sqlpp
@@ -0,0 +1,25 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+SET `compiler.arrayindex` "true";
+
+USE TestYelp;
+
+SELECT C.business_id
+FROM YelpCheckin C, C.dates D
+WHERE "2016-04-26 19:49:16" = D;
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/bulk-loading/on-index-creation/closed/use-case-2/use-case-2.1.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/bulk-loading/on-index-creation/closed/use-case-2/use-case-2.1.ddl.sqlpp
new file mode 100644
index 0000000..0b3cfea
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/bulk-loading/on-index-creation/closed/use-case-2/use-case-2.1.ddl.sqlpp
@@ -0,0 +1,37 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+/*
+ * Description: Verify the bulk-loading operation for a record->array->atomic index.
+ */
+
+DROP DATAVERSE TestYelp IF EXISTS;
+CREATE DATAVERSE TestYelp;
+USE TestYelp;
+
+CREATE TYPE CheckinType AS {
+    checkin_id: uuid,
+    business_id: string,
+    checkin_times: {
+        dates: [string],
+        times: [string]
+    }
+};
+
+CREATE DATASET YelpCheckin(CheckinType) PRIMARY KEY checkin_id AUTOGENERATED;
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/bulk-loading/on-index-creation/closed/use-case-2/use-case-2.2.update.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/bulk-loading/on-index-creation/closed/use-case-2/use-case-2.2.update.sqlpp
new file mode 100644
index 0000000..2549e8f
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/bulk-loading/on-index-creation/closed/use-case-2/use-case-2.2.update.sqlpp
@@ -0,0 +1,497 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+USE TestYelp;
+
+INSERT INTO YelpCheckin (
+    [
+        {
+            "business_id": "--1UhMGODdWsrMastO9DZw",
+            "checkin_times": {
+                "dates": [
+                    "2016-04-26",
+                    "2016-08-30",
+                    "2016-10-15",
+                    "2016-11-18",
+                    "2017-04-20",
+                    "2017-05-03",
+                    "2019-03-19"
+                ],
+                "times": [
+                    "19:49:16",
+                    "18:36:57",
+                    "02:45:18",
+                    "01:54:50",
+                    "18:39:06",
+                    "17:58:02",
+                    "22:04:48"
+                ]
+            }
+        },
+        {
+            "business_id": "--EF5N7P70J_UYBTPypYlA",
+            "checkin_times": {
+                "dates": [
+                    "2018-05-25",
+                    "2018-09-18",
+                    "2019-10-18"
+                ],
+                "times": [
+                    "19:52:07",
+                    "16:09:44",
+                    "21:29:09"
+                ]
+            }
+        },
+        {
+            "business_id": "--Ni3oJ4VOqfOEu7Sj2Vzg",
+            "checkin_times": {
+                "dates": [
+                    "2019-06-07"
+                ],
+                "times": [
+                    "17:54:58"
+                ]
+            }
+        },
+        {
+            "business_id": "--Y1Adl1YUWfYIRSd8vkmA",
+            "checkin_times": {
+                "dates": [
+                    "2011-05-03",
+                    "2011-08-23",
+                    "2014-12-04",
+                    "2016-11-16"
+                ],
+                "times": [
+                    "20:54:05",
+                    "20:49:45",
+                    "06:13:01",
+                    "19:25:55"
+                ]
+            }
+        },
+        {
+            "business_id": "--YPwqIlRJrhHkJcjY3eiA",
+            "checkin_times": {
+                "dates": [
+                    "2016-06-18",
+                    "2016-10-15"
+                ],
+                "times": [
+                    "21:35:45",
+                    "18:17:51"
+                ]
+            }
+        },
+        {
+            "business_id": "--e8PjCNhEz32pprnPhCwQ",
+            "checkin_times": {
+                "dates": [
+                    "2015-04-02"
+                ],
+                "times": [
+                    "21:45:17"
+                ]
+            }
+        },
+        {
+            "business_id": "--kinfHwmtdjz03g8B8z8Q",
+            "checkin_times": {
+                "dates": [
+                    "2014-08-27",
+                    "2015-12-19",
+                    "2018-11-27"
+                ],
+                "times": [
+                    "17:49:18",
+                    "21:30:31",
+                    "15:53:50"
+                ]
+            }
+        },
+        {
+            "business_id": "--q6datkI-f0EoVheXNEeQ",
+            "checkin_times": {
+                "dates": [
+                    "2014-01-28",
+                    "2014-11-16",
+                    "2015-11-15",
+                    "2015-11-15"
+                ],
+                "times": [
+                    "20:56:04",
+                    "16:11:58",
+                    "19:21:53",
+                    "19:33:39"
+                ]
+            }
+        },
+        {
+            "business_id": "--qvQS4MigHPykD2GV0-zw",
+            "checkin_times": {
+                "dates": [
+                    "2019-04-11"
+                ],
+                "times": [
+                    "18:30:12"
+                ]
+            }
+        },
+        {
+            "business_id": "--wIGbLEhlpl_UeAIyDmZQ",
+            "checkin_times": {
+                "dates": [
+                    "2015-06-06",
+                    "2019-03-14"
+                ],
+                "times": [
+                    "20:01:06",
+                    "22:01:52"
+                ]
+            }
+        },
+        {
+            "business_id": "-0FA-Qdi3SPYIoJz9UQw-A",
+            "checkin_times": {
+                "dates": [
+                    "2018-09-29",
+                    "2018-10-20",
+                    "2018-10-20"
+                ],
+                "times": [
+                    "18:55:17",
+                    "16:48:05",
+                    "22:20:24"
+                ]
+            }
+        },
+        {
+            "business_id": "-0Hj1hb_XW6ybWq2M7QhGA",
+            "checkin_times": {
+                "dates": [
+                    "2011-04-23",
+                    "2014-05-04",
+                    "2014-05-11",
+                    "2014-06-04",
+                    "2015-12-05",
+                    "2017-05-15"
+                ],
+                "times": [
+                    "21:11:22",
+                    "19:42:48",
+                    "19:16:08",
+                    "19:14:18",
+                    "19:22:42",
+                    "23:19:00"
+                ]
+            }
+        },
+        {
+            "business_id": "-0KMvRFwDWdVBeTpT11iHw",
+            "checkin_times": {
+                "dates": [
+                    "2012-07-13",
+                    "2016-12-24",
+                    "2017-08-31"
+                ],
+                "times": [
+                    "21:43:57",
+                    "02:27:31",
+                    "00:35:26"
+                ]
+            }
+        },
+        {
+            "business_id": "-0LPtgJC31FWMrMv317p0Q",
+            "checkin_times": {
+                "dates": [
+                    "2013-04-13",
+                    "2013-08-19",
+                    "2013-10-04"
+                ],
+                "times": [
+                    "12:35:33",
+                    "23:35:49",
+                    "19:14:56"
+                ]
+            }
+        },
+        {
+            "business_id": "-0M3o2uWBnQZwd3hmfEwuw",
+            "checkin_times": {
+                "dates": [
+                    "2016-09-10",
+                    "2018-09-08",
+                    "2019-09-13"
+                ],
+                "times": [
+                    "19:26:19",
+                    "14:15:37",
+                    "22:47:25"
+                ]
+            }
+        },
+        {
+            "business_id": "-0RRiWDtfnS16AKCtfvBZg",
+            "checkin_times": {
+                "dates": [
+                    "2017-05-19",
+                    "2017-05-19",
+                    "2017-08-28",
+                    "2017-09-20",
+                    "2017-10-01",
+                    "2017-10-01",
+                    "2017-12-27"
+                ],
+                "times": [
+                    "14:30:16",
+                    "14:30:25",
+                    "15:49:37",
+                    "20:19:51",
+                    "16:31:05",
+                    "16:56:27",
+                    "23:33:20"
+                ]
+            }
+        },
+        {
+            "business_id": "-0Soj75v-XoRcf2ERr8Bmg",
+            "checkin_times": {
+                "dates": [
+                    "2019-06-05"
+                ],
+                "times": [
+                    "18:22:49"
+                ]
+            }
+        },
+        {
+            "business_id": "-0ZumLlFjMh4ZW1z2nXGug",
+            "checkin_times": {
+                "dates": [
+                    "2011-09-24",
+                    "2014-03-10",
+                    "2015-05-27",
+                    "2015-08-29",
+                    "2018-03-16"
+                ],
+                "times": [
+                    "21:37:32",
+                    "20:20:07",
+                    "00:40:24",
+                    "17:58:15",
+                    "15:03:26"
+                ]
+            }
+        },
+        {
+            "business_id": "-0aOudcaAyac0VJbMX-L1g",
+            "checkin_times": {
+                "dates": [
+                    "2015-03-16",
+                    "2015-12-21",
+                    "2016-10-28",
+                    "2016-10-28"
+                ],
+                "times": [
+                    "23:51:16",
+                    "04:48:01",
+                    "20:22:42",
+                    "20:23:00"
+                ]
+            }
+        },
+        {
+            "business_id": "-0b86isaXMY0v4g-V8GZ9Q",
+            "checkin_times": {
+                "dates": [
+                    "2013-10-22",
+                    "2014-11-21"
+                ],
+                "times": [
+                    "16:49:21",
+                    "17:39:24"
+                ]
+            }
+        },
+        {
+            "business_id": "-0d-BfFSU0bwLcnMaGRxYw",
+            "checkin_times": {
+                "dates": [
+                    "2014-08-07",
+                    "2014-09-16",
+                    "2014-10-12",
+                    "2015-07-21",
+                    "2015-07-21"
+                ],
+                "times": [
+                    "18:30:48",
+                    "20:41:45",
+                    "23:22:27",
+                    "20:43:56",
+                    "20:45:07"
+                ]
+            }
+        },
+        {
+            "business_id": "-0jz6c3C6i7RG7Ag22K-Pg",
+            "checkin_times": {
+                "dates": [
+                    "2015-05-02",
+                    "2015-05-06",
+                    "2015-09-26"
+                ],
+                "times": [
+                    "19:49:05",
+                    "03:52:18",
+                    "01:13:19"
+                ]
+            }
+        },
+        {
+            "business_id": "-0y3MZU2oYP8r1ruDP1bfQ",
+            "checkin_times": {
+                "dates": [
+                    "2015-04-11",
+                    "2015-11-21",
+                    "2016-05-06",
+                    "2017-08-09",
+                    "2017-10-21"
+                ],
+                "times": [
+                    "13:14:14",
+                    "16:05:56",
+                    "14:10:04",
+                    "15:15:10",
+                    "15:12:56"
+                ]
+            }
+        },
+        {
+            "business_id": "-1BPe8UjF2_l3nVk-DFUjA",
+            "checkin_times": {
+                "dates": [
+                    "2015-12-03",
+                    "2016-03-17",
+                    "2016-11-02"
+                ],
+                "times": [
+                    "18:44:00",
+                    "18:19:21",
+                    "15:58:38"
+                ]
+            }
+        },
+        {
+            "business_id": "-1E2CQu_38mkghvmZgCCRw",
+            "checkin_times": {
+                "dates": [
+                    "2019-04-04"
+                ],
+                "times": [
+                    "22:02:37"
+                ]
+            }
+        },
+        {
+            "business_id": "-1wzk43IZ5D9Ysu6kzb5xA",
+            "checkin_times": {
+                "dates": [
+                    "2019-02-27"
+                ],
+                "times": [
+                    "14:03:08"
+                ]
+            }
+        },
+        {
+            "business_id": "-23R9P2eG7VTc6DVLjFKzA",
+            "checkin_times": {
+                "dates": [
+                    "2011-12-21",
+                    "2012-04-15",
+                    "2012-04-15",
+                    "2013-06-30",
+                    "2013-10-04",
+                    "2014-07-16"
+                ],
+                "times": [
+                    "19:02:51",
+                    "04:21:39",
+                    "14:23:56",
+                    "22:39:51",
+                    "20:34:13",
+                    "02:28:40"
+                ]
+            }
+        },
+        {
+            "business_id": "-26MGfikhJiTfCI-GqmzhQ",
+            "checkin_times": {
+                "dates": [
+                    "2018-06-13"
+                ],
+                "times": [
+                    "20:16:07"
+                ]
+            }
+        },
+        {
+            "business_id": "-2bLuJsMZ0WhI9daurVQNQ",
+            "checkin_times": {
+                "dates": [
+                    "2015-05-29",
+                    "2015-06-01"
+                ],
+                "times": [
+                    "16:46:17",
+                    "15:03:53"
+                ]
+            }
+        },
+        {
+            "business_id": "-2hDBMaza_ldqnZdiU06LQ",
+            "checkin_times": {
+                "dates": [
+                    "2011-10-08",
+                    "2014-08-18",
+                    "2016-01-07",
+                    "2016-10-21",
+                    "2016-12-01",
+                    "2016-12-29",
+                    "2018-07-22",
+                    "2018-09-07",
+                    "2019-03-08"
+                ],
+                "times": [
+                    "12:02:23",
+                    "02:11:11",
+                    "05:27:51",
+                    "20:15:55",
+                    "03:57:10",
+                    "01:54:42",
+                    "19:55:31",
+                    "01:42:54",
+                    "03:41:06"
+                ]
+            }
+        }
+    ]
+);
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/bulk-loading/on-index-creation/closed/use-case-2/use-case-2.3.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/bulk-loading/on-index-creation/closed/use-case-2/use-case-2.3.ddl.sqlpp
new file mode 100644
index 0000000..0452c4a
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/bulk-loading/on-index-creation/closed/use-case-2/use-case-2.3.ddl.sqlpp
@@ -0,0 +1,27 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+USE TestYelp;
+
+CREATE INDEX IdxYelpCheckinDates ON YelpCheckin (UNNEST checkin_times.dates);
+
+/* It would be more user-friendly to use (UNNEST checkin_times.dates), (business_id), but we are able to infer this
+   from our grammar alone. We leave this in to also exercise our parser. */
+// CREATE INDEX IdxYelpCheckinDatesBusiness ON YelpCheckin (UNNEST checkin_times.dates, business_id);
+// CREATE INDEX IdxYelpCheckinBusinessDates ON YelpCheckin (business_id, UNNEST checkin_times.dates);
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/bulk-loading/on-index-creation/closed/use-case-2/use-case-2.4.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/bulk-loading/on-index-creation/closed/use-case-2/use-case-2.4.query.sqlpp
new file mode 100644
index 0000000..0f23cd4
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/bulk-loading/on-index-creation/closed/use-case-2/use-case-2.4.query.sqlpp
@@ -0,0 +1,25 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+SET `compiler.arrayindex` "true";
+
+USE TestYelp;
+
+SELECT C.business_id
+FROM YelpCheckin C, C.checkin_times.dates D
+WHERE "2016-04-26" = D;
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/bulk-loading/on-index-creation/closed/use-case-3/use-case-3.1.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/bulk-loading/on-index-creation/closed/use-case-3/use-case-3.1.ddl.sqlpp
new file mode 100644
index 0000000..d820ff8
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/bulk-loading/on-index-creation/closed/use-case-3/use-case-3.1.ddl.sqlpp
@@ -0,0 +1,37 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+/*
+ * Description: Verify the bulk-loading operation for an array->record->atomic index, as well as a composite index on both fields. 
+ */
+
+DROP DATAVERSE TestYelp IF EXISTS;
+CREATE DATAVERSE TestYelp;
+USE TestYelp;
+
+CREATE TYPE CheckinType AS {
+    checkin_id: uuid,
+    business_id: string,
+    checkin_times: [{
+        date: string,
+        time: string
+    }]
+};
+
+CREATE DATASET YelpCheckin(CheckinType) PRIMARY KEY checkin_id AUTOGENERATED;
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/bulk-loading/on-index-creation/closed/use-case-3/use-case-3.2.update.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/bulk-loading/on-index-creation/closed/use-case-3/use-case-3.2.update.sqlpp
new file mode 100644
index 0000000..02ddf21
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/bulk-loading/on-index-creation/closed/use-case-3/use-case-3.2.update.sqlpp
@@ -0,0 +1,276 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+USE TestYelp;
+
+INSERT INTO YelpCheckin (
+    [
+        {
+            "business_id": "--1UhMGODdWsrMastO9DZw",
+            "checkin_times": [ 
+                {"date": "2016-04-26", "time": "19:49:16"},
+                {"date": "2016-08-30", "time": "18:36:57"},
+                {"date": "2016-10-15", "time": "02:45:18"},
+                {"date": "2016-11-18", "time": "01:54:50"},
+                {"date": "2017-04-20", "time": "18:39:06"},
+                {"date": "2017-05-03", "time": "17:58:02"},
+                {"date": "2019-03-19", "time": "22:04:48"}
+             ]
+        },
+        {
+            "business_id": "--EF5N7P70J_UYBTPypYlA",
+            "checkin_times": [ 
+                {"date": "2018-05-25", "time": "19:52:07"},
+                {"date": "2018-09-18", "time": "16:09:44"},
+                {"date": "2019-10-18", "time": "21:29:09"}
+             ]
+        },
+        {
+            "business_id": "--Ni3oJ4VOqfOEu7Sj2Vzg",
+            "checkin_times": [ 
+                {"date": "2019-06-07", "time": "17:54:58"}
+             ]
+        },
+        {
+            "business_id": "--Y1Adl1YUWfYIRSd8vkmA",
+            "checkin_times": [ 
+                {"date": "2011-05-03", "time": "20:54:05"},
+                {"date": "2011-08-23", "time": "20:49:45"},
+                {"date": "2014-12-04", "time": "06:13:01"},
+                {"date": "2016-11-16", "time": "19:25:55"}
+             ]
+        },
+        {
+            "business_id": "--YPwqIlRJrhHkJcjY3eiA",
+            "checkin_times": [ 
+                {"date": "2016-06-18", "time": "21:35:45"},
+                {"date": "2016-10-15", "time": "18:17:51"}
+             ]
+        },
+        {
+            "business_id": "--e8PjCNhEz32pprnPhCwQ",
+            "checkin_times": [ 
+                {"date": "2015-04-02", "time": "21:45:17"}
+             ]
+        },
+        {
+            "business_id": "--kinfHwmtdjz03g8B8z8Q",
+            "checkin_times": [ 
+                {"date": "2014-08-27", "time": "17:49:18"},
+                {"date": "2015-12-19", "time": "21:30:31"},
+                {"date": "2018-11-27", "time": "15:53:50"}
+             ]
+        },
+        {
+            "business_id": "--q6datkI-f0EoVheXNEeQ",
+            "checkin_times": [ 
+                {"date": "2014-01-28", "time": "20:56:04"},
+                {"date": "2014-11-16", "time": "16:11:58"},
+                {"date": "2015-11-15", "time": "19:21:53"},
+                {"date": "2015-11-15", "time": "19:33:39"}
+             ]
+        },
+        {
+            "business_id": "--qvQS4MigHPykD2GV0-zw",
+            "checkin_times": [ 
+                {"date": "2019-04-11", "time": "18:30:12"}
+             ]
+        },
+        {
+            "business_id": "--wIGbLEhlpl_UeAIyDmZQ",
+            "checkin_times": [ 
+                {"date": "2015-06-06", "time": "20:01:06"},
+                {"date": "2019-03-14", "time": "22:01:52"}
+             ]
+        },
+        {
+            "business_id": "-0FA-Qdi3SPYIoJz9UQw-A",
+            "checkin_times": [ 
+                {"date": "2018-09-29", "time": "18:55:17"},
+                {"date": "2018-10-20", "time": "16:48:05"},
+                {"date": "2018-10-20", "time": "22:20:24"}
+             ]
+        },
+        {
+            "business_id": "-0Hj1hb_XW6ybWq2M7QhGA",
+            "checkin_times": [ 
+                {"date": "2011-04-23", "time": "21:11:22"},
+                {"date": "2014-05-04", "time": "19:42:48"},
+                {"date": "2014-05-11", "time": "19:16:08"},
+                {"date": "2014-06-04", "time": "19:14:18"},
+                {"date": "2015-12-05", "time": "19:22:42"},
+                {"date": "2017-05-15", "time": "23:19:00"}
+             ]
+        },
+        {
+            "business_id": "-0KMvRFwDWdVBeTpT11iHw",
+            "checkin_times": [ 
+                {"date": "2012-07-13", "time": "21:43:57"},
+                {"date": "2016-12-24", "time": "02:27:31"},
+                {"date": "2017-08-31", "time": "00:35:26"}
+             ]
+        },
+        {
+            "business_id": "-0LPtgJC31FWMrMv317p0Q",
+            "checkin_times": [ 
+                {"date": "2013-04-13", "time": "12:35:33"},
+                {"date": "2013-08-19", "time": "23:35:49"},
+                {"date": "2013-10-04", "time": "19:14:56"}
+             ]
+        },
+        {
+            "business_id": "-0M3o2uWBnQZwd3hmfEwuw",
+            "checkin_times": [ 
+                {"date": "2016-09-10", "time": "19:26:19"},
+                {"date": "2018-09-08", "time": "14:15:37"},
+                {"date": "2019-09-13", "time": "22:47:25"}
+             ]
+        },
+        {
+            "business_id": "-0RRiWDtfnS16AKCtfvBZg",
+            "checkin_times": [ 
+                {"date": "2017-05-19", "time": "14:30:16"},
+                {"date": "2017-05-19", "time": "14:30:25"},
+                {"date": "2017-08-28", "time": "15:49:37"},
+                {"date": "2017-09-20", "time": "20:19:51"},
+                {"date": "2017-10-01", "time": "16:31:05"},
+                {"date": "2017-10-01", "time": "16:56:27"},
+                {"date": "2017-12-27", "time": "23:33:20"}
+             ]
+        },
+        {
+            "business_id": "-0Soj75v-XoRcf2ERr8Bmg",
+            "checkin_times": [ 
+                {"date": "2019-06-05", "time": "18:22:49"}
+             ]
+        },
+        {
+            "business_id": "-0ZumLlFjMh4ZW1z2nXGug",
+            "checkin_times": [ 
+                {"date": "2011-09-24", "time": "21:37:32"},
+                {"date": "2014-03-10", "time": "20:20:07"},
+                {"date": "2015-05-27", "time": "00:40:24"},
+                {"date": "2015-08-29", "time": "17:58:15"},
+                {"date": "2018-03-16", "time": "15:03:26"}
+             ]
+        },
+        {
+            "business_id": "-0aOudcaAyac0VJbMX-L1g",
+            "checkin_times": [ 
+                {"date": "2015-03-16", "time": "23:51:16"},
+                {"date": "2015-12-21", "time": "04:48:01"},
+                {"date": "2016-10-28", "time": "20:22:42"},
+                {"date": "2016-10-28", "time": "20:23:00"}
+             ]
+        },
+        {
+            "business_id": "-0b86isaXMY0v4g-V8GZ9Q",
+            "checkin_times": [ 
+                {"date": "2013-10-22", "time": "16:49:21"},
+                {"date": "2014-11-21", "time": "17:39:24"}
+             ]
+        },
+        {
+            "business_id": "-0d-BfFSU0bwLcnMaGRxYw",
+            "checkin_times": [ 
+                {"date": "2014-08-07", "time": "18:30:48"},
+                {"date": "2014-09-16", "time": "20:41:45"},
+                {"date": "2014-10-12", "time": "23:22:27"},
+                {"date": "2015-07-21", "time": "20:43:56"},
+                {"date": "2015-07-21", "time": "20:45:07"}
+             ]
+        },
+        {
+            "business_id": "-0jz6c3C6i7RG7Ag22K-Pg",
+            "checkin_times": [ 
+                {"date": "2015-05-02", "time": "19:49:05"},
+                {"date": "2015-05-06", "time": "03:52:18"},
+                {"date": "2015-09-26", "time": "01:13:19"}
+             ]
+        },
+        {
+            "business_id": "-0y3MZU2oYP8r1ruDP1bfQ",
+            "checkin_times": [ 
+                {"date": "2015-04-11", "time": "13:14:14"},
+                {"date": "2015-11-21", "time": "16:05:56"},
+                {"date": "2016-05-06", "time": "14:10:04"},
+                {"date": "2017-08-09", "time": "15:15:10"},
+                {"date": "2017-10-21", "time": "15:12:56"}
+             ]
+        },
+        {
+            "business_id": "-1BPe8UjF2_l3nVk-DFUjA",
+            "checkin_times": [ 
+                {"date": "2015-12-03", "time": "18:44:00"},
+                {"date": "2016-03-17", "time": "18:19:21"},
+                {"date": "2016-11-02", "time": "15:58:38"}
+             ]
+        },
+        {
+            "business_id": "-1E2CQu_38mkghvmZgCCRw",
+            "checkin_times": [ 
+                {"date": "2019-04-04", "time": "22:02:37"}
+             ]
+        },
+        {
+            "business_id": "-1wzk43IZ5D9Ysu6kzb5xA",
+            "checkin_times": [ 
+                {"date": "2019-02-27", "time": "14:03:08"}
+             ]
+        },
+        {
+            "business_id": "-23R9P2eG7VTc6DVLjFKzA",
+            "checkin_times": [ 
+                {"date": "2011-12-21", "time": "19:02:51"},
+                {"date": "2012-04-15", "time": "04:21:39"},
+                {"date": "2012-04-15", "time": "14:23:56"},
+                {"date": "2013-06-30", "time": "22:39:51"},
+                {"date": "2013-10-04", "time": "20:34:13"},
+                {"date": "2014-07-16", "time": "02:28:40"}
+             ]
+        },
+        {
+            "business_id": "-26MGfikhJiTfCI-GqmzhQ",
+            "checkin_times": [ 
+                {"date": "2018-06-13", "time": "20:16:07"}
+             ]
+        },
+        {
+            "business_id": "-2bLuJsMZ0WhI9daurVQNQ",
+            "checkin_times": [ 
+                {"date": "2015-05-29", "time": "16:46:17"},
+                {"date": "2015-06-01", "time": "15:03:53"}
+             ]
+        },
+        {
+            "business_id": "-2hDBMaza_ldqnZdiU06LQ",
+            "checkin_times": [ 
+                {"date": "2011-10-08", "time": "12:02:23"},
+                {"date": "2014-08-18", "time": "02:11:11"},
+                {"date": "2016-01-07", "time": "05:27:51"},
+                {"date": "2016-10-21", "time": "20:15:55"},
+                {"date": "2016-12-01", "time": "03:57:10"},
+                {"date": "2016-12-29", "time": "01:54:42"},
+                {"date": "2018-07-22", "time": "19:55:31"},
+                {"date": "2018-09-07", "time": "01:42:54"},
+                {"date": "2019-03-08", "time": "03:41:06"}
+             ]
+        }
+    ]
+);
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/bulk-loading/on-index-creation/closed/use-case-3/use-case-3.3.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/bulk-loading/on-index-creation/closed/use-case-3/use-case-3.3.ddl.sqlpp
new file mode 100644
index 0000000..ee911b4
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/bulk-loading/on-index-creation/closed/use-case-3/use-case-3.3.ddl.sqlpp
@@ -0,0 +1,22 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+USE TestYelp;
+
+CREATE INDEX IdxYelpCheckinDates ON YelpCheckin (UNNEST checkin_times SELECT date);
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/bulk-loading/on-index-creation/closed/use-case-3/use-case-3.4.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/bulk-loading/on-index-creation/closed/use-case-3/use-case-3.4.query.sqlpp
new file mode 100644
index 0000000..27f01aa
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/bulk-loading/on-index-creation/closed/use-case-3/use-case-3.4.query.sqlpp
@@ -0,0 +1,25 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+SET `compiler.arrayindex` "true";
+
+USE TestYelp;
+
+SELECT C.business_id
+FROM YelpCheckin C, C.checkin_times CT
+WHERE "2016-04-26" = CT.date;
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/bulk-loading/on-index-creation/closed/use-case-3/use-case-3.5.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/bulk-loading/on-index-creation/closed/use-case-3/use-case-3.5.ddl.sqlpp
new file mode 100644
index 0000000..ad50985
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/bulk-loading/on-index-creation/closed/use-case-3/use-case-3.5.ddl.sqlpp
@@ -0,0 +1,25 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+USE TestYelp;
+
+DROP INDEX YelpCheckin.IdxYelpCheckinDates;
+CREATE INDEX IdxYelpCheckinDatesTimes ON YelpCheckin (UNNEST checkin_times SELECT date, time);
+// CREATE INDEX IdxYelpCheckinDatesTimesBusiness ON YelpCheckin ((UNNEST checkin_times SELECT date, time), (business_id));
+// CREATE INDEX IdxYelpCheckinBusinessDatesTimes ON YelpCheckin (business_id, UNNEST checkin_times SELECT date, time);
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/bulk-loading/on-index-creation/closed/use-case-3/use-case-3.6.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/bulk-loading/on-index-creation/closed/use-case-3/use-case-3.6.query.sqlpp
new file mode 100644
index 0000000..27f01aa
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/bulk-loading/on-index-creation/closed/use-case-3/use-case-3.6.query.sqlpp
@@ -0,0 +1,25 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+SET `compiler.arrayindex` "true";
+
+USE TestYelp;
+
+SELECT C.business_id
+FROM YelpCheckin C, C.checkin_times CT
+WHERE "2016-04-26" = CT.date;
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/bulk-loading/on-index-creation/closed/use-case-4/use-case-4.1.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/bulk-loading/on-index-creation/closed/use-case-4/use-case-4.1.ddl.sqlpp
new file mode 100644
index 0000000..f42b348
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/bulk-loading/on-index-creation/closed/use-case-4/use-case-4.1.ddl.sqlpp
@@ -0,0 +1,37 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+/*
+ * Description: Verify the bulk-loading operation for an array->record->array->atomic index.
+ */
+
+DROP DATAVERSE TestYelp IF EXISTS;
+CREATE DATAVERSE TestYelp;
+USE TestYelp;
+
+CREATE TYPE CheckinType AS {
+    checkin_id: uuid,
+    business_id: string,
+    checkin_times: [{
+        dates: [string],
+        times: [string]
+    }]
+};
+
+CREATE DATASET YelpCheckin(CheckinType) PRIMARY KEY checkin_id AUTOGENERATED;
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/bulk-loading/on-index-creation/closed/use-case-4/use-case-4.2.update.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/bulk-loading/on-index-creation/closed/use-case-4/use-case-4.2.update.sqlpp
new file mode 100644
index 0000000..7fcf9e4
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/bulk-loading/on-index-creation/closed/use-case-4/use-case-4.2.update.sqlpp
@@ -0,0 +1,425 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+USE TestYelp;
+
+INSERT INTO YelpCheckin (
+    [
+        {
+            "business_id": "--1UhMGODdWsrMastO9DZw",
+            "checkin_times": [ 
+                {
+                    "dates": ["2016-04-26", "2016-08-30", "2016-10-15", "2016-11-18"],
+                    "times": ["19:49:16", "18:36:57", "02:45:18", "01:54:50"]
+                },
+                {
+                    "dates": ["2017-04-20", "2017-05-03"],
+                    "times": ["18:39:06", "17:58:02"]
+                },
+                {
+                    "dates": ["2019-03-19"],
+                    "times": ["22:04:48"]
+                }
+             ]
+        },
+        {
+            "business_id": "--EF5N7P70J_UYBTPypYlA",
+            "checkin_times": [ 
+                {
+                    "dates": ["2018-05-25", "2018-09-18"],
+                    "times": ["19:52:07", "16:09:44"]
+                },
+                {
+                    "dates": ["2019-10-18"],
+                    "times": ["21:29:09"]
+                }
+             ]
+        },
+        {
+            "business_id": "--Ni3oJ4VOqfOEu7Sj2Vzg",
+            "checkin_times": [ 
+                {
+                    "dates": ["2019-06-07"],
+                    "times": ["17:54:58"]
+                }
+             ]
+        },
+        {
+            "business_id": "--Y1Adl1YUWfYIRSd8vkmA",
+            "checkin_times": [ 
+                {
+                    "dates": ["2011-05-03", "2011-08-23"],
+                    "times": ["20:54:05", "20:49:45"]
+                },
+                {
+                    "dates": ["2014-12-04"],
+                    "times": ["06:13:01"]
+                },
+                {
+                    "dates": ["2016-11-16"],
+                    "times": ["19:25:55"]
+                }
+             ]
+        },
+        {
+            "business_id": "--YPwqIlRJrhHkJcjY3eiA",
+            "checkin_times": [ 
+               {
+                    "dates": ["2016-06-18", "2016-10-15"],
+                    "times": ["21:35:45", "18:17:51"]
+                }
+             ]
+        },
+        {
+            "business_id": "--e8PjCNhEz32pprnPhCwQ",
+            "checkin_times": [ 
+                {
+                    "dates": ["2015-04-02"],
+                    "times": ["21:45:17"]
+                }
+             ]
+        },
+        {
+            "business_id": "--kinfHwmtdjz03g8B8z8Q",
+            "checkin_times": [ 
+                {
+                    "dates": ["2014-08-27"],
+                    "times": ["17:49:18"]
+                },
+                {
+                    "dates": ["2015-12-19"],
+                    "times": ["21:30:31"]
+                },
+                {
+                    "dates": ["2018-11-27"],
+                    "times": ["15:53:50"]
+                }
+             ]
+        },
+        {
+            "business_id": "--q6datkI-f0EoVheXNEeQ",
+            "checkin_times": [ 
+                {
+                    "dates": ["2014-01-28", "2014-11-16"],
+                    "times": ["20:56:04", "16:11:58"]
+                },
+                {
+                    "dates": ["2015-11-15", "2015-11-15"],
+                    "times": ["19:21:53", "19:33:39"]
+                }
+             ]
+        },
+        {
+            "business_id": "--qvQS4MigHPykD2GV0-zw",
+            "checkin_times": [ 
+                {
+                    "dates": ["2019-04-11"],
+                    "times": ["18:30:12"]
+                }
+             ]
+        },
+        {
+            "business_id": "--wIGbLEhlpl_UeAIyDmZQ",
+            "checkin_times": [ 
+                {
+                    "dates": ["2015-06-06"],
+                    "times": ["20:01:06"]
+                },
+                {
+                    "dates": ["2019-03-14"],
+                    "times": ["22:01:52"]
+                }
+             ]
+        },
+        {
+            "business_id": "-0FA-Qdi3SPYIoJz9UQw-A",
+            "checkin_times": [ 
+                {
+                    "dates": ["2018-09-29", "2018-10-20", "2018-10-20"],
+                    "times": ["18:55:17", "16:48:05", "22:20:24"]
+                }
+             ]
+        },
+        {
+            "business_id": "-0Hj1hb_XW6ybWq2M7QhGA",
+            "checkin_times": [ 
+                {
+                    "dates": ["2011-04-23"],
+                    "times": ["21:11:22"]
+                },
+                {
+                    "dates": ["2014-05-04", "2014-05-11", "2014-06-04"],
+                    "times": ["19:42:48", "19:16:08", "19:14:18"]
+                },
+                {
+                    "dates": ["2015-12-05"],
+                    "times": ["19:22:42"]
+                },
+                {
+                    "dates": ["2017-05-15"],
+                    "times": ["23:19:00"]
+                }
+             ]
+        },
+        {
+            "business_id": "-0KMvRFwDWdVBeTpT11iHw",
+            "checkin_times": [ 
+                {
+                    "dates": ["2012-07-13"],
+                    "times": ["21:43:57"]
+                },
+                {
+                    "dates": ["2016-12-24"],
+                    "times": ["02:27:31"]
+                },
+                {
+                    "dates": ["2017-08-31"],
+                    "times": ["00:35:26"]
+                }
+             ]
+        },
+        {
+            "business_id": "-0LPtgJC31FWMrMv317p0Q",
+            "checkin_times": [ 
+                {
+                    "dates": ["2013-04-13", "2013-08-19", "2013-10-04"],
+                    "times": ["12:35:33", "23:35:49", "19:14:56"]
+                }
+
+             ]
+        },
+        {
+            "business_id": "-0M3o2uWBnQZwd3hmfEwuw",
+            "checkin_times": [ 
+                {
+                    "dates": ["2016-09-10"],
+                    "times": ["19:26:19"]
+                },
+                {
+                    "dates": ["2018-09-08"],
+                    "times": ["14:15:37"]
+                },
+                {
+                    "dates": ["2019-09-13"],
+                    "times": ["22:47:25"]
+                }
+             ]
+        },
+        {
+            "business_id": "-0RRiWDtfnS16AKCtfvBZg",
+            "checkin_times": [ 
+                 {
+                    "dates": ["2017-05-19", "2017-05-19", "2017-08-28", "2017-09-20", "2017-10-01", "2017-10-01", "2017-12-27"],
+                    "times": ["14:30:16", "14:30:25", "15:49:37", "20:19:51", "16:31:05", "16:56:27", "23:33:20"]
+                }
+             ]
+        },
+        {
+            "business_id": "-0Soj75v-XoRcf2ERr8Bmg",
+            "checkin_times": [ 
+                {
+                    "dates": ["2019-06-05"],
+                    "times": ["18:22:49"]
+                }
+             ]
+        },
+        {
+            "business_id": "-0ZumLlFjMh4ZW1z2nXGug",
+            "checkin_times": [ 
+                {
+                    "dates": ["2011-09-24"],
+                    "times": ["21:37:32"]
+                },
+                {
+                    "dates": ["2014-03-10"],
+                    "times": ["20:20:07"]
+                },
+                {
+                    "dates": ["2015-05-27", "2015-08-29"],
+                    "times": ["00:40:24", "17:58:15"]
+                },
+                {
+                    "dates": ["2018-03-16"],
+                    "times": ["15:03:26"]
+                }
+             ]
+        },
+        {
+            "business_id": "-0aOudcaAyac0VJbMX-L1g",
+            "checkin_times": [ 
+                {
+                    "dates": ["2015-03-16", "2015-12-21"],
+                    "times": ["23:51:16", "04:48:01"]
+                },
+                {
+                    "dates": ["2016-10-28", "2016-10-28"],
+                    "times": ["20:22:42", "20:23:00"]
+                }
+             ]
+        },
+        {
+            "business_id": "-0b86isaXMY0v4g-V8GZ9Q",
+            "checkin_times": [ 
+                {
+                    "dates": ["2013-10-22"],
+                    "times": ["16:49:21"]
+                },
+                {
+                    "dates": ["2014-11-21"],
+                    "times": ["17:39:24"]
+                }
+             ]
+        },
+        {
+            "business_id": "-0d-BfFSU0bwLcnMaGRxYw",
+            "checkin_times": [ 
+                {
+                    "dates": ["2014-08-07", "2014-09-16", "2014-10-12"],
+                    "times": ["18:30:48", "20:41:45", "23:22:27"]
+                },
+                {
+                    "dates": ["2015-07-21", "2015-07-21"],
+                    "times": ["20:43:56", "20:45:07"]
+                }
+             ]
+        },
+        {
+            "business_id": "-0jz6c3C6i7RG7Ag22K-Pg",
+            "checkin_times": [ 
+                {
+                    "dates": ["2015-05-02", "2015-05-06", "2015-09-26"],
+                    "times": ["19:49:05", "03:52:18", "01:13:19"]
+                }
+
+             ]
+        },
+        {
+            "business_id": "-0y3MZU2oYP8r1ruDP1bfQ",
+            "checkin_times": [ 
+                {
+                    "dates": ["2015-04-11", "2015-11-21"],
+                    "times": ["13:14:14", "16:05:56"]
+                },
+                {
+                    "dates": ["2016-05-06"],
+                    "times": ["14:10:04"]
+                },
+                {
+                    "dates": ["2017-08-09", "2017-10-21"],
+                    "times": ["15:15:10", "15:12:56"]
+                }
+             ]
+        },
+        {
+            "business_id": "-1BPe8UjF2_l3nVk-DFUjA",
+            "checkin_times": [ 
+                {
+                    "dates": ["2015-12-03"],
+                    "times": ["18:44:00"]
+                },
+                {
+                    "dates": ["2016-03-17", "2016-11-02"],
+                    "times": ["18:19:21", "15:58:38"]
+                }
+             ]
+        },
+        {
+            "business_id": "-1E2CQu_38mkghvmZgCCRw",
+            "checkin_times": [ 
+                {
+                    "dates": ["2019-04-04"],
+                    "times": ["22:02:37"]
+                }
+             ]
+        },
+        {
+            "business_id": "-1wzk43IZ5D9Ysu6kzb5xA",
+            "checkin_times": [ 
+                {
+                    "dates": ["2019-02-27"],
+                    "times": ["14:03:08"]
+                }
+             ]
+        },
+        {
+            "business_id": "-23R9P2eG7VTc6DVLjFKzA",
+            "checkin_times": [ 
+                {
+                    "dates": ["2011-12-21"],
+                    "times": ["19:02:51"]
+                },
+                {
+                    "dates": ["2012-04-15", "2012-04-15"],
+                    "times": ["04:21:39", "14:23:56"]
+                },
+                {
+                    "dates": ["2013-06-30", "2013-10-04"],
+                    "times": ["22:39:51", "20:34:13"]
+                },
+                {
+                    "dates": ["2014-07-16"],
+                    "times": ["02:28:40"]
+                }
+             ]
+        },
+        {
+            "business_id": "-26MGfikhJiTfCI-GqmzhQ",
+            "checkin_times": [ 
+                {
+                    "dates": ["2018-06-13"],
+                    "times": ["20:16:07"]
+                }
+             ]
+        },
+        {
+            "business_id": "-2bLuJsMZ0WhI9daurVQNQ",
+            "checkin_times": [ 
+                {
+                    "dates": ["2015-05-29", "2015-06-01"],
+                    "times": ["16:46:17", "15:03:53"]
+                }
+             ]
+        },
+        {
+            "business_id": "-2hDBMaza_ldqnZdiU06LQ",
+            "checkin_times": [ 
+                {
+                    "dates": ["2011-10-08"],
+                    "times": ["12:02:23"]
+                },
+                {
+                    "dates": ["2014-08-18"],
+                    "times": ["02:11:11"]
+                },
+                {
+                    "dates": ["2016-01-07", "2016-10-21", "2016-12-01", "2016-12-29"],
+                    "times": ["05:27:51", "20:15:55", "03:57:10", "01:54:42"]
+                },
+                {
+                    "dates": ["2018-07-22", "2018-09-07"],
+                    "times": ["19:55:31", "01:42:54"]
+                },
+                {
+                    "dates": ["2019-03-08"],
+                    "times": ["03:41:06"]
+                }
+             ]
+        }
+    ]
+);
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/bulk-loading/on-index-creation/closed/use-case-4/use-case-4.3.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/bulk-loading/on-index-creation/closed/use-case-4/use-case-4.3.ddl.sqlpp
new file mode 100644
index 0000000..f571dbc
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/bulk-loading/on-index-creation/closed/use-case-4/use-case-4.3.ddl.sqlpp
@@ -0,0 +1,24 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+USE TestYelp;
+
+CREATE INDEX IdxYelpCheckinDates ON YelpCheckin (UNNEST checkin_times UNNEST dates);
+// CREATE INDEX IdxYelpCheckinDatesBusiness ON YelpCheckin ((UNNEST checkin_times UNNEST dates), business_id);
+// CREATE INDEX IdxYelpCheckinBusinessDates ON YelpCheckin (business_id, UNNEST checkin_times UNNEST dates);
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/bulk-loading/on-index-creation/closed/use-case-4/use-case-4.4.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/bulk-loading/on-index-creation/closed/use-case-4/use-case-4.4.query.sqlpp
new file mode 100644
index 0000000..8ba650f
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/bulk-loading/on-index-creation/closed/use-case-4/use-case-4.4.query.sqlpp
@@ -0,0 +1,25 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+SET `compiler.arrayindex` "true";
+
+USE TestYelp;
+
+SELECT C.business_id
+FROM YelpCheckin C, C.checkin_times CT, CT.dates D
+WHERE "2016-04-26" = D;
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/bulk-loading/on-index-creation/closed/with-3-level-record-path/with-3-level-record-path.1.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/bulk-loading/on-index-creation/closed/with-3-level-record-path/with-3-level-record-path.1.ddl.sqlpp
new file mode 100644
index 0000000..acd005f
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/bulk-loading/on-index-creation/closed/with-3-level-record-path/with-3-level-record-path.1.ddl.sqlpp
@@ -0,0 +1,42 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+/*
+ * Description: Verify the bulk-loading operation for a record->record->record->array->atomic index.
+ */
+
+DROP DATAVERSE TestYelp IF EXISTS;
+CREATE DATAVERSE TestYelp;
+USE TestYelp;
+
+CREATE TYPE CheckinType AS {
+    checkin_id: uuid,
+    business_id: string,
+    checkin_data: {
+        checkin_temporal: {
+            checkin_times: {
+                dates: [string],
+                times: [string]
+            }
+        }
+    }
+
+};
+
+CREATE DATASET YelpCheckin(CheckinType) PRIMARY KEY checkin_id AUTOGENERATED;
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/bulk-loading/on-index-creation/closed/with-3-level-record-path/with-3-level-record-path.2.update.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/bulk-loading/on-index-creation/closed/with-3-level-record-path/with-3-level-record-path.2.update.sqlpp
new file mode 100644
index 0000000..884c224
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/bulk-loading/on-index-creation/closed/with-3-level-record-path/with-3-level-record-path.2.update.sqlpp
@@ -0,0 +1,617 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+USE TestYelp;
+
+INSERT INTO YelpCheckin (
+    [
+        {
+            "business_id": "--1UhMGODdWsrMastO9DZw",
+            "checkin_data": {
+                "checkin_temporal": {
+                    "checkin_times": {
+                        "dates": [
+                            "2016-04-26",
+                            "2016-08-30",
+                            "2016-10-15",
+                            "2016-11-18",
+                            "2017-04-20",
+                            "2017-05-03",
+                            "2019-03-19"
+                        ],
+                        "times": [
+                            "19:49:16",
+                            "18:36:57",
+                            "02:45:18",
+                            "01:54:50",
+                            "18:39:06",
+                            "17:58:02",
+                            "22:04:48"
+                        ]
+                    }
+                }
+            }
+        },
+        {
+            "business_id": "--EF5N7P70J_UYBTPypYlA",
+            "checkin_data": {
+                "checkin_temporal": {
+                    "checkin_times": {
+                        "dates": [
+                            "2018-05-25",
+                            "2018-09-18",
+                            "2019-10-18"
+                        ],
+                        "times": [
+                            "19:52:07",
+                            "16:09:44",
+                            "21:29:09"
+                        ]
+                    }
+                }
+            }
+        },
+        {
+            "business_id": "--Ni3oJ4VOqfOEu7Sj2Vzg",
+            "checkin_data": {
+                "checkin_temporal": {
+                    "checkin_times": {
+                        "dates": [
+                            "2019-06-07"
+                        ],
+                        "times": [
+                            "17:54:58"
+                        ]
+                    }
+                }
+            }
+        },
+        {
+            "business_id": "--Y1Adl1YUWfYIRSd8vkmA",
+            "checkin_data": {
+                "checkin_temporal": {
+                    "checkin_times": {
+                        "dates": [
+                            "2011-05-03",
+                            "2011-08-23",
+                            "2014-12-04",
+                            "2016-11-16"
+                        ],
+                        "times": [
+                            "20:54:05",
+                            "20:49:45",
+                            "06:13:01",
+                            "19:25:55"
+                        ]
+                    }
+                }
+            }
+        },
+        {
+            "business_id": "--YPwqIlRJrhHkJcjY3eiA",
+            "checkin_data": {
+                "checkin_temporal": {
+                    "checkin_times": {
+                        "dates": [
+                            "2016-06-18",
+                            "2016-10-15"
+                        ],
+                        "times": [
+                            "21:35:45",
+                            "18:17:51"
+                        ]
+                    }
+                }
+            }
+        },
+        {
+            "business_id": "--e8PjCNhEz32pprnPhCwQ",
+            "checkin_data": {
+                "checkin_temporal": {
+                    "checkin_times": {
+                        "dates": [
+                            "2015-04-02"
+                        ],
+                        "times": [
+                            "21:45:17"
+                        ]
+                    }
+                }
+            }
+        },
+        {
+            "business_id": "--kinfHwmtdjz03g8B8z8Q",
+            "checkin_data": {
+                "checkin_temporal": {
+                    "checkin_times": {
+                        "dates": [
+                            "2014-08-27",
+                            "2015-12-19",
+                            "2018-11-27"
+                        ],
+                        "times": [
+                            "17:49:18",
+                            "21:30:31",
+                            "15:53:50"
+                        ]
+                    }
+                }
+            }
+        },
+        {
+            "business_id": "--q6datkI-f0EoVheXNEeQ",
+            "checkin_data": {
+                "checkin_temporal": {
+                    "checkin_times": {
+                        "dates": [
+                            "2014-01-28",
+                            "2014-11-16",
+                            "2015-11-15",
+                            "2015-11-15"
+                        ],
+                        "times": [
+                            "20:56:04",
+                            "16:11:58",
+                            "19:21:53",
+                            "19:33:39"
+                        ]
+                    }
+                }
+            }
+        },
+        {
+            "business_id": "--qvQS4MigHPykD2GV0-zw",
+            "checkin_data": {
+                "checkin_temporal": {
+                    "checkin_times": {
+                        "dates": [
+                            "2019-04-11"
+                        ],
+                        "times": [
+                            "18:30:12"
+                        ]
+                    }
+                }
+            }
+        },
+        {
+            "business_id": "--wIGbLEhlpl_UeAIyDmZQ",
+            "checkin_data": {
+                "checkin_temporal": {
+                    "checkin_times": {
+                        "dates": [
+                            "2015-06-06",
+                            "2019-03-14"
+                        ],
+                        "times": [
+                            "20:01:06",
+                            "22:01:52"
+                        ]
+                    }
+                }
+            }
+        },
+        {
+            "business_id": "-0FA-Qdi3SPYIoJz9UQw-A",
+            "checkin_data": {
+                "checkin_temporal": {
+                    "checkin_times": {
+                        "dates": [
+                            "2018-09-29",
+                            "2018-10-20",
+                            "2018-10-20"
+                        ],
+                        "times": [
+                            "18:55:17",
+                            "16:48:05",
+                            "22:20:24"
+                        ]
+                    }
+                }
+            }
+        },
+        {
+            "business_id": "-0Hj1hb_XW6ybWq2M7QhGA",
+            "checkin_data": {
+                "checkin_temporal": {
+                    "checkin_times": {
+                        "dates": [
+                            "2011-04-23",
+                            "2014-05-04",
+                            "2014-05-11",
+                            "2014-06-04",
+                            "2015-12-05",
+                            "2017-05-15"
+                        ],
+                        "times": [
+                            "21:11:22",
+                            "19:42:48",
+                            "19:16:08",
+                            "19:14:18",
+                            "19:22:42",
+                            "23:19:00"
+                        ]
+                    }
+                }
+            }
+        },
+        {
+            "business_id": "-0KMvRFwDWdVBeTpT11iHw",
+            "checkin_data": {
+                "checkin_temporal": {
+                    "checkin_times": {
+                        "dates": [
+                            "2012-07-13",
+                            "2016-12-24",
+                            "2017-08-31"
+                        ],
+                        "times": [
+                            "21:43:57",
+                            "02:27:31",
+                            "00:35:26"
+                        ]
+                    }
+                }
+            }
+        },
+        {
+            "business_id": "-0LPtgJC31FWMrMv317p0Q",
+            "checkin_data": {
+                "checkin_temporal": {
+                    "checkin_times": {
+                        "dates": [
+                            "2013-04-13",
+                            "2013-08-19",
+                            "2013-10-04"
+                        ],
+                        "times": [
+                            "12:35:33",
+                            "23:35:49",
+                            "19:14:56"
+                        ]
+                    }
+                }
+            }
+        },
+        {
+            "business_id": "-0M3o2uWBnQZwd3hmfEwuw",
+            "checkin_data": {
+                "checkin_temporal": {
+                    "checkin_times": {
+                        "dates": [
+                            "2016-09-10",
+                            "2018-09-08",
+                            "2019-09-13"
+                        ],
+                        "times": [
+                            "19:26:19",
+                            "14:15:37",
+                            "22:47:25"
+                        ]
+                    }
+                }
+            }
+        },
+        {
+            "business_id": "-0RRiWDtfnS16AKCtfvBZg",
+            "checkin_data": {
+                "checkin_temporal": {
+                    "checkin_times": {
+                        "dates": [
+                            "2017-05-19",
+                            "2017-05-19",
+                            "2017-08-28",
+                            "2017-09-20",
+                            "2017-10-01",
+                            "2017-10-01",
+                            "2017-12-27"
+                        ],
+                        "times": [
+                            "14:30:16",
+                            "14:30:25",
+                            "15:49:37",
+                            "20:19:51",
+                            "16:31:05",
+                            "16:56:27",
+                            "23:33:20"
+                        ]
+                    }
+                }
+            }
+        },
+        {
+            "business_id": "-0Soj75v-XoRcf2ERr8Bmg",
+            "checkin_data": {
+                "checkin_temporal": {
+                    "checkin_times": {
+                        "dates": [
+                            "2019-06-05"
+                        ],
+                        "times": [
+                            "18:22:49"
+                        ]
+                    }
+                }
+            }
+        },
+        {
+            "business_id": "-0ZumLlFjMh4ZW1z2nXGug",
+            "checkin_data": {
+                "checkin_temporal": {
+                    "checkin_times": {
+                        "dates": [
+                            "2011-09-24",
+                            "2014-03-10",
+                            "2015-05-27",
+                            "2015-08-29",
+                            "2018-03-16"
+                        ],
+                        "times": [
+                            "21:37:32",
+                            "20:20:07",
+                            "00:40:24",
+                            "17:58:15",
+                            "15:03:26"
+                        ]
+                    }
+                }
+            }
+        },
+        {
+            "business_id": "-0aOudcaAyac0VJbMX-L1g",
+            "checkin_data": {
+                "checkin_temporal": {
+                    "checkin_times": {
+                        "dates": [
+                            "2015-03-16",
+                            "2015-12-21",
+                            "2016-10-28",
+                            "2016-10-28"
+                        ],
+                        "times": [
+                            "23:51:16",
+                            "04:48:01",
+                            "20:22:42",
+                            "20:23:00"
+                        ]
+                    }
+                }
+            }
+        },
+        {
+            "business_id": "-0b86isaXMY0v4g-V8GZ9Q",
+            "checkin_data": {
+                "checkin_temporal": {
+                    "checkin_times": {
+                        "dates": [
+                            "2013-10-22",
+                            "2014-11-21"
+                        ],
+                        "times": [
+                            "16:49:21",
+                            "17:39:24"
+                        ]
+                    }
+                }
+            }
+        },
+        {
+            "business_id": "-0d-BfFSU0bwLcnMaGRxYw",
+            "checkin_data": {
+                "checkin_temporal": {
+                    "checkin_times": {
+                        "dates": [
+                            "2014-08-07",
+                            "2014-09-16",
+                            "2014-10-12",
+                            "2015-07-21",
+                            "2015-07-21"
+                        ],
+                        "times": [
+                            "18:30:48",
+                            "20:41:45",
+                            "23:22:27",
+                            "20:43:56",
+                            "20:45:07"
+                        ]
+                    }
+                }
+            }
+        },
+        {
+            "business_id": "-0jz6c3C6i7RG7Ag22K-Pg",
+            "checkin_data": {
+                "checkin_temporal": {
+                    "checkin_times": {
+                        "dates": [
+                            "2015-05-02",
+                            "2015-05-06",
+                            "2015-09-26"
+                        ],
+                        "times": [
+                            "19:49:05",
+                            "03:52:18",
+                            "01:13:19"
+                        ]
+                    }
+                }
+            }
+        },
+        {
+            "business_id": "-0y3MZU2oYP8r1ruDP1bfQ",
+            "checkin_data": {
+                "checkin_temporal": {
+                    "checkin_times": {
+                        "dates": [
+                            "2015-04-11",
+                            "2015-11-21",
+                            "2016-05-06",
+                            "2017-08-09",
+                            "2017-10-21"
+                        ],
+                        "times": [
+                            "13:14:14",
+                            "16:05:56",
+                            "14:10:04",
+                            "15:15:10",
+                            "15:12:56"
+                        ]
+                    }
+                }
+            }
+        },
+        {
+            "business_id": "-1BPe8UjF2_l3nVk-DFUjA",
+            "checkin_data": {
+                "checkin_temporal": {
+                    "checkin_times": {
+                        "dates": [
+                            "2015-12-03",
+                            "2016-03-17",
+                            "2016-11-02"
+                        ],
+                        "times": [
+                            "18:44:00",
+                            "18:19:21",
+                            "15:58:38"
+                        ]
+                    }
+                }
+            }
+        },
+        {
+            "business_id": "-1E2CQu_38mkghvmZgCCRw",
+            "checkin_data": {
+                "checkin_temporal": {
+                    "checkin_times": {
+                        "dates": [
+                            "2019-04-04"
+                        ],
+                        "times": [
+                            "22:02:37"
+                        ]
+                    }
+                }
+            }
+        },
+        {
+            "business_id": "-1wzk43IZ5D9Ysu6kzb5xA",
+            "checkin_data": {
+                "checkin_temporal": {
+                    "checkin_times": {
+                        "dates": [
+                            "2019-02-27"
+                        ],
+                        "times": [
+                            "14:03:08"
+                        ]
+                    }
+                }
+            }
+        },
+        {
+            "business_id": "-23R9P2eG7VTc6DVLjFKzA",
+            "checkin_data": {
+                "checkin_temporal": {
+                    "checkin_times": {
+                        "dates": [
+                            "2011-12-21",
+                            "2012-04-15",
+                            "2012-04-15",
+                            "2013-06-30",
+                            "2013-10-04",
+                            "2014-07-16"
+                        ],
+                        "times": [
+                            "19:02:51",
+                            "04:21:39",
+                            "14:23:56",
+                            "22:39:51",
+                            "20:34:13",
+                            "02:28:40"
+                        ]
+                    }
+                }
+            }
+        },
+        {
+            "business_id": "-26MGfikhJiTfCI-GqmzhQ",
+            "checkin_data": {
+                "checkin_temporal": {
+                    "checkin_times": {
+                        "dates": [
+                            "2018-06-13"
+                        ],
+                        "times": [
+                            "20:16:07"
+                        ]
+                    }
+                }
+            }
+        },
+        {
+            "business_id": "-2bLuJsMZ0WhI9daurVQNQ",
+            "checkin_data": {
+                "checkin_temporal": {
+                    "checkin_times": {
+                        "dates": [
+                            "2015-05-29",
+                            "2015-06-01"
+                        ],
+                        "times": [
+                            "16:46:17",
+                            "15:03:53"
+                        ]
+                    }
+                }
+            }
+        },
+        {
+            "business_id": "-2hDBMaza_ldqnZdiU06LQ",
+            "checkin_data": {
+                "checkin_temporal": {
+                    "checkin_times": {
+                        "dates": [
+                            "2011-10-08",
+                            "2014-08-18",
+                            "2016-01-07",
+                            "2016-10-21",
+                            "2016-12-01",
+                            "2016-12-29",
+                            "2018-07-22",
+                            "2018-09-07",
+                            "2019-03-08"
+                        ],
+                        "times": [
+                            "12:02:23",
+                            "02:11:11",
+                            "05:27:51",
+                            "20:15:55",
+                            "03:57:10",
+                            "01:54:42",
+                            "19:55:31",
+                            "01:42:54",
+                            "03:41:06"
+                        ]
+                    }
+                }
+            }
+        }
+    ]
+);
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/bulk-loading/on-index-creation/closed/with-3-level-record-path/with-3-level-record-path.3.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/bulk-loading/on-index-creation/closed/with-3-level-record-path/with-3-level-record-path.3.ddl.sqlpp
new file mode 100644
index 0000000..5ef37bc
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/bulk-loading/on-index-creation/closed/with-3-level-record-path/with-3-level-record-path.3.ddl.sqlpp
@@ -0,0 +1,22 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+USE TestYelp;
+
+CREATE INDEX IdxYelpCheckinDates ON YelpCheckin (UNNEST checkin_data.checkin_temporal.checkin_times.dates);
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/bulk-loading/on-index-creation/closed/with-3-level-record-path/with-3-level-record-path.4.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/bulk-loading/on-index-creation/closed/with-3-level-record-path/with-3-level-record-path.4.query.sqlpp
new file mode 100644
index 0000000..cee1887
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/bulk-loading/on-index-creation/closed/with-3-level-record-path/with-3-level-record-path.4.query.sqlpp
@@ -0,0 +1,25 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+SET `compiler.arrayindex` "true";
+
+USE TestYelp;
+
+SELECT C.business_id
+FROM YelpCheckin C, C.checkin_data.checkin_temporal.checkin_times.dates D
+WHERE "2016-04-26" = D;
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/bulk-loading/on-index-creation/closed/with-composite-pk/with-composite-pk.1.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/bulk-loading/on-index-creation/closed/with-composite-pk/with-composite-pk.1.ddl.sqlpp
new file mode 100644
index 0000000..88ac065
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/bulk-loading/on-index-creation/closed/with-composite-pk/with-composite-pk.1.ddl.sqlpp
@@ -0,0 +1,34 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+/*
+ * Description: Verify the bulk-loading operation on array indexes with more than one PK.
+ */
+
+DROP DATAVERSE TestYelp IF EXISTS;
+CREATE DATAVERSE TestYelp;
+USE TestYelp;
+
+CREATE TYPE CheckinType AS {
+    checkin_id: smallint,
+    business_id: string,
+    dates: [string]
+};
+
+CREATE DATASET YelpCheckin(CheckinType) PRIMARY KEY checkin_id, business_id;
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/bulk-loading/on-index-creation/closed/with-composite-pk/with-composite-pk.2.update.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/bulk-loading/on-index-creation/closed/with-composite-pk/with-composite-pk.2.update.sqlpp
new file mode 100644
index 0000000..6498c14
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/bulk-loading/on-index-creation/closed/with-composite-pk/with-composite-pk.2.update.sqlpp
@@ -0,0 +1,306 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+USE TestYelp;
+
+INSERT INTO YelpCheckin (
+    [
+        {
+            "checkin_id": 1,
+            "business_id": "--1UhMGODdWsrMastO9DZw",
+            "dates": [
+                "2016-04-26 19:49:16",
+                "2016-08-30 18:36:57",
+                "2016-10-15 02:45:18",
+                "2016-11-18 01:54:50",
+                "2017-04-20 18:39:06",
+                "2017-05-03 17:58:02",
+                "2019-03-19 22:04:48"
+            ]
+        },
+        {
+            "checkin_id": 2,
+            "business_id": "--EF5N7P70J_UYBTPypYlA",
+            "dates": [
+                "2018-05-25 19:52:07",
+                "2018-09-18 16:09:44",
+                "2019-10-18 21:29:09"
+            ]
+        },
+        {
+            "checkin_id": 3,
+            "business_id": "--Ni3oJ4VOqfOEu7Sj2Vzg",
+            "dates": [
+                "2019-06-07 17:54:58"
+            ]
+        },
+        {
+            "checkin_id": 4,
+            "business_id": "--Y1Adl1YUWfYIRSd8vkmA",
+            "dates": [
+                "2011-05-03 20:54:05",
+                "2011-08-23 20:49:45",
+                "2014-12-04 06:13:01",
+                "2016-11-16 19:25:55"
+            ]
+        },
+        {
+            "checkin_id": 5,
+            "business_id": "--YPwqIlRJrhHkJcjY3eiA",
+            "dates": [
+                "2016-06-18 21:35:45",
+                "2016-10-15 18:17:51"
+            ]
+        },
+        {
+            "checkin_id": 6,
+            "business_id": "--e8PjCNhEz32pprnPhCwQ",
+            "dates": [
+                "2015-04-02 21:45:17"
+            ]
+        },
+        {
+            "checkin_id": 7,
+            "business_id": "--kinfHwmtdjz03g8B8z8Q",
+            "dates": [
+                "2014-08-27 17:49:18",
+                "2015-12-19 21:30:31",
+                "2018-11-27 15:53:50"
+            ]
+        },
+        {
+            "checkin_id": 8,
+            "business_id": "--q6datkI-f0EoVheXNEeQ",
+            "dates": [
+                "2014-01-28 20:56:04",
+                "2014-11-16 16:11:58",
+                "2015-11-15 19:21:53",
+                "2015-11-15 19:33:39"
+            ]
+        },
+        {
+            "checkin_id": 9,
+            "business_id": "--qvQS4MigHPykD2GV0-zw",
+            "dates": [
+                "2019-04-11 18:30:12"
+            ]
+        },
+        {
+            "checkin_id": 10,
+            "business_id": "--wIGbLEhlpl_UeAIyDmZQ",
+            "dates": [
+                "2015-06-06 20:01:06",
+                "2019-03-14 22:01:52"
+            ]
+        },
+        {
+            "checkin_id": 11,
+            "business_id": "-0FA-Qdi3SPYIoJz9UQw-A",
+            "dates": [
+                "2018-09-29 18:55:17",
+                "2018-10-20 16:48:05",
+                "2018-10-20 22:20:24"
+            ]
+        },
+        {
+            "checkin_id": 12,
+            "business_id": "-0Hj1hb_XW6ybWq2M7QhGA",
+            "dates": [
+                "2011-04-23 21:11:22",
+                "2014-05-04 19:42:48",
+                "2014-05-11 19:16:08",
+                "2014-06-04 19:14:18",
+                "2015-12-05 19:22:42",
+                "2017-05-15 23:19:00"
+            ]
+        },
+        {
+            "checkin_id": 13,
+            "business_id": "-0KMvRFwDWdVBeTpT11iHw",
+            "dates": [
+                "2012-07-13 21:43:57",
+                "2016-12-24 02:27:31",
+                "2017-08-31 00:35:26"
+            ]
+        },
+        {
+            "checkin_id": 14,
+            "business_id": "-0LPtgJC31FWMrMv317p0Q",
+            "dates": [
+                "2013-04-13 12:35:33",
+                "2013-08-19 23:35:49",
+                "2013-10-04 19:14:56"
+            ]
+        },
+        {
+            "checkin_id": 15,
+            "business_id": "-0M3o2uWBnQZwd3hmfEwuw",
+            "dates": [
+                "2016-09-10 19:26:19",
+                "2018-09-08 14:15:37",
+                "2019-09-13 22:47:25"
+            ]
+        },
+        {
+            "checkin_id": 16,
+            "business_id": "-0RRiWDtfnS16AKCtfvBZg",
+            "dates": [
+                "2017-05-19 14:30:16",
+                "2017-05-19 14:30:25",
+                "2017-08-28 15:49:37",
+                "2017-09-20 20:19:51",
+                "2017-10-01 16:31:05",
+                "2017-10-01 16:56:27",
+                "2017-12-27 23:33:20"
+            ]
+        },
+        {
+            "checkin_id": 17,
+            "business_id": "-0Soj75v-XoRcf2ERr8Bmg",
+            "dates": [
+                "2019-06-05 18:22:49"
+            ]
+        },
+        {
+            "checkin_id": 18,
+            "business_id": "-0ZumLlFjMh4ZW1z2nXGug",
+            "dates": [
+                "2011-09-24 21:37:32",
+                "2014-03-10 20:20:07",
+                "2015-05-27 00:40:24",
+                "2015-08-29 17:58:15",
+                "2018-03-16 15:03:26"
+            ]
+        },
+        {
+            "checkin_id": 19,
+            "business_id": "-0aOudcaAyac0VJbMX-L1g",
+            "dates": [
+                "2015-03-16 23:51:16",
+                "2015-12-21 04:48:01",
+                "2016-10-28 20:22:42",
+                "2016-10-28 20:23:00"
+            ]
+        },
+        {
+            "checkin_id": 20,
+            "business_id": "-0b86isaXMY0v4g-V8GZ9Q",
+            "dates": [
+                "2013-10-22 16:49:21",
+                "2014-11-21 17:39:24"
+            ]
+        },
+        {
+            "checkin_id": 21,
+            "business_id": "-0d-BfFSU0bwLcnMaGRxYw",
+            "dates": [
+                "2014-08-07 18:30:48",
+                "2014-09-16 20:41:45",
+                "2014-10-12 23:22:27",
+                "2015-07-21 20:43:56",
+                "2015-07-21 20:45:07"
+            ]
+        },
+        {
+            "checkin_id": 22,
+            "business_id": "-0jz6c3C6i7RG7Ag22K-Pg",
+            "dates": [
+                "2015-05-02 19:49:05",
+                "2015-05-06 03:52:18",
+                "2015-09-26 01:13:19"
+            ]
+        },
+        {
+            "checkin_id": 23,
+            "business_id": "-0y3MZU2oYP8r1ruDP1bfQ",
+            "dates": [
+                "2015-04-11 13:14:14",
+                "2015-11-21 16:05:56",
+                "2016-05-06 14:10:04",
+                "2017-08-09 15:15:10",
+                "2017-10-21 15:12:56"
+            ]
+        },
+        {
+            "checkin_id": 24,
+            "business_id": "-1BPe8UjF2_l3nVk-DFUjA",
+            "dates": [
+                "2015-12-03 18:44:00",
+                "2016-03-17 18:19:21",
+                "2016-11-02 15:58:38"
+            ]
+        },
+        {
+            "checkin_id": 25,
+            "business_id": "-1E2CQu_38mkghvmZgCCRw",
+            "dates": [
+                "2019-04-04 22:02:37"
+            ]
+        },
+        {
+            "checkin_id": 26,
+            "business_id": "-1wzk43IZ5D9Ysu6kzb5xA",
+            "dates": [
+                "2019-02-27 14:03:08"
+            ]
+        },
+        {
+            "checkin_id": 27,
+            "business_id": "-23R9P2eG7VTc6DVLjFKzA",
+            "dates": [
+                "2011-12-21 19:02:51",
+                "2012-04-15 04:21:39",
+                "2012-04-15 14:23:56",
+                "2013-06-30 22:39:51",
+                "2013-10-04 20:34:13",
+                "2014-07-16 02:28:40"
+            ]
+        },
+        {
+            "checkin_id": 28,
+            "business_id": "-26MGfikhJiTfCI-GqmzhQ",
+            "dates": [
+                "2018-06-13 20:16:07"
+            ]
+        },
+        {
+            "checkin_id": 29,
+            "business_id": "-2bLuJsMZ0WhI9daurVQNQ",
+            "dates": [
+                "2015-05-29 16:46:17",
+                "2015-06-01 15:03:53"
+            ]
+        },
+        {
+            "checkin_id": 30,
+            "business_id": "-2hDBMaza_ldqnZdiU06LQ",
+            "dates": [
+                "2011-10-08 12:02:23",
+                "2014-08-18 02:11:11",
+                "2016-01-07 05:27:51",
+                "2016-10-21 20:15:55",
+                "2016-12-01 03:57:10",
+                "2016-12-29 01:54:42",
+                "2018-07-22 19:55:31",
+                "2018-09-07 01:42:54",
+                "2019-03-08 03:41:06"
+            ]
+        }
+    ]
+);
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/bulk-loading/on-index-creation/closed/with-composite-pk/with-composite-pk.3.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/bulk-loading/on-index-creation/closed/with-composite-pk/with-composite-pk.3.ddl.sqlpp
new file mode 100644
index 0000000..a0fc1bf
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/bulk-loading/on-index-creation/closed/with-composite-pk/with-composite-pk.3.ddl.sqlpp
@@ -0,0 +1,22 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+USE TestYelp;
+
+CREATE INDEX IdxYelpCheckinDates ON YelpCheckin (UNNEST dates);
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/bulk-loading/on-index-creation/closed/with-composite-pk/with-composite-pk.4.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/bulk-loading/on-index-creation/closed/with-composite-pk/with-composite-pk.4.query.sqlpp
new file mode 100644
index 0000000..ec79bc6
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/bulk-loading/on-index-creation/closed/with-composite-pk/with-composite-pk.4.query.sqlpp
@@ -0,0 +1,25 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+SET `compiler.arrayindex` "true";
+
+USE TestYelp;
+
+SELECT C.business_id
+FROM YelpCheckin C, C.dates D
+WHERE "2016-04-26 19:49:16" = D;
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/bulk-loading/on-index-creation/closed/with-filter-fields/with-filter-fields.1.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/bulk-loading/on-index-creation/closed/with-filter-fields/with-filter-fields.1.ddl.sqlpp
new file mode 100644
index 0000000..29fd915
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/bulk-loading/on-index-creation/closed/with-filter-fields/with-filter-fields.1.ddl.sqlpp
@@ -0,0 +1,34 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+/*
+ * Description: Verify the bulk-loading operation for an array index with filter fields.
+ */
+
+DROP DATAVERSE TestYelp IF EXISTS;
+CREATE DATAVERSE TestYelp;
+USE TestYelp;
+
+CREATE TYPE CheckinType AS {
+    checkin_id: uuid,
+    business_id: string,
+    dates: [string]
+};
+
+CREATE DATASET YelpCheckin(CheckinType) PRIMARY KEY checkin_id AUTOGENERATED WITH FILTER ON business_id;
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/bulk-loading/on-index-creation/closed/with-filter-fields/with-filter-fields.2.update.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/bulk-loading/on-index-creation/closed/with-filter-fields/with-filter-fields.2.update.sqlpp
new file mode 100644
index 0000000..b75bf1b
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/bulk-loading/on-index-creation/closed/with-filter-fields/with-filter-fields.2.update.sqlpp
@@ -0,0 +1,276 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+USE TestYelp;
+
+INSERT INTO YelpCheckin (
+    [
+        {
+            "business_id": "--1UhMGODdWsrMastO9DZw",
+            "dates": [
+                "2016-04-26 19:49:16",
+                "2016-08-30 18:36:57",
+                "2016-10-15 02:45:18",
+                "2016-11-18 01:54:50",
+                "2017-04-20 18:39:06",
+                "2017-05-03 17:58:02",
+                "2019-03-19 22:04:48"
+            ]
+        },
+        {
+            "business_id": "--EF5N7P70J_UYBTPypYlA",
+            "dates": [
+                "2018-05-25 19:52:07",
+                "2018-09-18 16:09:44",
+                "2019-10-18 21:29:09"
+            ]
+        },
+        {
+            "business_id": "--Ni3oJ4VOqfOEu7Sj2Vzg",
+            "dates": [
+                "2019-06-07 17:54:58"
+            ]
+        },
+        {
+            "business_id": "--Y1Adl1YUWfYIRSd8vkmA",
+            "dates": [
+                "2011-05-03 20:54:05",
+                "2011-08-23 20:49:45",
+                "2014-12-04 06:13:01",
+                "2016-11-16 19:25:55"
+            ]
+        },
+        {
+            "business_id": "--YPwqIlRJrhHkJcjY3eiA",
+            "dates": [
+                "2016-06-18 21:35:45",
+                "2016-10-15 18:17:51"
+            ]
+        },
+        {
+            "business_id": "--e8PjCNhEz32pprnPhCwQ",
+            "dates": [
+                "2015-04-02 21:45:17"
+            ]
+        },
+        {
+            "business_id": "--kinfHwmtdjz03g8B8z8Q",
+            "dates": [
+                "2014-08-27 17:49:18",
+                "2015-12-19 21:30:31",
+                "2018-11-27 15:53:50"
+            ]
+        },
+        {
+            "business_id": "--q6datkI-f0EoVheXNEeQ",
+            "dates": [
+                "2014-01-28 20:56:04",
+                "2014-11-16 16:11:58",
+                "2015-11-15 19:21:53",
+                "2015-11-15 19:33:39"
+            ]
+        },
+        {
+            "business_id": "--qvQS4MigHPykD2GV0-zw",
+            "dates": [
+                "2019-04-11 18:30:12"
+            ]
+        },
+        {
+            "business_id": "--wIGbLEhlpl_UeAIyDmZQ",
+            "dates": [
+                "2015-06-06 20:01:06",
+                "2019-03-14 22:01:52"
+            ]
+        },
+        {
+            "business_id": "-0FA-Qdi3SPYIoJz9UQw-A",
+            "dates": [
+                "2018-09-29 18:55:17",
+                "2018-10-20 16:48:05",
+                "2018-10-20 22:20:24"
+            ]
+        },
+        {
+            "business_id": "-0Hj1hb_XW6ybWq2M7QhGA",
+            "dates": [
+                "2011-04-23 21:11:22",
+                "2014-05-04 19:42:48",
+                "2014-05-11 19:16:08",
+                "2014-06-04 19:14:18",
+                "2015-12-05 19:22:42",
+                "2017-05-15 23:19:00"
+            ]
+        },
+        {
+            "business_id": "-0KMvRFwDWdVBeTpT11iHw",
+            "dates": [
+                "2012-07-13 21:43:57",
+                "2016-12-24 02:27:31",
+                "2017-08-31 00:35:26"
+            ]
+        },
+        {
+            "business_id": "-0LPtgJC31FWMrMv317p0Q",
+            "dates": [
+                "2013-04-13 12:35:33",
+                "2013-08-19 23:35:49",
+                "2013-10-04 19:14:56"
+            ]
+        },
+        {
+            "business_id": "-0M3o2uWBnQZwd3hmfEwuw",
+            "dates": [
+                "2016-09-10 19:26:19",
+                "2018-09-08 14:15:37",
+                "2019-09-13 22:47:25"
+            ]
+        },
+        {
+            "business_id": "-0RRiWDtfnS16AKCtfvBZg",
+            "dates": [
+                "2017-05-19 14:30:16",
+                "2017-05-19 14:30:25",
+                "2017-08-28 15:49:37",
+                "2017-09-20 20:19:51",
+                "2017-10-01 16:31:05",
+                "2017-10-01 16:56:27",
+                "2017-12-27 23:33:20"
+            ]
+        },
+        {
+            "business_id": "-0Soj75v-XoRcf2ERr8Bmg",
+            "dates": [
+                "2019-06-05 18:22:49"
+            ]
+        },
+        {
+            "business_id": "-0ZumLlFjMh4ZW1z2nXGug",
+            "dates": [
+                "2011-09-24 21:37:32",
+                "2014-03-10 20:20:07",
+                "2015-05-27 00:40:24",
+                "2015-08-29 17:58:15",
+                "2018-03-16 15:03:26"
+            ]
+        },
+        {
+            "business_id": "-0aOudcaAyac0VJbMX-L1g",
+            "dates": [
+                "2015-03-16 23:51:16",
+                "2015-12-21 04:48:01",
+                "2016-10-28 20:22:42",
+                "2016-10-28 20:23:00"
+            ]
+        },
+        {
+            "business_id": "-0b86isaXMY0v4g-V8GZ9Q",
+            "dates": [
+                "2013-10-22 16:49:21",
+                "2014-11-21 17:39:24"
+            ]
+        },
+        {
+            "business_id": "-0d-BfFSU0bwLcnMaGRxYw",
+            "dates": [
+                "2014-08-07 18:30:48",
+                "2014-09-16 20:41:45",
+                "2014-10-12 23:22:27",
+                "2015-07-21 20:43:56",
+                "2015-07-21 20:45:07"
+            ]
+        },
+        {
+            "business_id": "-0jz6c3C6i7RG7Ag22K-Pg",
+            "dates": [
+                "2015-05-02 19:49:05",
+                "2015-05-06 03:52:18",
+                "2015-09-26 01:13:19"
+            ]
+        },
+        {
+            "business_id": "-0y3MZU2oYP8r1ruDP1bfQ",
+            "dates": [
+                "2015-04-11 13:14:14",
+                "2015-11-21 16:05:56",
+                "2016-05-06 14:10:04",
+                "2017-08-09 15:15:10",
+                "2017-10-21 15:12:56"
+            ]
+        },
+        {
+            "business_id": "-1BPe8UjF2_l3nVk-DFUjA",
+            "dates": [
+                "2015-12-03 18:44:00",
+                "2016-03-17 18:19:21",
+                "2016-11-02 15:58:38"
+            ]
+        },
+        {
+            "business_id": "-1E2CQu_38mkghvmZgCCRw",
+            "dates": [
+                "2019-04-04 22:02:37"
+            ]
+        },
+        {
+            "business_id": "-1wzk43IZ5D9Ysu6kzb5xA",
+            "dates": [
+                "2019-02-27 14:03:08"
+            ]
+        },
+        {
+            "business_id": "-23R9P2eG7VTc6DVLjFKzA",
+            "dates": [
+                "2011-12-21 19:02:51",
+                "2012-04-15 04:21:39",
+                "2012-04-15 14:23:56",
+                "2013-06-30 22:39:51",
+                "2013-10-04 20:34:13",
+                "2014-07-16 02:28:40"
+            ]
+        },
+        {
+            "business_id": "-26MGfikhJiTfCI-GqmzhQ",
+            "dates": [
+                "2018-06-13 20:16:07"
+            ]
+        },
+        {
+            "business_id": "-2bLuJsMZ0WhI9daurVQNQ",
+            "dates": [
+                "2015-05-29 16:46:17",
+                "2015-06-01 15:03:53"
+            ]
+        },
+        {
+            "business_id": "-2hDBMaza_ldqnZdiU06LQ",
+            "dates": [
+                "2011-10-08 12:02:23",
+                "2014-08-18 02:11:11",
+                "2016-01-07 05:27:51",
+                "2016-10-21 20:15:55",
+                "2016-12-01 03:57:10",
+                "2016-12-29 01:54:42",
+                "2018-07-22 19:55:31",
+                "2018-09-07 01:42:54",
+                "2019-03-08 03:41:06"
+            ]
+        }
+    ]
+);
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/bulk-loading/on-index-creation/closed/with-filter-fields/with-filter-fields.3.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/bulk-loading/on-index-creation/closed/with-filter-fields/with-filter-fields.3.ddl.sqlpp
new file mode 100644
index 0000000..a0fc1bf
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/bulk-loading/on-index-creation/closed/with-filter-fields/with-filter-fields.3.ddl.sqlpp
@@ -0,0 +1,22 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+USE TestYelp;
+
+CREATE INDEX IdxYelpCheckinDates ON YelpCheckin (UNNEST dates);
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/bulk-loading/on-index-creation/closed/with-filter-fields/with-filter-fields.4.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/bulk-loading/on-index-creation/closed/with-filter-fields/with-filter-fields.4.query.sqlpp
new file mode 100644
index 0000000..ec79bc6
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/bulk-loading/on-index-creation/closed/with-filter-fields/with-filter-fields.4.query.sqlpp
@@ -0,0 +1,25 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+SET `compiler.arrayindex` "true";
+
+USE TestYelp;
+
+SELECT C.business_id
+FROM YelpCheckin C, C.dates D
+WHERE "2016-04-26 19:49:16" = D;
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/bulk-loading/on-index-creation/open/use-case-1/use-case-1.1.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/bulk-loading/on-index-creation/open/use-case-1/use-case-1.1.ddl.sqlpp
new file mode 100644
index 0000000..4076942
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/bulk-loading/on-index-creation/open/use-case-1/use-case-1.1.ddl.sqlpp
@@ -0,0 +1,32 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+/*
+ * Description: Verify the bulk-loading operation for an array->atomic index.
+ */
+
+DROP DATAVERSE TestYelp IF EXISTS;
+CREATE DATAVERSE TestYelp;
+USE TestYelp;
+
+CREATE TYPE CheckinType AS {
+    checkin_id: uuid
+};
+
+CREATE DATASET YelpCheckin(CheckinType) PRIMARY KEY checkin_id AUTOGENERATED;
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/bulk-loading/on-index-creation/open/use-case-1/use-case-1.2.update.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/bulk-loading/on-index-creation/open/use-case-1/use-case-1.2.update.sqlpp
new file mode 100644
index 0000000..b75bf1b
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/bulk-loading/on-index-creation/open/use-case-1/use-case-1.2.update.sqlpp
@@ -0,0 +1,276 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+USE TestYelp;
+
+INSERT INTO YelpCheckin (
+    [
+        {
+            "business_id": "--1UhMGODdWsrMastO9DZw",
+            "dates": [
+                "2016-04-26 19:49:16",
+                "2016-08-30 18:36:57",
+                "2016-10-15 02:45:18",
+                "2016-11-18 01:54:50",
+                "2017-04-20 18:39:06",
+                "2017-05-03 17:58:02",
+                "2019-03-19 22:04:48"
+            ]
+        },
+        {
+            "business_id": "--EF5N7P70J_UYBTPypYlA",
+            "dates": [
+                "2018-05-25 19:52:07",
+                "2018-09-18 16:09:44",
+                "2019-10-18 21:29:09"
+            ]
+        },
+        {
+            "business_id": "--Ni3oJ4VOqfOEu7Sj2Vzg",
+            "dates": [
+                "2019-06-07 17:54:58"
+            ]
+        },
+        {
+            "business_id": "--Y1Adl1YUWfYIRSd8vkmA",
+            "dates": [
+                "2011-05-03 20:54:05",
+                "2011-08-23 20:49:45",
+                "2014-12-04 06:13:01",
+                "2016-11-16 19:25:55"
+            ]
+        },
+        {
+            "business_id": "--YPwqIlRJrhHkJcjY3eiA",
+            "dates": [
+                "2016-06-18 21:35:45",
+                "2016-10-15 18:17:51"
+            ]
+        },
+        {
+            "business_id": "--e8PjCNhEz32pprnPhCwQ",
+            "dates": [
+                "2015-04-02 21:45:17"
+            ]
+        },
+        {
+            "business_id": "--kinfHwmtdjz03g8B8z8Q",
+            "dates": [
+                "2014-08-27 17:49:18",
+                "2015-12-19 21:30:31",
+                "2018-11-27 15:53:50"
+            ]
+        },
+        {
+            "business_id": "--q6datkI-f0EoVheXNEeQ",
+            "dates": [
+                "2014-01-28 20:56:04",
+                "2014-11-16 16:11:58",
+                "2015-11-15 19:21:53",
+                "2015-11-15 19:33:39"
+            ]
+        },
+        {
+            "business_id": "--qvQS4MigHPykD2GV0-zw",
+            "dates": [
+                "2019-04-11 18:30:12"
+            ]
+        },
+        {
+            "business_id": "--wIGbLEhlpl_UeAIyDmZQ",
+            "dates": [
+                "2015-06-06 20:01:06",
+                "2019-03-14 22:01:52"
+            ]
+        },
+        {
+            "business_id": "-0FA-Qdi3SPYIoJz9UQw-A",
+            "dates": [
+                "2018-09-29 18:55:17",
+                "2018-10-20 16:48:05",
+                "2018-10-20 22:20:24"
+            ]
+        },
+        {
+            "business_id": "-0Hj1hb_XW6ybWq2M7QhGA",
+            "dates": [
+                "2011-04-23 21:11:22",
+                "2014-05-04 19:42:48",
+                "2014-05-11 19:16:08",
+                "2014-06-04 19:14:18",
+                "2015-12-05 19:22:42",
+                "2017-05-15 23:19:00"
+            ]
+        },
+        {
+            "business_id": "-0KMvRFwDWdVBeTpT11iHw",
+            "dates": [
+                "2012-07-13 21:43:57",
+                "2016-12-24 02:27:31",
+                "2017-08-31 00:35:26"
+            ]
+        },
+        {
+            "business_id": "-0LPtgJC31FWMrMv317p0Q",
+            "dates": [
+                "2013-04-13 12:35:33",
+                "2013-08-19 23:35:49",
+                "2013-10-04 19:14:56"
+            ]
+        },
+        {
+            "business_id": "-0M3o2uWBnQZwd3hmfEwuw",
+            "dates": [
+                "2016-09-10 19:26:19",
+                "2018-09-08 14:15:37",
+                "2019-09-13 22:47:25"
+            ]
+        },
+        {
+            "business_id": "-0RRiWDtfnS16AKCtfvBZg",
+            "dates": [
+                "2017-05-19 14:30:16",
+                "2017-05-19 14:30:25",
+                "2017-08-28 15:49:37",
+                "2017-09-20 20:19:51",
+                "2017-10-01 16:31:05",
+                "2017-10-01 16:56:27",
+                "2017-12-27 23:33:20"
+            ]
+        },
+        {
+            "business_id": "-0Soj75v-XoRcf2ERr8Bmg",
+            "dates": [
+                "2019-06-05 18:22:49"
+            ]
+        },
+        {
+            "business_id": "-0ZumLlFjMh4ZW1z2nXGug",
+            "dates": [
+                "2011-09-24 21:37:32",
+                "2014-03-10 20:20:07",
+                "2015-05-27 00:40:24",
+                "2015-08-29 17:58:15",
+                "2018-03-16 15:03:26"
+            ]
+        },
+        {
+            "business_id": "-0aOudcaAyac0VJbMX-L1g",
+            "dates": [
+                "2015-03-16 23:51:16",
+                "2015-12-21 04:48:01",
+                "2016-10-28 20:22:42",
+                "2016-10-28 20:23:00"
+            ]
+        },
+        {
+            "business_id": "-0b86isaXMY0v4g-V8GZ9Q",
+            "dates": [
+                "2013-10-22 16:49:21",
+                "2014-11-21 17:39:24"
+            ]
+        },
+        {
+            "business_id": "-0d-BfFSU0bwLcnMaGRxYw",
+            "dates": [
+                "2014-08-07 18:30:48",
+                "2014-09-16 20:41:45",
+                "2014-10-12 23:22:27",
+                "2015-07-21 20:43:56",
+                "2015-07-21 20:45:07"
+            ]
+        },
+        {
+            "business_id": "-0jz6c3C6i7RG7Ag22K-Pg",
+            "dates": [
+                "2015-05-02 19:49:05",
+                "2015-05-06 03:52:18",
+                "2015-09-26 01:13:19"
+            ]
+        },
+        {
+            "business_id": "-0y3MZU2oYP8r1ruDP1bfQ",
+            "dates": [
+                "2015-04-11 13:14:14",
+                "2015-11-21 16:05:56",
+                "2016-05-06 14:10:04",
+                "2017-08-09 15:15:10",
+                "2017-10-21 15:12:56"
+            ]
+        },
+        {
+            "business_id": "-1BPe8UjF2_l3nVk-DFUjA",
+            "dates": [
+                "2015-12-03 18:44:00",
+                "2016-03-17 18:19:21",
+                "2016-11-02 15:58:38"
+            ]
+        },
+        {
+            "business_id": "-1E2CQu_38mkghvmZgCCRw",
+            "dates": [
+                "2019-04-04 22:02:37"
+            ]
+        },
+        {
+            "business_id": "-1wzk43IZ5D9Ysu6kzb5xA",
+            "dates": [
+                "2019-02-27 14:03:08"
+            ]
+        },
+        {
+            "business_id": "-23R9P2eG7VTc6DVLjFKzA",
+            "dates": [
+                "2011-12-21 19:02:51",
+                "2012-04-15 04:21:39",
+                "2012-04-15 14:23:56",
+                "2013-06-30 22:39:51",
+                "2013-10-04 20:34:13",
+                "2014-07-16 02:28:40"
+            ]
+        },
+        {
+            "business_id": "-26MGfikhJiTfCI-GqmzhQ",
+            "dates": [
+                "2018-06-13 20:16:07"
+            ]
+        },
+        {
+            "business_id": "-2bLuJsMZ0WhI9daurVQNQ",
+            "dates": [
+                "2015-05-29 16:46:17",
+                "2015-06-01 15:03:53"
+            ]
+        },
+        {
+            "business_id": "-2hDBMaza_ldqnZdiU06LQ",
+            "dates": [
+                "2011-10-08 12:02:23",
+                "2014-08-18 02:11:11",
+                "2016-01-07 05:27:51",
+                "2016-10-21 20:15:55",
+                "2016-12-01 03:57:10",
+                "2016-12-29 01:54:42",
+                "2018-07-22 19:55:31",
+                "2018-09-07 01:42:54",
+                "2019-03-08 03:41:06"
+            ]
+        }
+    ]
+);
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/bulk-loading/on-index-creation/open/use-case-1/use-case-1.3.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/bulk-loading/on-index-creation/open/use-case-1/use-case-1.3.ddl.sqlpp
new file mode 100644
index 0000000..b933d8a
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/bulk-loading/on-index-creation/open/use-case-1/use-case-1.3.ddl.sqlpp
@@ -0,0 +1,22 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+USE TestYelp;
+
+CREATE INDEX IdxYelpCheckinDates ON YelpCheckin (UNNEST dates : string ?) ;
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/bulk-loading/on-index-creation/open/use-case-1/use-case-1.4.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/bulk-loading/on-index-creation/open/use-case-1/use-case-1.4.query.sqlpp
new file mode 100644
index 0000000..ec79bc6
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/bulk-loading/on-index-creation/open/use-case-1/use-case-1.4.query.sqlpp
@@ -0,0 +1,25 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+SET `compiler.arrayindex` "true";
+
+USE TestYelp;
+
+SELECT C.business_id
+FROM YelpCheckin C, C.dates D
+WHERE "2016-04-26 19:49:16" = D;
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/bulk-loading/on-index-creation/open/use-case-2/use-case-2.1.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/bulk-loading/on-index-creation/open/use-case-2/use-case-2.1.ddl.sqlpp
new file mode 100644
index 0000000..01511bb
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/bulk-loading/on-index-creation/open/use-case-2/use-case-2.1.ddl.sqlpp
@@ -0,0 +1,32 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+/*
+ * Description: Verify the bulk-loading operation for a record->array->atomic index.
+ */
+
+DROP DATAVERSE TestYelp IF EXISTS;
+CREATE DATAVERSE TestYelp;
+USE TestYelp;
+
+CREATE TYPE CheckinType AS {
+    checkin_id: uuid
+};
+
+CREATE DATASET YelpCheckin(CheckinType) PRIMARY KEY checkin_id AUTOGENERATED;
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/bulk-loading/on-index-creation/open/use-case-2/use-case-2.2.update.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/bulk-loading/on-index-creation/open/use-case-2/use-case-2.2.update.sqlpp
new file mode 100644
index 0000000..2549e8f
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/bulk-loading/on-index-creation/open/use-case-2/use-case-2.2.update.sqlpp
@@ -0,0 +1,497 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+USE TestYelp;
+
+INSERT INTO YelpCheckin (
+    [
+        {
+            "business_id": "--1UhMGODdWsrMastO9DZw",
+            "checkin_times": {
+                "dates": [
+                    "2016-04-26",
+                    "2016-08-30",
+                    "2016-10-15",
+                    "2016-11-18",
+                    "2017-04-20",
+                    "2017-05-03",
+                    "2019-03-19"
+                ],
+                "times": [
+                    "19:49:16",
+                    "18:36:57",
+                    "02:45:18",
+                    "01:54:50",
+                    "18:39:06",
+                    "17:58:02",
+                    "22:04:48"
+                ]
+            }
+        },
+        {
+            "business_id": "--EF5N7P70J_UYBTPypYlA",
+            "checkin_times": {
+                "dates": [
+                    "2018-05-25",
+                    "2018-09-18",
+                    "2019-10-18"
+                ],
+                "times": [
+                    "19:52:07",
+                    "16:09:44",
+                    "21:29:09"
+                ]
+            }
+        },
+        {
+            "business_id": "--Ni3oJ4VOqfOEu7Sj2Vzg",
+            "checkin_times": {
+                "dates": [
+                    "2019-06-07"
+                ],
+                "times": [
+                    "17:54:58"
+                ]
+            }
+        },
+        {
+            "business_id": "--Y1Adl1YUWfYIRSd8vkmA",
+            "checkin_times": {
+                "dates": [
+                    "2011-05-03",
+                    "2011-08-23",
+                    "2014-12-04",
+                    "2016-11-16"
+                ],
+                "times": [
+                    "20:54:05",
+                    "20:49:45",
+                    "06:13:01",
+                    "19:25:55"
+                ]
+            }
+        },
+        {
+            "business_id": "--YPwqIlRJrhHkJcjY3eiA",
+            "checkin_times": {
+                "dates": [
+                    "2016-06-18",
+                    "2016-10-15"
+                ],
+                "times": [
+                    "21:35:45",
+                    "18:17:51"
+                ]
+            }
+        },
+        {
+            "business_id": "--e8PjCNhEz32pprnPhCwQ",
+            "checkin_times": {
+                "dates": [
+                    "2015-04-02"
+                ],
+                "times": [
+                    "21:45:17"
+                ]
+            }
+        },
+        {
+            "business_id": "--kinfHwmtdjz03g8B8z8Q",
+            "checkin_times": {
+                "dates": [
+                    "2014-08-27",
+                    "2015-12-19",
+                    "2018-11-27"
+                ],
+                "times": [
+                    "17:49:18",
+                    "21:30:31",
+                    "15:53:50"
+                ]
+            }
+        },
+        {
+            "business_id": "--q6datkI-f0EoVheXNEeQ",
+            "checkin_times": {
+                "dates": [
+                    "2014-01-28",
+                    "2014-11-16",
+                    "2015-11-15",
+                    "2015-11-15"
+                ],
+                "times": [
+                    "20:56:04",
+                    "16:11:58",
+                    "19:21:53",
+                    "19:33:39"
+                ]
+            }
+        },
+        {
+            "business_id": "--qvQS4MigHPykD2GV0-zw",
+            "checkin_times": {
+                "dates": [
+                    "2019-04-11"
+                ],
+                "times": [
+                    "18:30:12"
+                ]
+            }
+        },
+        {
+            "business_id": "--wIGbLEhlpl_UeAIyDmZQ",
+            "checkin_times": {
+                "dates": [
+                    "2015-06-06",
+                    "2019-03-14"
+                ],
+                "times": [
+                    "20:01:06",
+                    "22:01:52"
+                ]
+            }
+        },
+        {
+            "business_id": "-0FA-Qdi3SPYIoJz9UQw-A",
+            "checkin_times": {
+                "dates": [
+                    "2018-09-29",
+                    "2018-10-20",
+                    "2018-10-20"
+                ],
+                "times": [
+                    "18:55:17",
+                    "16:48:05",
+                    "22:20:24"
+                ]
+            }
+        },
+        {
+            "business_id": "-0Hj1hb_XW6ybWq2M7QhGA",
+            "checkin_times": {
+                "dates": [
+                    "2011-04-23",
+                    "2014-05-04",
+                    "2014-05-11",
+                    "2014-06-04",
+                    "2015-12-05",
+                    "2017-05-15"
+                ],
+                "times": [
+                    "21:11:22",
+                    "19:42:48",
+                    "19:16:08",
+                    "19:14:18",
+                    "19:22:42",
+                    "23:19:00"
+                ]
+            }
+        },
+        {
+            "business_id": "-0KMvRFwDWdVBeTpT11iHw",
+            "checkin_times": {
+                "dates": [
+                    "2012-07-13",
+                    "2016-12-24",
+                    "2017-08-31"
+                ],
+                "times": [
+                    "21:43:57",
+                    "02:27:31",
+                    "00:35:26"
+                ]
+            }
+        },
+        {
+            "business_id": "-0LPtgJC31FWMrMv317p0Q",
+            "checkin_times": {
+                "dates": [
+                    "2013-04-13",
+                    "2013-08-19",
+                    "2013-10-04"
+                ],
+                "times": [
+                    "12:35:33",
+                    "23:35:49",
+                    "19:14:56"
+                ]
+            }
+        },
+        {
+            "business_id": "-0M3o2uWBnQZwd3hmfEwuw",
+            "checkin_times": {
+                "dates": [
+                    "2016-09-10",
+                    "2018-09-08",
+                    "2019-09-13"
+                ],
+                "times": [
+                    "19:26:19",
+                    "14:15:37",
+                    "22:47:25"
+                ]
+            }
+        },
+        {
+            "business_id": "-0RRiWDtfnS16AKCtfvBZg",
+            "checkin_times": {
+                "dates": [
+                    "2017-05-19",
+                    "2017-05-19",
+                    "2017-08-28",
+                    "2017-09-20",
+                    "2017-10-01",
+                    "2017-10-01",
+                    "2017-12-27"
+                ],
+                "times": [
+                    "14:30:16",
+                    "14:30:25",
+                    "15:49:37",
+                    "20:19:51",
+                    "16:31:05",
+                    "16:56:27",
+                    "23:33:20"
+                ]
+            }
+        },
+        {
+            "business_id": "-0Soj75v-XoRcf2ERr8Bmg",
+            "checkin_times": {
+                "dates": [
+                    "2019-06-05"
+                ],
+                "times": [
+                    "18:22:49"
+                ]
+            }
+        },
+        {
+            "business_id": "-0ZumLlFjMh4ZW1z2nXGug",
+            "checkin_times": {
+                "dates": [
+                    "2011-09-24",
+                    "2014-03-10",
+                    "2015-05-27",
+                    "2015-08-29",
+                    "2018-03-16"
+                ],
+                "times": [
+                    "21:37:32",
+                    "20:20:07",
+                    "00:40:24",
+                    "17:58:15",
+                    "15:03:26"
+                ]
+            }
+        },
+        {
+            "business_id": "-0aOudcaAyac0VJbMX-L1g",
+            "checkin_times": {
+                "dates": [
+                    "2015-03-16",
+                    "2015-12-21",
+                    "2016-10-28",
+                    "2016-10-28"
+                ],
+                "times": [
+                    "23:51:16",
+                    "04:48:01",
+                    "20:22:42",
+                    "20:23:00"
+                ]
+            }
+        },
+        {
+            "business_id": "-0b86isaXMY0v4g-V8GZ9Q",
+            "checkin_times": {
+                "dates": [
+                    "2013-10-22",
+                    "2014-11-21"
+                ],
+                "times": [
+                    "16:49:21",
+                    "17:39:24"
+                ]
+            }
+        },
+        {
+            "business_id": "-0d-BfFSU0bwLcnMaGRxYw",
+            "checkin_times": {
+                "dates": [
+                    "2014-08-07",
+                    "2014-09-16",
+                    "2014-10-12",
+                    "2015-07-21",
+                    "2015-07-21"
+                ],
+                "times": [
+                    "18:30:48",
+                    "20:41:45",
+                    "23:22:27",
+                    "20:43:56",
+                    "20:45:07"
+                ]
+            }
+        },
+        {
+            "business_id": "-0jz6c3C6i7RG7Ag22K-Pg",
+            "checkin_times": {
+                "dates": [
+                    "2015-05-02",
+                    "2015-05-06",
+                    "2015-09-26"
+                ],
+                "times": [
+                    "19:49:05",
+                    "03:52:18",
+                    "01:13:19"
+                ]
+            }
+        },
+        {
+            "business_id": "-0y3MZU2oYP8r1ruDP1bfQ",
+            "checkin_times": {
+                "dates": [
+                    "2015-04-11",
+                    "2015-11-21",
+                    "2016-05-06",
+                    "2017-08-09",
+                    "2017-10-21"
+                ],
+                "times": [
+                    "13:14:14",
+                    "16:05:56",
+                    "14:10:04",
+                    "15:15:10",
+                    "15:12:56"
+                ]
+            }
+        },
+        {
+            "business_id": "-1BPe8UjF2_l3nVk-DFUjA",
+            "checkin_times": {
+                "dates": [
+                    "2015-12-03",
+                    "2016-03-17",
+                    "2016-11-02"
+                ],
+                "times": [
+                    "18:44:00",
+                    "18:19:21",
+                    "15:58:38"
+                ]
+            }
+        },
+        {
+            "business_id": "-1E2CQu_38mkghvmZgCCRw",
+            "checkin_times": {
+                "dates": [
+                    "2019-04-04"
+                ],
+                "times": [
+                    "22:02:37"
+                ]
+            }
+        },
+        {
+            "business_id": "-1wzk43IZ5D9Ysu6kzb5xA",
+            "checkin_times": {
+                "dates": [
+                    "2019-02-27"
+                ],
+                "times": [
+                    "14:03:08"
+                ]
+            }
+        },
+        {
+            "business_id": "-23R9P2eG7VTc6DVLjFKzA",
+            "checkin_times": {
+                "dates": [
+                    "2011-12-21",
+                    "2012-04-15",
+                    "2012-04-15",
+                    "2013-06-30",
+                    "2013-10-04",
+                    "2014-07-16"
+                ],
+                "times": [
+                    "19:02:51",
+                    "04:21:39",
+                    "14:23:56",
+                    "22:39:51",
+                    "20:34:13",
+                    "02:28:40"
+                ]
+            }
+        },
+        {
+            "business_id": "-26MGfikhJiTfCI-GqmzhQ",
+            "checkin_times": {
+                "dates": [
+                    "2018-06-13"
+                ],
+                "times": [
+                    "20:16:07"
+                ]
+            }
+        },
+        {
+            "business_id": "-2bLuJsMZ0WhI9daurVQNQ",
+            "checkin_times": {
+                "dates": [
+                    "2015-05-29",
+                    "2015-06-01"
+                ],
+                "times": [
+                    "16:46:17",
+                    "15:03:53"
+                ]
+            }
+        },
+        {
+            "business_id": "-2hDBMaza_ldqnZdiU06LQ",
+            "checkin_times": {
+                "dates": [
+                    "2011-10-08",
+                    "2014-08-18",
+                    "2016-01-07",
+                    "2016-10-21",
+                    "2016-12-01",
+                    "2016-12-29",
+                    "2018-07-22",
+                    "2018-09-07",
+                    "2019-03-08"
+                ],
+                "times": [
+                    "12:02:23",
+                    "02:11:11",
+                    "05:27:51",
+                    "20:15:55",
+                    "03:57:10",
+                    "01:54:42",
+                    "19:55:31",
+                    "01:42:54",
+                    "03:41:06"
+                ]
+            }
+        }
+    ]
+);
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/bulk-loading/on-index-creation/open/use-case-2/use-case-2.3.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/bulk-loading/on-index-creation/open/use-case-2/use-case-2.3.ddl.sqlpp
new file mode 100644
index 0000000..51a1c82
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/bulk-loading/on-index-creation/open/use-case-2/use-case-2.3.ddl.sqlpp
@@ -0,0 +1,22 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+USE TestYelp;
+
+CREATE INDEX IdxYelpCheckinDates ON YelpCheckin (UNNEST checkin_times.dates : string ?) ;
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/bulk-loading/on-index-creation/open/use-case-2/use-case-2.4.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/bulk-loading/on-index-creation/open/use-case-2/use-case-2.4.query.sqlpp
new file mode 100644
index 0000000..0f23cd4
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/bulk-loading/on-index-creation/open/use-case-2/use-case-2.4.query.sqlpp
@@ -0,0 +1,25 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+SET `compiler.arrayindex` "true";
+
+USE TestYelp;
+
+SELECT C.business_id
+FROM YelpCheckin C, C.checkin_times.dates D
+WHERE "2016-04-26" = D;
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/bulk-loading/on-index-creation/open/use-case-3/use-case-3.1.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/bulk-loading/on-index-creation/open/use-case-3/use-case-3.1.ddl.sqlpp
new file mode 100644
index 0000000..e2340e0
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/bulk-loading/on-index-creation/open/use-case-3/use-case-3.1.ddl.sqlpp
@@ -0,0 +1,32 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+/*
+ * Description: Verify the bulk-loading operation for an array->record->atomic index, as well as a composite index on both fields. 
+ */
+
+DROP DATAVERSE TestYelp IF EXISTS;
+CREATE DATAVERSE TestYelp;
+USE TestYelp;
+
+CREATE TYPE CheckinType AS {
+    checkin_id: uuid
+};
+
+CREATE DATASET YelpCheckin(CheckinType) PRIMARY KEY checkin_id AUTOGENERATED;
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/bulk-loading/on-index-creation/open/use-case-3/use-case-3.2.update.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/bulk-loading/on-index-creation/open/use-case-3/use-case-3.2.update.sqlpp
new file mode 100644
index 0000000..02ddf21
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/bulk-loading/on-index-creation/open/use-case-3/use-case-3.2.update.sqlpp
@@ -0,0 +1,276 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+USE TestYelp;
+
+INSERT INTO YelpCheckin (
+    [
+        {
+            "business_id": "--1UhMGODdWsrMastO9DZw",
+            "checkin_times": [ 
+                {"date": "2016-04-26", "time": "19:49:16"},
+                {"date": "2016-08-30", "time": "18:36:57"},
+                {"date": "2016-10-15", "time": "02:45:18"},
+                {"date": "2016-11-18", "time": "01:54:50"},
+                {"date": "2017-04-20", "time": "18:39:06"},
+                {"date": "2017-05-03", "time": "17:58:02"},
+                {"date": "2019-03-19", "time": "22:04:48"}
+             ]
+        },
+        {
+            "business_id": "--EF5N7P70J_UYBTPypYlA",
+            "checkin_times": [ 
+                {"date": "2018-05-25", "time": "19:52:07"},
+                {"date": "2018-09-18", "time": "16:09:44"},
+                {"date": "2019-10-18", "time": "21:29:09"}
+             ]
+        },
+        {
+            "business_id": "--Ni3oJ4VOqfOEu7Sj2Vzg",
+            "checkin_times": [ 
+                {"date": "2019-06-07", "time": "17:54:58"}
+             ]
+        },
+        {
+            "business_id": "--Y1Adl1YUWfYIRSd8vkmA",
+            "checkin_times": [ 
+                {"date": "2011-05-03", "time": "20:54:05"},
+                {"date": "2011-08-23", "time": "20:49:45"},
+                {"date": "2014-12-04", "time": "06:13:01"},
+                {"date": "2016-11-16", "time": "19:25:55"}
+             ]
+        },
+        {
+            "business_id": "--YPwqIlRJrhHkJcjY3eiA",
+            "checkin_times": [ 
+                {"date": "2016-06-18", "time": "21:35:45"},
+                {"date": "2016-10-15", "time": "18:17:51"}
+             ]
+        },
+        {
+            "business_id": "--e8PjCNhEz32pprnPhCwQ",
+            "checkin_times": [ 
+                {"date": "2015-04-02", "time": "21:45:17"}
+             ]
+        },
+        {
+            "business_id": "--kinfHwmtdjz03g8B8z8Q",
+            "checkin_times": [ 
+                {"date": "2014-08-27", "time": "17:49:18"},
+                {"date": "2015-12-19", "time": "21:30:31"},
+                {"date": "2018-11-27", "time": "15:53:50"}
+             ]
+        },
+        {
+            "business_id": "--q6datkI-f0EoVheXNEeQ",
+            "checkin_times": [ 
+                {"date": "2014-01-28", "time": "20:56:04"},
+                {"date": "2014-11-16", "time": "16:11:58"},
+                {"date": "2015-11-15", "time": "19:21:53"},
+                {"date": "2015-11-15", "time": "19:33:39"}
+             ]
+        },
+        {
+            "business_id": "--qvQS4MigHPykD2GV0-zw",
+            "checkin_times": [ 
+                {"date": "2019-04-11", "time": "18:30:12"}
+             ]
+        },
+        {
+            "business_id": "--wIGbLEhlpl_UeAIyDmZQ",
+            "checkin_times": [ 
+                {"date": "2015-06-06", "time": "20:01:06"},
+                {"date": "2019-03-14", "time": "22:01:52"}
+             ]
+        },
+        {
+            "business_id": "-0FA-Qdi3SPYIoJz9UQw-A",
+            "checkin_times": [ 
+                {"date": "2018-09-29", "time": "18:55:17"},
+                {"date": "2018-10-20", "time": "16:48:05"},
+                {"date": "2018-10-20", "time": "22:20:24"}
+             ]
+        },
+        {
+            "business_id": "-0Hj1hb_XW6ybWq2M7QhGA",
+            "checkin_times": [ 
+                {"date": "2011-04-23", "time": "21:11:22"},
+                {"date": "2014-05-04", "time": "19:42:48"},
+                {"date": "2014-05-11", "time": "19:16:08"},
+                {"date": "2014-06-04", "time": "19:14:18"},
+                {"date": "2015-12-05", "time": "19:22:42"},
+                {"date": "2017-05-15", "time": "23:19:00"}
+             ]
+        },
+        {
+            "business_id": "-0KMvRFwDWdVBeTpT11iHw",
+            "checkin_times": [ 
+                {"date": "2012-07-13", "time": "21:43:57"},
+                {"date": "2016-12-24", "time": "02:27:31"},
+                {"date": "2017-08-31", "time": "00:35:26"}
+             ]
+        },
+        {
+            "business_id": "-0LPtgJC31FWMrMv317p0Q",
+            "checkin_times": [ 
+                {"date": "2013-04-13", "time": "12:35:33"},
+                {"date": "2013-08-19", "time": "23:35:49"},
+                {"date": "2013-10-04", "time": "19:14:56"}
+             ]
+        },
+        {
+            "business_id": "-0M3o2uWBnQZwd3hmfEwuw",
+            "checkin_times": [ 
+                {"date": "2016-09-10", "time": "19:26:19"},
+                {"date": "2018-09-08", "time": "14:15:37"},
+                {"date": "2019-09-13", "time": "22:47:25"}
+             ]
+        },
+        {
+            "business_id": "-0RRiWDtfnS16AKCtfvBZg",
+            "checkin_times": [ 
+                {"date": "2017-05-19", "time": "14:30:16"},
+                {"date": "2017-05-19", "time": "14:30:25"},
+                {"date": "2017-08-28", "time": "15:49:37"},
+                {"date": "2017-09-20", "time": "20:19:51"},
+                {"date": "2017-10-01", "time": "16:31:05"},
+                {"date": "2017-10-01", "time": "16:56:27"},
+                {"date": "2017-12-27", "time": "23:33:20"}
+             ]
+        },
+        {
+            "business_id": "-0Soj75v-XoRcf2ERr8Bmg",
+            "checkin_times": [ 
+                {"date": "2019-06-05", "time": "18:22:49"}
+             ]
+        },
+        {
+            "business_id": "-0ZumLlFjMh4ZW1z2nXGug",
+            "checkin_times": [ 
+                {"date": "2011-09-24", "time": "21:37:32"},
+                {"date": "2014-03-10", "time": "20:20:07"},
+                {"date": "2015-05-27", "time": "00:40:24"},
+                {"date": "2015-08-29", "time": "17:58:15"},
+                {"date": "2018-03-16", "time": "15:03:26"}
+             ]
+        },
+        {
+            "business_id": "-0aOudcaAyac0VJbMX-L1g",
+            "checkin_times": [ 
+                {"date": "2015-03-16", "time": "23:51:16"},
+                {"date": "2015-12-21", "time": "04:48:01"},
+                {"date": "2016-10-28", "time": "20:22:42"},
+                {"date": "2016-10-28", "time": "20:23:00"}
+             ]
+        },
+        {
+            "business_id": "-0b86isaXMY0v4g-V8GZ9Q",
+            "checkin_times": [ 
+                {"date": "2013-10-22", "time": "16:49:21"},
+                {"date": "2014-11-21", "time": "17:39:24"}
+             ]
+        },
+        {
+            "business_id": "-0d-BfFSU0bwLcnMaGRxYw",
+            "checkin_times": [ 
+                {"date": "2014-08-07", "time": "18:30:48"},
+                {"date": "2014-09-16", "time": "20:41:45"},
+                {"date": "2014-10-12", "time": "23:22:27"},
+                {"date": "2015-07-21", "time": "20:43:56"},
+                {"date": "2015-07-21", "time": "20:45:07"}
+             ]
+        },
+        {
+            "business_id": "-0jz6c3C6i7RG7Ag22K-Pg",
+            "checkin_times": [ 
+                {"date": "2015-05-02", "time": "19:49:05"},
+                {"date": "2015-05-06", "time": "03:52:18"},
+                {"date": "2015-09-26", "time": "01:13:19"}
+             ]
+        },
+        {
+            "business_id": "-0y3MZU2oYP8r1ruDP1bfQ",
+            "checkin_times": [ 
+                {"date": "2015-04-11", "time": "13:14:14"},
+                {"date": "2015-11-21", "time": "16:05:56"},
+                {"date": "2016-05-06", "time": "14:10:04"},
+                {"date": "2017-08-09", "time": "15:15:10"},
+                {"date": "2017-10-21", "time": "15:12:56"}
+             ]
+        },
+        {
+            "business_id": "-1BPe8UjF2_l3nVk-DFUjA",
+            "checkin_times": [ 
+                {"date": "2015-12-03", "time": "18:44:00"},
+                {"date": "2016-03-17", "time": "18:19:21"},
+                {"date": "2016-11-02", "time": "15:58:38"}
+             ]
+        },
+        {
+            "business_id": "-1E2CQu_38mkghvmZgCCRw",
+            "checkin_times": [ 
+                {"date": "2019-04-04", "time": "22:02:37"}
+             ]
+        },
+        {
+            "business_id": "-1wzk43IZ5D9Ysu6kzb5xA",
+            "checkin_times": [ 
+                {"date": "2019-02-27", "time": "14:03:08"}
+             ]
+        },
+        {
+            "business_id": "-23R9P2eG7VTc6DVLjFKzA",
+            "checkin_times": [ 
+                {"date": "2011-12-21", "time": "19:02:51"},
+                {"date": "2012-04-15", "time": "04:21:39"},
+                {"date": "2012-04-15", "time": "14:23:56"},
+                {"date": "2013-06-30", "time": "22:39:51"},
+                {"date": "2013-10-04", "time": "20:34:13"},
+                {"date": "2014-07-16", "time": "02:28:40"}
+             ]
+        },
+        {
+            "business_id": "-26MGfikhJiTfCI-GqmzhQ",
+            "checkin_times": [ 
+                {"date": "2018-06-13", "time": "20:16:07"}
+             ]
+        },
+        {
+            "business_id": "-2bLuJsMZ0WhI9daurVQNQ",
+            "checkin_times": [ 
+                {"date": "2015-05-29", "time": "16:46:17"},
+                {"date": "2015-06-01", "time": "15:03:53"}
+             ]
+        },
+        {
+            "business_id": "-2hDBMaza_ldqnZdiU06LQ",
+            "checkin_times": [ 
+                {"date": "2011-10-08", "time": "12:02:23"},
+                {"date": "2014-08-18", "time": "02:11:11"},
+                {"date": "2016-01-07", "time": "05:27:51"},
+                {"date": "2016-10-21", "time": "20:15:55"},
+                {"date": "2016-12-01", "time": "03:57:10"},
+                {"date": "2016-12-29", "time": "01:54:42"},
+                {"date": "2018-07-22", "time": "19:55:31"},
+                {"date": "2018-09-07", "time": "01:42:54"},
+                {"date": "2019-03-08", "time": "03:41:06"}
+             ]
+        }
+    ]
+);
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/bulk-loading/on-index-creation/open/use-case-3/use-case-3.3.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/bulk-loading/on-index-creation/open/use-case-3/use-case-3.3.ddl.sqlpp
new file mode 100644
index 0000000..aa50e82
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/bulk-loading/on-index-creation/open/use-case-3/use-case-3.3.ddl.sqlpp
@@ -0,0 +1,22 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+USE TestYelp;
+
+CREATE INDEX IdxYelpCheckinDates ON YelpCheckin (UNNEST checkin_times SELECT date : string ?) ;
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/bulk-loading/on-index-creation/open/use-case-3/use-case-3.4.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/bulk-loading/on-index-creation/open/use-case-3/use-case-3.4.query.sqlpp
new file mode 100644
index 0000000..27f01aa
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/bulk-loading/on-index-creation/open/use-case-3/use-case-3.4.query.sqlpp
@@ -0,0 +1,25 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+SET `compiler.arrayindex` "true";
+
+USE TestYelp;
+
+SELECT C.business_id
+FROM YelpCheckin C, C.checkin_times CT
+WHERE "2016-04-26" = CT.date;
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/bulk-loading/on-index-creation/open/use-case-3/use-case-3.5.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/bulk-loading/on-index-creation/open/use-case-3/use-case-3.5.ddl.sqlpp
new file mode 100644
index 0000000..408a465
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/bulk-loading/on-index-creation/open/use-case-3/use-case-3.5.ddl.sqlpp
@@ -0,0 +1,23 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+USE TestYelp;
+
+DROP INDEX YelpCheckin.IdxYelpCheckinDates;
+CREATE INDEX IdxYelpCheckinDates ON YelpCheckin (UNNEST checkin_times SELECT date : string ?, time : string ?) ;
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/bulk-loading/on-index-creation/open/use-case-3/use-case-3.6.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/bulk-loading/on-index-creation/open/use-case-3/use-case-3.6.query.sqlpp
new file mode 100644
index 0000000..27f01aa
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/bulk-loading/on-index-creation/open/use-case-3/use-case-3.6.query.sqlpp
@@ -0,0 +1,25 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+SET `compiler.arrayindex` "true";
+
+USE TestYelp;
+
+SELECT C.business_id
+FROM YelpCheckin C, C.checkin_times CT
+WHERE "2016-04-26" = CT.date;
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/bulk-loading/on-index-creation/open/use-case-4/use-case-4.1.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/bulk-loading/on-index-creation/open/use-case-4/use-case-4.1.ddl.sqlpp
new file mode 100644
index 0000000..3487452
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/bulk-loading/on-index-creation/open/use-case-4/use-case-4.1.ddl.sqlpp
@@ -0,0 +1,32 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+/*
+ * Description: Verify the bulk-loading operation for an array->record->array->atomic index.
+ */
+
+DROP DATAVERSE TestYelp IF EXISTS;
+CREATE DATAVERSE TestYelp;
+USE TestYelp;
+
+CREATE TYPE CheckinType AS {
+    checkin_id: uuid
+};
+
+CREATE DATASET YelpCheckin(CheckinType) PRIMARY KEY checkin_id AUTOGENERATED;
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/bulk-loading/on-index-creation/open/use-case-4/use-case-4.2.update.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/bulk-loading/on-index-creation/open/use-case-4/use-case-4.2.update.sqlpp
new file mode 100644
index 0000000..7fcf9e4
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/bulk-loading/on-index-creation/open/use-case-4/use-case-4.2.update.sqlpp
@@ -0,0 +1,425 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+USE TestYelp;
+
+INSERT INTO YelpCheckin (
+    [
+        {
+            "business_id": "--1UhMGODdWsrMastO9DZw",
+            "checkin_times": [ 
+                {
+                    "dates": ["2016-04-26", "2016-08-30", "2016-10-15", "2016-11-18"],
+                    "times": ["19:49:16", "18:36:57", "02:45:18", "01:54:50"]
+                },
+                {
+                    "dates": ["2017-04-20", "2017-05-03"],
+                    "times": ["18:39:06", "17:58:02"]
+                },
+                {
+                    "dates": ["2019-03-19"],
+                    "times": ["22:04:48"]
+                }
+             ]
+        },
+        {
+            "business_id": "--EF5N7P70J_UYBTPypYlA",
+            "checkin_times": [ 
+                {
+                    "dates": ["2018-05-25", "2018-09-18"],
+                    "times": ["19:52:07", "16:09:44"]
+                },
+                {
+                    "dates": ["2019-10-18"],
+                    "times": ["21:29:09"]
+                }
+             ]
+        },
+        {
+            "business_id": "--Ni3oJ4VOqfOEu7Sj2Vzg",
+            "checkin_times": [ 
+                {
+                    "dates": ["2019-06-07"],
+                    "times": ["17:54:58"]
+                }
+             ]
+        },
+        {
+            "business_id": "--Y1Adl1YUWfYIRSd8vkmA",
+            "checkin_times": [ 
+                {
+                    "dates": ["2011-05-03", "2011-08-23"],
+                    "times": ["20:54:05", "20:49:45"]
+                },
+                {
+                    "dates": ["2014-12-04"],
+                    "times": ["06:13:01"]
+                },
+                {
+                    "dates": ["2016-11-16"],
+                    "times": ["19:25:55"]
+                }
+             ]
+        },
+        {
+            "business_id": "--YPwqIlRJrhHkJcjY3eiA",
+            "checkin_times": [ 
+               {
+                    "dates": ["2016-06-18", "2016-10-15"],
+                    "times": ["21:35:45", "18:17:51"]
+                }
+             ]
+        },
+        {
+            "business_id": "--e8PjCNhEz32pprnPhCwQ",
+            "checkin_times": [ 
+                {
+                    "dates": ["2015-04-02"],
+                    "times": ["21:45:17"]
+                }
+             ]
+        },
+        {
+            "business_id": "--kinfHwmtdjz03g8B8z8Q",
+            "checkin_times": [ 
+                {
+                    "dates": ["2014-08-27"],
+                    "times": ["17:49:18"]
+                },
+                {
+                    "dates": ["2015-12-19"],
+                    "times": ["21:30:31"]
+                },
+                {
+                    "dates": ["2018-11-27"],
+                    "times": ["15:53:50"]
+                }
+             ]
+        },
+        {
+            "business_id": "--q6datkI-f0EoVheXNEeQ",
+            "checkin_times": [ 
+                {
+                    "dates": ["2014-01-28", "2014-11-16"],
+                    "times": ["20:56:04", "16:11:58"]
+                },
+                {
+                    "dates": ["2015-11-15", "2015-11-15"],
+                    "times": ["19:21:53", "19:33:39"]
+                }
+             ]
+        },
+        {
+            "business_id": "--qvQS4MigHPykD2GV0-zw",
+            "checkin_times": [ 
+                {
+                    "dates": ["2019-04-11"],
+                    "times": ["18:30:12"]
+                }
+             ]
+        },
+        {
+            "business_id": "--wIGbLEhlpl_UeAIyDmZQ",
+            "checkin_times": [ 
+                {
+                    "dates": ["2015-06-06"],
+                    "times": ["20:01:06"]
+                },
+                {
+                    "dates": ["2019-03-14"],
+                    "times": ["22:01:52"]
+                }
+             ]
+        },
+        {
+            "business_id": "-0FA-Qdi3SPYIoJz9UQw-A",
+            "checkin_times": [ 
+                {
+                    "dates": ["2018-09-29", "2018-10-20", "2018-10-20"],
+                    "times": ["18:55:17", "16:48:05", "22:20:24"]
+                }
+             ]
+        },
+        {
+            "business_id": "-0Hj1hb_XW6ybWq2M7QhGA",
+            "checkin_times": [ 
+                {
+                    "dates": ["2011-04-23"],
+                    "times": ["21:11:22"]
+                },
+                {
+                    "dates": ["2014-05-04", "2014-05-11", "2014-06-04"],
+                    "times": ["19:42:48", "19:16:08", "19:14:18"]
+                },
+                {
+                    "dates": ["2015-12-05"],
+                    "times": ["19:22:42"]
+                },
+                {
+                    "dates": ["2017-05-15"],
+                    "times": ["23:19:00"]
+                }
+             ]
+        },
+        {
+            "business_id": "-0KMvRFwDWdVBeTpT11iHw",
+            "checkin_times": [ 
+                {
+                    "dates": ["2012-07-13"],
+                    "times": ["21:43:57"]
+                },
+                {
+                    "dates": ["2016-12-24"],
+                    "times": ["02:27:31"]
+                },
+                {
+                    "dates": ["2017-08-31"],
+                    "times": ["00:35:26"]
+                }
+             ]
+        },
+        {
+            "business_id": "-0LPtgJC31FWMrMv317p0Q",
+            "checkin_times": [ 
+                {
+                    "dates": ["2013-04-13", "2013-08-19", "2013-10-04"],
+                    "times": ["12:35:33", "23:35:49", "19:14:56"]
+                }
+
+             ]
+        },
+        {
+            "business_id": "-0M3o2uWBnQZwd3hmfEwuw",
+            "checkin_times": [ 
+                {
+                    "dates": ["2016-09-10"],
+                    "times": ["19:26:19"]
+                },
+                {
+                    "dates": ["2018-09-08"],
+                    "times": ["14:15:37"]
+                },
+                {
+                    "dates": ["2019-09-13"],
+                    "times": ["22:47:25"]
+                }
+             ]
+        },
+        {
+            "business_id": "-0RRiWDtfnS16AKCtfvBZg",
+            "checkin_times": [ 
+                 {
+                    "dates": ["2017-05-19", "2017-05-19", "2017-08-28", "2017-09-20", "2017-10-01", "2017-10-01", "2017-12-27"],
+                    "times": ["14:30:16", "14:30:25", "15:49:37", "20:19:51", "16:31:05", "16:56:27", "23:33:20"]
+                }
+             ]
+        },
+        {
+            "business_id": "-0Soj75v-XoRcf2ERr8Bmg",
+            "checkin_times": [ 
+                {
+                    "dates": ["2019-06-05"],
+                    "times": ["18:22:49"]
+                }
+             ]
+        },
+        {
+            "business_id": "-0ZumLlFjMh4ZW1z2nXGug",
+            "checkin_times": [ 
+                {
+                    "dates": ["2011-09-24"],
+                    "times": ["21:37:32"]
+                },
+                {
+                    "dates": ["2014-03-10"],
+                    "times": ["20:20:07"]
+                },
+                {
+                    "dates": ["2015-05-27", "2015-08-29"],
+                    "times": ["00:40:24", "17:58:15"]
+                },
+                {
+                    "dates": ["2018-03-16"],
+                    "times": ["15:03:26"]
+                }
+             ]
+        },
+        {
+            "business_id": "-0aOudcaAyac0VJbMX-L1g",
+            "checkin_times": [ 
+                {
+                    "dates": ["2015-03-16", "2015-12-21"],
+                    "times": ["23:51:16", "04:48:01"]
+                },
+                {
+                    "dates": ["2016-10-28", "2016-10-28"],
+                    "times": ["20:22:42", "20:23:00"]
+                }
+             ]
+        },
+        {
+            "business_id": "-0b86isaXMY0v4g-V8GZ9Q",
+            "checkin_times": [ 
+                {
+                    "dates": ["2013-10-22"],
+                    "times": ["16:49:21"]
+                },
+                {
+                    "dates": ["2014-11-21"],
+                    "times": ["17:39:24"]
+                }
+             ]
+        },
+        {
+            "business_id": "-0d-BfFSU0bwLcnMaGRxYw",
+            "checkin_times": [ 
+                {
+                    "dates": ["2014-08-07", "2014-09-16", "2014-10-12"],
+                    "times": ["18:30:48", "20:41:45", "23:22:27"]
+                },
+                {
+                    "dates": ["2015-07-21", "2015-07-21"],
+                    "times": ["20:43:56", "20:45:07"]
+                }
+             ]
+        },
+        {
+            "business_id": "-0jz6c3C6i7RG7Ag22K-Pg",
+            "checkin_times": [ 
+                {
+                    "dates": ["2015-05-02", "2015-05-06", "2015-09-26"],
+                    "times": ["19:49:05", "03:52:18", "01:13:19"]
+                }
+
+             ]
+        },
+        {
+            "business_id": "-0y3MZU2oYP8r1ruDP1bfQ",
+            "checkin_times": [ 
+                {
+                    "dates": ["2015-04-11", "2015-11-21"],
+                    "times": ["13:14:14", "16:05:56"]
+                },
+                {
+                    "dates": ["2016-05-06"],
+                    "times": ["14:10:04"]
+                },
+                {
+                    "dates": ["2017-08-09", "2017-10-21"],
+                    "times": ["15:15:10", "15:12:56"]
+                }
+             ]
+        },
+        {
+            "business_id": "-1BPe8UjF2_l3nVk-DFUjA",
+            "checkin_times": [ 
+                {
+                    "dates": ["2015-12-03"],
+                    "times": ["18:44:00"]
+                },
+                {
+                    "dates": ["2016-03-17", "2016-11-02"],
+                    "times": ["18:19:21", "15:58:38"]
+                }
+             ]
+        },
+        {
+            "business_id": "-1E2CQu_38mkghvmZgCCRw",
+            "checkin_times": [ 
+                {
+                    "dates": ["2019-04-04"],
+                    "times": ["22:02:37"]
+                }
+             ]
+        },
+        {
+            "business_id": "-1wzk43IZ5D9Ysu6kzb5xA",
+            "checkin_times": [ 
+                {
+                    "dates": ["2019-02-27"],
+                    "times": ["14:03:08"]
+                }
+             ]
+        },
+        {
+            "business_id": "-23R9P2eG7VTc6DVLjFKzA",
+            "checkin_times": [ 
+                {
+                    "dates": ["2011-12-21"],
+                    "times": ["19:02:51"]
+                },
+                {
+                    "dates": ["2012-04-15", "2012-04-15"],
+                    "times": ["04:21:39", "14:23:56"]
+                },
+                {
+                    "dates": ["2013-06-30", "2013-10-04"],
+                    "times": ["22:39:51", "20:34:13"]
+                },
+                {
+                    "dates": ["2014-07-16"],
+                    "times": ["02:28:40"]
+                }
+             ]
+        },
+        {
+            "business_id": "-26MGfikhJiTfCI-GqmzhQ",
+            "checkin_times": [ 
+                {
+                    "dates": ["2018-06-13"],
+                    "times": ["20:16:07"]
+                }
+             ]
+        },
+        {
+            "business_id": "-2bLuJsMZ0WhI9daurVQNQ",
+            "checkin_times": [ 
+                {
+                    "dates": ["2015-05-29", "2015-06-01"],
+                    "times": ["16:46:17", "15:03:53"]
+                }
+             ]
+        },
+        {
+            "business_id": "-2hDBMaza_ldqnZdiU06LQ",
+            "checkin_times": [ 
+                {
+                    "dates": ["2011-10-08"],
+                    "times": ["12:02:23"]
+                },
+                {
+                    "dates": ["2014-08-18"],
+                    "times": ["02:11:11"]
+                },
+                {
+                    "dates": ["2016-01-07", "2016-10-21", "2016-12-01", "2016-12-29"],
+                    "times": ["05:27:51", "20:15:55", "03:57:10", "01:54:42"]
+                },
+                {
+                    "dates": ["2018-07-22", "2018-09-07"],
+                    "times": ["19:55:31", "01:42:54"]
+                },
+                {
+                    "dates": ["2019-03-08"],
+                    "times": ["03:41:06"]
+                }
+             ]
+        }
+    ]
+);
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/bulk-loading/on-index-creation/open/use-case-4/use-case-4.3.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/bulk-loading/on-index-creation/open/use-case-4/use-case-4.3.ddl.sqlpp
new file mode 100644
index 0000000..b349a8a
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/bulk-loading/on-index-creation/open/use-case-4/use-case-4.3.ddl.sqlpp
@@ -0,0 +1,22 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+USE TestYelp;
+
+CREATE INDEX IdxYelpCheckinDates ON YelpCheckin (UNNEST checkin_times UNNEST dates : string ?) ;
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/bulk-loading/on-index-creation/open/use-case-4/use-case-4.4.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/bulk-loading/on-index-creation/open/use-case-4/use-case-4.4.query.sqlpp
new file mode 100644
index 0000000..8ba650f
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/bulk-loading/on-index-creation/open/use-case-4/use-case-4.4.query.sqlpp
@@ -0,0 +1,25 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+SET `compiler.arrayindex` "true";
+
+USE TestYelp;
+
+SELECT C.business_id
+FROM YelpCheckin C, C.checkin_times CT, CT.dates D
+WHERE "2016-04-26" = D;
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/error-handling/index-mixed-composite/index-mixed-composite.1.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/error-handling/index-mixed-composite/index-mixed-composite.1.ddl.sqlpp
new file mode 100644
index 0000000..c26bf92
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/error-handling/index-mixed-composite/index-mixed-composite.1.ddl.sqlpp
@@ -0,0 +1,39 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+/*
+ * Description: Verify that an error is thrown when trying to create a mixed composite atomic and array index (not supported for now).
+ */
+
+DROP DATAVERSE TestYelp IF EXISTS;
+CREATE DATAVERSE TestYelp;
+USE TestYelp;
+
+CREATE TYPE CheckinTimeType AS {
+    time: string
+};
+CREATE TYPE CheckinType AS {
+    checkin_id: uuid,
+    business_id: string,
+    checkin_times: [CheckinTimeType]
+};
+
+CREATE DATASET YelpCheckin(CheckinType) PRIMARY KEY checkin_id AUTOGENERATED;
+CREATE INDEX IdxYelpCheckinTimeBusiness ON YelpCheckin ((UNNEST checkin_times SELECT time), (business_id));
+
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/error-handling/index-on-closed-array/index-on-closed-array.1.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/error-handling/index-on-closed-array/index-on-closed-array.1.ddl.sqlpp
new file mode 100644
index 0000000..65cdcfe
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/error-handling/index-on-closed-array/index-on-closed-array.1.ddl.sqlpp
@@ -0,0 +1,39 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+/*
+ * Description: Verify that an error is thrown when trying to create a typed index on a non-existent closed field within an array.
+ */
+
+DROP DATAVERSE TestYelp IF EXISTS;
+CREATE DATAVERSE TestYelp;
+USE TestYelp;
+
+CREATE TYPE CheckinTimeType AS CLOSED {
+    time: string
+};
+CREATE TYPE CheckinType AS CLOSED {
+    checkin_id: uuid,
+    business_id: string,
+    checkin_times: [CheckinTimeType]
+};
+
+CREATE DATASET YelpCheckin(CheckinType) PRIMARY KEY checkin_id AUTOGENERATED;
+CREATE INDEX IdxYelpCheckinDates ON YelpCheckin (UNNEST checkin_times SELECT date : string?);
+
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/error-handling/index-two-array-fields/index-two-array-fields.1.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/error-handling/index-two-array-fields/index-two-array-fields.1.ddl.sqlpp
new file mode 100644
index 0000000..83d5407
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/error-handling/index-two-array-fields/index-two-array-fields.1.ddl.sqlpp
@@ -0,0 +1,37 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+/*
+ * Description: Verify that we are not able to index more than one array field at different depths.
+ */
+
+DROP DATAVERSE TestYelp IF EXISTS;
+CREATE DATAVERSE TestYelp;
+USE TestYelp;
+
+CREATE TYPE CheckinType AS {
+    checkin_id: uuid,
+    business_id: string,
+    dates: [string],
+    times: [string]
+};
+
+CREATE DATASET YelpCheckin(CheckinType) PRIMARY KEY checkin_id AUTOGENERATED;
+CREATE INDEX IdxYelpCheckinDates ON YelpCheckin (UNNEST dates, UNNEST times);
+
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/error-handling/index-two-array-fields/index-two-array-fields.2.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/error-handling/index-two-array-fields/index-two-array-fields.2.ddl.sqlpp
new file mode 100644
index 0000000..d610c14
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/error-handling/index-two-array-fields/index-two-array-fields.2.ddl.sqlpp
@@ -0,0 +1,34 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+DROP DATAVERSE TestYelp IF EXISTS;
+CREATE DATAVERSE TestYelp;
+USE TestYelp;
+
+CREATE TYPE CheckinType AS {
+    checkin_id: uuid,
+    business_id: string,
+    checkin_times: [{
+        date: string,
+        times: [string]
+    }]
+};
+
+CREATE DATASET YelpCheckin(CheckinType) PRIMARY KEY checkin_id AUTOGENERATED;
+CREATE INDEX IdxYelpCheckinDates ON YelpCheckin ((UNNEST checkin_times SELECT date), UNNEST checkin_times UNNEST times);
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/error-handling/index-with-enforced-type/index-with-enforced-type.1.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/error-handling/index-with-enforced-type/index-with-enforced-type.1.ddl.sqlpp
new file mode 100644
index 0000000..45ecfbe
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/error-handling/index-with-enforced-type/index-with-enforced-type.1.ddl.sqlpp
@@ -0,0 +1,39 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+/*
+ * Description: Verify that an error is thrown when trying to create an enforced typed index.
+ */
+
+DROP DATAVERSE TestYelp IF EXISTS;
+CREATE DATAVERSE TestYelp;
+USE TestYelp;
+
+CREATE TYPE CheckinTimeType AS {
+    time: string
+};
+CREATE TYPE CheckinType AS {
+    checkin_id: uuid,
+    business_id: string,
+    checkin_times: [CheckinTimeType]
+};
+
+CREATE DATASET YelpCheckin(CheckinType) PRIMARY KEY checkin_id AUTOGENERATED;
+CREATE INDEX IdxYelpCheckinDates ON YelpCheckin (UNNEST checkin_times SELECT date : string?) ENFORCED;
+
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/error-handling/invalid-array-path/invalid-array-path.1.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/error-handling/invalid-array-path/invalid-array-path.1.ddl.sqlpp
new file mode 100644
index 0000000..2402382
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/error-handling/invalid-array-path/invalid-array-path.1.ddl.sqlpp
@@ -0,0 +1,39 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+/*
+ * Description: Verify that a specified path that is valid through record-pathing, but invalid through array pathing.
+ */
+
+DROP DATAVERSE TestYelp IF EXISTS;
+CREATE DATAVERSE TestYelp;
+USE TestYelp;
+
+CREATE TYPE CheckinType AS {
+    checkin_id: uuid,
+    business_id: string,
+    checkin_time: {
+        date: string,
+        time: string
+    }
+};
+
+CREATE DATASET YelpCheckin(CheckinType) PRIMARY KEY checkin_id AUTOGENERATED;
+CREATE INDEX IdxYelpCheckinDates ON YelpCheckin (UNNEST checkin_time SELECT date);
+
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/error-handling/invalid-array-path/invalid-array-path.2.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/error-handling/invalid-array-path/invalid-array-path.2.ddl.sqlpp
new file mode 100644
index 0000000..d658dbe
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/error-handling/invalid-array-path/invalid-array-path.2.ddl.sqlpp
@@ -0,0 +1,39 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+/*
+ * Description: Verify that a specified path that is valid through record-pathing, but invalid through array pathing.
+ */
+
+DROP DATAVERSE TestYelp IF EXISTS;
+CREATE DATAVERSE TestYelp;
+USE TestYelp;
+
+CREATE TYPE CheckinType AS {
+    checkin_id: uuid,
+    business_id: string,
+    checkin_time: {
+        date: string,
+        time: string
+    }
+};
+
+CREATE DATASET YelpCheckin(CheckinType) PRIMARY KEY checkin_id AUTOGENERATED;
+CREATE INDEX IdxYelpCheckinDates ON YelpCheckin (UNNEST checkin_time.date);
+
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/insert-upsert-delete/closed/use-case-1/use-case-1.1.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/insert-upsert-delete/closed/use-case-1/use-case-1.1.ddl.sqlpp
new file mode 100644
index 0000000..7498c23
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/insert-upsert-delete/closed/use-case-1/use-case-1.1.ddl.sqlpp
@@ -0,0 +1,37 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+/*
+ * Description: Verify the following DML actions for an array->atomic index: 
+ *     1) Insert into an empty index (non bulk-load operation), additionally with two records that have no array-index qualifying entries.
+ *     2) Delete all-but-one entry from the index.
+ *     3) Upsert all *original* (all records have qualifying array-index entries now) entries into the index.
+ */
+
+DROP DATAVERSE TestYelp IF EXISTS;
+CREATE DATAVERSE TestYelp;
+USE TestYelp;
+
+CREATE TYPE CheckinType AS {
+    business_id: string,
+    dates: [string]
+};
+
+CREATE DATASET YelpCheckin(CheckinType) PRIMARY KEY business_id;
+CREATE INDEX IdxYelpCheckinDates ON YelpCheckin (UNNEST dates);
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/insert-upsert-delete/closed/use-case-1/use-case-1.2.update.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/insert-upsert-delete/closed/use-case-1/use-case-1.2.update.sqlpp
new file mode 100644
index 0000000..ba9a31b
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/insert-upsert-delete/closed/use-case-1/use-case-1.2.update.sqlpp
@@ -0,0 +1,272 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+USE TestYelp;
+
+INSERT INTO YelpCheckin (
+    [
+        {
+            "business_id": "--1UhMGODdWsrMastO9DZw",
+            "dates": [
+                "2016-04-26 19:49:16",
+                "2016-08-30 18:36:57",
+                "2016-10-15 02:45:18",
+                "2016-11-18 01:54:50",
+                "2017-04-20 18:39:06",
+                "2017-05-03 17:58:02",
+                "2019-03-19 22:04:48"
+            ]
+        },
+        {
+            "business_id": "--EF5N7P70J_UYBTPypYlA",
+            "dates": [
+                "2018-05-25 19:52:07",
+                "2018-09-18 16:09:44",
+                "2019-10-18 21:29:09"
+            ]
+        },
+        {
+            "business_id": "--Ni3oJ4VOqfOEu7Sj2Vzg",
+            "dates": [
+                "2019-06-07 17:54:58"
+            ]
+        },
+        {
+            "business_id": "--Y1Adl1YUWfYIRSd8vkmA",
+            "dates": [
+                "2011-05-03 20:54:05",
+                "2011-08-23 20:49:45",
+                "2014-12-04 06:13:01",
+                "2016-11-16 19:25:55"
+            ]
+        },
+        {
+            "business_id": "--YPwqIlRJrhHkJcjY3eiA",
+            "dates": [
+                "2016-06-18 21:35:45",
+                "2016-10-15 18:17:51"
+            ]
+        },
+        {
+            "business_id": "--e8PjCNhEz32pprnPhCwQ",
+            "dates": [
+                "2015-04-02 21:45:17"
+            ]
+        },
+        {
+            "business_id": "--kinfHwmtdjz03g8B8z8Q",
+            "dates": [
+                "2014-08-27 17:49:18",
+                "2015-12-19 21:30:31",
+                "2018-11-27 15:53:50"
+            ]
+        },
+        {
+            "business_id": "--q6datkI-f0EoVheXNEeQ",
+            "dates": [
+                "2014-01-28 20:56:04",
+                "2014-11-16 16:11:58",
+                "2015-11-15 19:21:53",
+                "2015-11-15 19:33:39"
+            ]
+        },
+        {
+            "business_id": "--qvQS4MigHPykD2GV0-zw",
+            "dates": [
+                "2019-04-11 18:30:12"
+            ]
+        },
+        {
+            "business_id": "--wIGbLEhlpl_UeAIyDmZQ",
+            "dates": [
+                "2015-06-06 20:01:06",
+                "2019-03-14 22:01:52"
+            ]
+        },
+        {
+            "business_id": "-0FA-Qdi3SPYIoJz9UQw-A",
+            "dates": [
+                "2018-09-29 18:55:17",
+                "2018-10-20 16:48:05",
+                "2018-10-20 22:20:24"
+            ]
+        },
+        {
+            "business_id": "-0Hj1hb_XW6ybWq2M7QhGA",
+            "dates": [
+                "2011-04-23 21:11:22",
+                "2014-05-04 19:42:48",
+                "2014-05-11 19:16:08",
+                "2014-06-04 19:14:18",
+                "2015-12-05 19:22:42",
+                "2017-05-15 23:19:00"
+            ]
+        },
+        {
+            "business_id": "-0KMvRFwDWdVBeTpT11iHw",
+            "dates": [
+                "2012-07-13 21:43:57",
+                "2016-12-24 02:27:31",
+                "2017-08-31 00:35:26"
+            ]
+        },
+        {
+            "business_id": "-0LPtgJC31FWMrMv317p0Q",
+            "dates": [
+                "2013-04-13 12:35:33",
+                "2013-08-19 23:35:49",
+                "2013-10-04 19:14:56"
+            ]
+        },
+        {
+            "business_id": "-0M3o2uWBnQZwd3hmfEwuw",
+            "dates": [
+                "2016-09-10 19:26:19",
+                "2018-09-08 14:15:37",
+                "2019-09-13 22:47:25"
+            ]
+        },
+        {
+            "business_id": "-0RRiWDtfnS16AKCtfvBZg",
+            "dates": [
+                "2017-05-19 14:30:16",
+                "2017-05-19 14:30:25",
+                "2017-08-28 15:49:37",
+                "2017-09-20 20:19:51",
+                "2017-10-01 16:31:05",
+                "2017-10-01 16:56:27",
+                "2017-12-27 23:33:20"
+            ]
+        },
+        {
+            "business_id": "-0Soj75v-XoRcf2ERr8Bmg",
+            "dates": [
+                "2019-06-05 18:22:49"
+            ]
+        },
+        {
+            "business_id": "-0ZumLlFjMh4ZW1z2nXGug",
+            "dates": [
+                "2011-09-24 21:37:32",
+                "2014-03-10 20:20:07",
+                "2015-05-27 00:40:24",
+                "2015-08-29 17:58:15",
+                "2018-03-16 15:03:26"
+            ]
+        },
+        {
+            "business_id": "-0aOudcaAyac0VJbMX-L1g",
+            "dates": [
+                "2015-03-16 23:51:16",
+                "2015-12-21 04:48:01",
+                "2016-10-28 20:22:42",
+                "2016-10-28 20:23:00"
+            ]
+        },
+        {
+            "business_id": "-0b86isaXMY0v4g-V8GZ9Q",
+            "dates": [
+                "2013-10-22 16:49:21",
+                "2014-11-21 17:39:24"
+            ]
+        },
+        {
+            "business_id": "-0d-BfFSU0bwLcnMaGRxYw",
+            "dates": [
+                "2014-08-07 18:30:48",
+                "2014-09-16 20:41:45",
+                "2014-10-12 23:22:27",
+                "2015-07-21 20:43:56",
+                "2015-07-21 20:45:07"
+            ]
+        },
+        {
+            "business_id": "-0jz6c3C6i7RG7Ag22K-Pg",
+            "dates": [
+                "2015-05-02 19:49:05",
+                "2015-05-06 03:52:18",
+                "2015-09-26 01:13:19"
+            ]
+        },
+        {
+            "business_id": "-0y3MZU2oYP8r1ruDP1bfQ",
+            "dates": [
+                "2015-04-11 13:14:14",
+                "2015-11-21 16:05:56",
+                "2016-05-06 14:10:04",
+                "2017-08-09 15:15:10",
+                "2017-10-21 15:12:56"
+            ]
+        },
+        {
+            "business_id": "-1BPe8UjF2_l3nVk-DFUjA",
+            "dates": [
+                "2015-12-03 18:44:00",
+                "2016-03-17 18:19:21",
+                "2016-11-02 15:58:38"
+            ]
+        },
+        {
+            "business_id": "-1E2CQu_38mkghvmZgCCRw",
+            "dates": []
+        },
+        {
+            "business_id": "-1wzk43IZ5D9Ysu6kzb5xA",
+            "dates": []
+        },
+        {
+            "business_id": "-23R9P2eG7VTc6DVLjFKzA",
+            "dates": [
+                "2011-12-21 19:02:51",
+                "2012-04-15 04:21:39",
+                "2012-04-15 14:23:56",
+                "2013-06-30 22:39:51",
+                "2013-10-04 20:34:13",
+                "2014-07-16 02:28:40"
+            ]
+        },
+        {
+            "business_id": "-26MGfikhJiTfCI-GqmzhQ",
+            "dates": [
+                "2018-06-13 20:16:07"
+            ]
+        },
+        {
+            "business_id": "-2bLuJsMZ0WhI9daurVQNQ",
+            "dates": [
+                "2015-05-29 16:46:17",
+                "2015-06-01 15:03:53"
+            ]
+        },
+        {
+            "business_id": "-2hDBMaza_ldqnZdiU06LQ",
+            "dates": [
+                "2011-10-08 12:02:23",
+                "2014-08-18 02:11:11",
+                "2016-01-07 05:27:51",
+                "2016-10-21 20:15:55",
+                "2016-12-01 03:57:10",
+                "2016-12-29 01:54:42",
+                "2018-07-22 19:55:31",
+                "2018-09-07 01:42:54",
+                "2019-03-08 03:41:06"
+            ]
+        }
+    ]
+);
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/insert-upsert-delete/closed/use-case-1/use-case-1.3.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/insert-upsert-delete/closed/use-case-1/use-case-1.3.query.sqlpp
new file mode 100644
index 0000000..69fe7cc
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/insert-upsert-delete/closed/use-case-1/use-case-1.3.query.sqlpp
@@ -0,0 +1,25 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+SET `compiler.arrayindex` "true";
+
+USE TestYelp;
+
+SELECT COUNT(*)
+FROM YelpCheckin C, C.dates D
+WHERE D > " ";
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/insert-upsert-delete/closed/use-case-1/use-case-1.4.update.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/insert-upsert-delete/closed/use-case-1/use-case-1.4.update.sqlpp
new file mode 100644
index 0000000..ea1dba3
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/insert-upsert-delete/closed/use-case-1/use-case-1.4.update.sqlpp
@@ -0,0 +1,23 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+USE TestYelp;
+
+DELETE FROM YelpCheckin C
+WHERE C.business_id != "--1UhMGODdWsrMastO9DZw";
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/insert-upsert-delete/closed/use-case-1/use-case-1.5.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/insert-upsert-delete/closed/use-case-1/use-case-1.5.query.sqlpp
new file mode 100644
index 0000000..69fe7cc
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/insert-upsert-delete/closed/use-case-1/use-case-1.5.query.sqlpp
@@ -0,0 +1,25 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+SET `compiler.arrayindex` "true";
+
+USE TestYelp;
+
+SELECT COUNT(*)
+FROM YelpCheckin C, C.dates D
+WHERE D > " ";
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/insert-upsert-delete/closed/use-case-1/use-case-1.6.update.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/insert-upsert-delete/closed/use-case-1/use-case-1.6.update.sqlpp
new file mode 100644
index 0000000..055fe8c
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/insert-upsert-delete/closed/use-case-1/use-case-1.6.update.sqlpp
@@ -0,0 +1,276 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+USE TestYelp;
+
+UPSERT INTO YelpCheckin (
+    [
+        {
+            "business_id": "--1UhMGODdWsrMastO9DZw",
+            "dates": [
+                "2016-04-26 19:49:16",
+                "2016-08-30 18:36:57",
+                "2016-10-15 02:45:18",
+                "2016-11-18 01:54:50",
+                "2017-04-20 18:39:06",
+                "2017-05-03 17:58:02",
+                "2019-03-19 22:04:48"
+            ]
+        },
+        {
+            "business_id": "--EF5N7P70J_UYBTPypYlA",
+            "dates": [
+                "2018-05-25 19:52:07",
+                "2018-09-18 16:09:44",
+                "2019-10-18 21:29:09"
+            ]
+        },
+        {
+            "business_id": "--Ni3oJ4VOqfOEu7Sj2Vzg",
+            "dates": [
+                "2019-06-07 17:54:58"
+            ]
+        },
+        {
+            "business_id": "--Y1Adl1YUWfYIRSd8vkmA",
+            "dates": [
+                "2011-05-03 20:54:05",
+                "2011-08-23 20:49:45",
+                "2014-12-04 06:13:01",
+                "2016-11-16 19:25:55"
+            ]
+        },
+        {
+            "business_id": "--YPwqIlRJrhHkJcjY3eiA",
+            "dates": [
+                "2016-06-18 21:35:45",
+                "2016-10-15 18:17:51"
+            ]
+        },
+        {
+            "business_id": "--e8PjCNhEz32pprnPhCwQ",
+            "dates": [
+                "2015-04-02 21:45:17"
+            ]
+        },
+        {
+            "business_id": "--kinfHwmtdjz03g8B8z8Q",
+            "dates": [
+                "2014-08-27 17:49:18",
+                "2015-12-19 21:30:31",
+                "2018-11-27 15:53:50"
+            ]
+        },
+        {
+            "business_id": "--q6datkI-f0EoVheXNEeQ",
+            "dates": [
+                "2014-01-28 20:56:04",
+                "2014-11-16 16:11:58",
+                "2015-11-15 19:21:53",
+                "2015-11-15 19:33:39"
+            ]
+        },
+        {
+            "business_id": "--qvQS4MigHPykD2GV0-zw",
+            "dates": [
+                "2019-04-11 18:30:12"
+            ]
+        },
+        {
+            "business_id": "--wIGbLEhlpl_UeAIyDmZQ",
+            "dates": [
+                "2015-06-06 20:01:06",
+                "2019-03-14 22:01:52"
+            ]
+        },
+        {
+            "business_id": "-0FA-Qdi3SPYIoJz9UQw-A",
+            "dates": [
+                "2018-09-29 18:55:17",
+                "2018-10-20 16:48:05",
+                "2018-10-20 22:20:24"
+            ]
+        },
+        {
+            "business_id": "-0Hj1hb_XW6ybWq2M7QhGA",
+            "dates": [
+                "2011-04-23 21:11:22",
+                "2014-05-04 19:42:48",
+                "2014-05-11 19:16:08",
+                "2014-06-04 19:14:18",
+                "2015-12-05 19:22:42",
+                "2017-05-15 23:19:00"
+            ]
+        },
+        {
+            "business_id": "-0KMvRFwDWdVBeTpT11iHw",
+            "dates": [
+                "2012-07-13 21:43:57",
+                "2016-12-24 02:27:31",
+                "2017-08-31 00:35:26"
+            ]
+        },
+        {
+            "business_id": "-0LPtgJC31FWMrMv317p0Q",
+            "dates": [
+                "2013-04-13 12:35:33",
+                "2013-08-19 23:35:49",
+                "2013-10-04 19:14:56"
+            ]
+        },
+        {
+            "business_id": "-0M3o2uWBnQZwd3hmfEwuw",
+            "dates": [
+                "2016-09-10 19:26:19",
+                "2018-09-08 14:15:37",
+                "2019-09-13 22:47:25"
+            ]
+        },
+        {
+            "business_id": "-0RRiWDtfnS16AKCtfvBZg",
+            "dates": [
+                "2017-05-19 14:30:16",
+                "2017-05-19 14:30:25",
+                "2017-08-28 15:49:37",
+                "2017-09-20 20:19:51",
+                "2017-10-01 16:31:05",
+                "2017-10-01 16:56:27",
+                "2017-12-27 23:33:20"
+            ]
+        },
+        {
+            "business_id": "-0Soj75v-XoRcf2ERr8Bmg",
+            "dates": [
+                "2019-06-05 18:22:49"
+            ]
+        },
+        {
+            "business_id": "-0ZumLlFjMh4ZW1z2nXGug",
+            "dates": [
+                "2011-09-24 21:37:32",
+                "2014-03-10 20:20:07",
+                "2015-05-27 00:40:24",
+                "2015-08-29 17:58:15",
+                "2018-03-16 15:03:26"
+            ]
+        },
+        {
+            "business_id": "-0aOudcaAyac0VJbMX-L1g",
+            "dates": [
+                "2015-03-16 23:51:16",
+                "2015-12-21 04:48:01",
+                "2016-10-28 20:22:42",
+                "2016-10-28 20:23:00"
+            ]
+        },
+        {
+            "business_id": "-0b86isaXMY0v4g-V8GZ9Q",
+            "dates": [
+                "2013-10-22 16:49:21",
+                "2014-11-21 17:39:24"
+            ]
+        },
+        {
+            "business_id": "-0d-BfFSU0bwLcnMaGRxYw",
+            "dates": [
+                "2014-08-07 18:30:48",
+                "2014-09-16 20:41:45",
+                "2014-10-12 23:22:27",
+                "2015-07-21 20:43:56",
+                "2015-07-21 20:45:07"
+            ]
+        },
+        {
+            "business_id": "-0jz6c3C6i7RG7Ag22K-Pg",
+            "dates": [
+                "2015-05-02 19:49:05",
+                "2015-05-06 03:52:18",
+                "2015-09-26 01:13:19"
+            ]
+        },
+        {
+            "business_id": "-0y3MZU2oYP8r1ruDP1bfQ",
+            "dates": [
+                "2015-04-11 13:14:14",
+                "2015-11-21 16:05:56",
+                "2016-05-06 14:10:04",
+                "2017-08-09 15:15:10",
+                "2017-10-21 15:12:56"
+            ]
+        },
+        {
+            "business_id": "-1BPe8UjF2_l3nVk-DFUjA",
+            "dates": [
+                "2015-12-03 18:44:00",
+                "2016-03-17 18:19:21",
+                "2016-11-02 15:58:38"
+            ]
+        },
+        {
+            "business_id": "-1E2CQu_38mkghvmZgCCRw",
+            "dates": [
+                "2019-04-04 22:02:37"
+            ]
+        },
+        {
+            "business_id": "-1wzk43IZ5D9Ysu6kzb5xA",
+            "dates": [
+                "2019-02-27 14:03:08"
+            ]
+        },
+        {
+            "business_id": "-23R9P2eG7VTc6DVLjFKzA",
+            "dates": [
+                "2011-12-21 19:02:51",
+                "2012-04-15 04:21:39",
+                "2012-04-15 14:23:56",
+                "2013-06-30 22:39:51",
+                "2013-10-04 20:34:13",
+                "2014-07-16 02:28:40"
+            ]
+        },
+        {
+            "business_id": "-26MGfikhJiTfCI-GqmzhQ",
+            "dates": [
+                "2018-06-13 20:16:07"
+            ]
+        },
+        {
+            "business_id": "-2bLuJsMZ0WhI9daurVQNQ",
+            "dates": [
+                "2015-05-29 16:46:17",
+                "2015-06-01 15:03:53"
+            ]
+        },
+        {
+            "business_id": "-2hDBMaza_ldqnZdiU06LQ",
+            "dates": [
+                "2011-10-08 12:02:23",
+                "2014-08-18 02:11:11",
+                "2016-01-07 05:27:51",
+                "2016-10-21 20:15:55",
+                "2016-12-01 03:57:10",
+                "2016-12-29 01:54:42",
+                "2018-07-22 19:55:31",
+                "2018-09-07 01:42:54",
+                "2019-03-08 03:41:06"
+            ]
+        }
+    ]
+);
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/insert-upsert-delete/closed/use-case-1/use-case-1.7.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/insert-upsert-delete/closed/use-case-1/use-case-1.7.query.sqlpp
new file mode 100644
index 0000000..69fe7cc
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/insert-upsert-delete/closed/use-case-1/use-case-1.7.query.sqlpp
@@ -0,0 +1,25 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+SET `compiler.arrayindex` "true";
+
+USE TestYelp;
+
+SELECT COUNT(*)
+FROM YelpCheckin C, C.dates D
+WHERE D > " ";
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/insert-upsert-delete/closed/use-case-2/use-case-2.1.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/insert-upsert-delete/closed/use-case-2/use-case-2.1.ddl.sqlpp
new file mode 100644
index 0000000..81b322c
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/insert-upsert-delete/closed/use-case-2/use-case-2.1.ddl.sqlpp
@@ -0,0 +1,41 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+/*
+ * Description: Verify the following DML actions for an array->atomic index:
+ *     1) Insert into an empty index (non bulk-load operation).
+ *     2) Delete all-but-one entry from the index.
+ *     3) Upsert all entries into the index.
+ */
+
+DROP DATAVERSE TestYelp IF EXISTS;
+CREATE DATAVERSE TestYelp;
+USE TestYelp;
+
+CREATE TYPE CheckinType AS {
+    business_id: string,
+    checkin_times: {
+        dates: [string],
+        times: [string]
+    }
+};
+
+CREATE DATASET YelpCheckin(CheckinType) PRIMARY KEY business_id;
+CREATE INDEX IdxYelpCheckinDates ON YelpCheckin (UNNEST checkin_times.dates);
+CREATE INDEX IdxYelpCheckinTimes ON YelpCheckin (UNNEST checkin_times.times);
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/insert-upsert-delete/closed/use-case-2/use-case-2.2.update.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/insert-upsert-delete/closed/use-case-2/use-case-2.2.update.sqlpp
new file mode 100644
index 0000000..2549e8f
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/insert-upsert-delete/closed/use-case-2/use-case-2.2.update.sqlpp
@@ -0,0 +1,497 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+USE TestYelp;
+
+INSERT INTO YelpCheckin (
+    [
+        {
+            "business_id": "--1UhMGODdWsrMastO9DZw",
+            "checkin_times": {
+                "dates": [
+                    "2016-04-26",
+                    "2016-08-30",
+                    "2016-10-15",
+                    "2016-11-18",
+                    "2017-04-20",
+                    "2017-05-03",
+                    "2019-03-19"
+                ],
+                "times": [
+                    "19:49:16",
+                    "18:36:57",
+                    "02:45:18",
+                    "01:54:50",
+                    "18:39:06",
+                    "17:58:02",
+                    "22:04:48"
+                ]
+            }
+        },
+        {
+            "business_id": "--EF5N7P70J_UYBTPypYlA",
+            "checkin_times": {
+                "dates": [
+                    "2018-05-25",
+                    "2018-09-18",
+                    "2019-10-18"
+                ],
+                "times": [
+                    "19:52:07",
+                    "16:09:44",
+                    "21:29:09"
+                ]
+            }
+        },
+        {
+            "business_id": "--Ni3oJ4VOqfOEu7Sj2Vzg",
+            "checkin_times": {
+                "dates": [
+                    "2019-06-07"
+                ],
+                "times": [
+                    "17:54:58"
+                ]
+            }
+        },
+        {
+            "business_id": "--Y1Adl1YUWfYIRSd8vkmA",
+            "checkin_times": {
+                "dates": [
+                    "2011-05-03",
+                    "2011-08-23",
+                    "2014-12-04",
+                    "2016-11-16"
+                ],
+                "times": [
+                    "20:54:05",
+                    "20:49:45",
+                    "06:13:01",
+                    "19:25:55"
+                ]
+            }
+        },
+        {
+            "business_id": "--YPwqIlRJrhHkJcjY3eiA",
+            "checkin_times": {
+                "dates": [
+                    "2016-06-18",
+                    "2016-10-15"
+                ],
+                "times": [
+                    "21:35:45",
+                    "18:17:51"
+                ]
+            }
+        },
+        {
+            "business_id": "--e8PjCNhEz32pprnPhCwQ",
+            "checkin_times": {
+                "dates": [
+                    "2015-04-02"
+                ],
+                "times": [
+                    "21:45:17"
+                ]
+            }
+        },
+        {
+            "business_id": "--kinfHwmtdjz03g8B8z8Q",
+            "checkin_times": {
+                "dates": [
+                    "2014-08-27",
+                    "2015-12-19",
+                    "2018-11-27"
+                ],
+                "times": [
+                    "17:49:18",
+                    "21:30:31",
+                    "15:53:50"
+                ]
+            }
+        },
+        {
+            "business_id": "--q6datkI-f0EoVheXNEeQ",
+            "checkin_times": {
+                "dates": [
+                    "2014-01-28",
+                    "2014-11-16",
+                    "2015-11-15",
+                    "2015-11-15"
+                ],
+                "times": [
+                    "20:56:04",
+                    "16:11:58",
+                    "19:21:53",
+                    "19:33:39"
+                ]
+            }
+        },
+        {
+            "business_id": "--qvQS4MigHPykD2GV0-zw",
+            "checkin_times": {
+                "dates": [
+                    "2019-04-11"
+                ],
+                "times": [
+                    "18:30:12"
+                ]
+            }
+        },
+        {
+            "business_id": "--wIGbLEhlpl_UeAIyDmZQ",
+            "checkin_times": {
+                "dates": [
+                    "2015-06-06",
+                    "2019-03-14"
+                ],
+                "times": [
+                    "20:01:06",
+                    "22:01:52"
+                ]
+            }
+        },
+        {
+            "business_id": "-0FA-Qdi3SPYIoJz9UQw-A",
+            "checkin_times": {
+                "dates": [
+                    "2018-09-29",
+                    "2018-10-20",
+                    "2018-10-20"
+                ],
+                "times": [
+                    "18:55:17",
+                    "16:48:05",
+                    "22:20:24"
+                ]
+            }
+        },
+        {
+            "business_id": "-0Hj1hb_XW6ybWq2M7QhGA",
+            "checkin_times": {
+                "dates": [
+                    "2011-04-23",
+                    "2014-05-04",
+                    "2014-05-11",
+                    "2014-06-04",
+                    "2015-12-05",
+                    "2017-05-15"
+                ],
+                "times": [
+                    "21:11:22",
+                    "19:42:48",
+                    "19:16:08",
+                    "19:14:18",
+                    "19:22:42",
+                    "23:19:00"
+                ]
+            }
+        },
+        {
+            "business_id": "-0KMvRFwDWdVBeTpT11iHw",
+            "checkin_times": {
+                "dates": [
+                    "2012-07-13",
+                    "2016-12-24",
+                    "2017-08-31"
+                ],
+                "times": [
+                    "21:43:57",
+                    "02:27:31",
+                    "00:35:26"
+                ]
+            }
+        },
+        {
+            "business_id": "-0LPtgJC31FWMrMv317p0Q",
+            "checkin_times": {
+                "dates": [
+                    "2013-04-13",
+                    "2013-08-19",
+                    "2013-10-04"
+                ],
+                "times": [
+                    "12:35:33",
+                    "23:35:49",
+                    "19:14:56"
+                ]
+            }
+        },
+        {
+            "business_id": "-0M3o2uWBnQZwd3hmfEwuw",
+            "checkin_times": {
+                "dates": [
+                    "2016-09-10",
+                    "2018-09-08",
+                    "2019-09-13"
+                ],
+                "times": [
+                    "19:26:19",
+                    "14:15:37",
+                    "22:47:25"
+                ]
+            }
+        },
+        {
+            "business_id": "-0RRiWDtfnS16AKCtfvBZg",
+            "checkin_times": {
+                "dates": [
+                    "2017-05-19",
+                    "2017-05-19",
+                    "2017-08-28",
+                    "2017-09-20",
+                    "2017-10-01",
+                    "2017-10-01",
+                    "2017-12-27"
+                ],
+                "times": [
+                    "14:30:16",
+                    "14:30:25",
+                    "15:49:37",
+                    "20:19:51",
+                    "16:31:05",
+                    "16:56:27",
+                    "23:33:20"
+                ]
+            }
+        },
+        {
+            "business_id": "-0Soj75v-XoRcf2ERr8Bmg",
+            "checkin_times": {
+                "dates": [
+                    "2019-06-05"
+                ],
+                "times": [
+                    "18:22:49"
+                ]
+            }
+        },
+        {
+            "business_id": "-0ZumLlFjMh4ZW1z2nXGug",
+            "checkin_times": {
+                "dates": [
+                    "2011-09-24",
+                    "2014-03-10",
+                    "2015-05-27",
+                    "2015-08-29",
+                    "2018-03-16"
+                ],
+                "times": [
+                    "21:37:32",
+                    "20:20:07",
+                    "00:40:24",
+                    "17:58:15",
+                    "15:03:26"
+                ]
+            }
+        },
+        {
+            "business_id": "-0aOudcaAyac0VJbMX-L1g",
+            "checkin_times": {
+                "dates": [
+                    "2015-03-16",
+                    "2015-12-21",
+                    "2016-10-28",
+                    "2016-10-28"
+                ],
+                "times": [
+                    "23:51:16",
+                    "04:48:01",
+                    "20:22:42",
+                    "20:23:00"
+                ]
+            }
+        },
+        {
+            "business_id": "-0b86isaXMY0v4g-V8GZ9Q",
+            "checkin_times": {
+                "dates": [
+                    "2013-10-22",
+                    "2014-11-21"
+                ],
+                "times": [
+                    "16:49:21",
+                    "17:39:24"
+                ]
+            }
+        },
+        {
+            "business_id": "-0d-BfFSU0bwLcnMaGRxYw",
+            "checkin_times": {
+                "dates": [
+                    "2014-08-07",
+                    "2014-09-16",
+                    "2014-10-12",
+                    "2015-07-21",
+                    "2015-07-21"
+                ],
+                "times": [
+                    "18:30:48",
+                    "20:41:45",
+                    "23:22:27",
+                    "20:43:56",
+                    "20:45:07"
+                ]
+            }
+        },
+        {
+            "business_id": "-0jz6c3C6i7RG7Ag22K-Pg",
+            "checkin_times": {
+                "dates": [
+                    "2015-05-02",
+                    "2015-05-06",
+                    "2015-09-26"
+                ],
+                "times": [
+                    "19:49:05",
+                    "03:52:18",
+                    "01:13:19"
+                ]
+            }
+        },
+        {
+            "business_id": "-0y3MZU2oYP8r1ruDP1bfQ",
+            "checkin_times": {
+                "dates": [
+                    "2015-04-11",
+                    "2015-11-21",
+                    "2016-05-06",
+                    "2017-08-09",
+                    "2017-10-21"
+                ],
+                "times": [
+                    "13:14:14",
+                    "16:05:56",
+                    "14:10:04",
+                    "15:15:10",
+                    "15:12:56"
+                ]
+            }
+        },
+        {
+            "business_id": "-1BPe8UjF2_l3nVk-DFUjA",
+            "checkin_times": {
+                "dates": [
+                    "2015-12-03",
+                    "2016-03-17",
+                    "2016-11-02"
+                ],
+                "times": [
+                    "18:44:00",
+                    "18:19:21",
+                    "15:58:38"
+                ]
+            }
+        },
+        {
+            "business_id": "-1E2CQu_38mkghvmZgCCRw",
+            "checkin_times": {
+                "dates": [
+                    "2019-04-04"
+                ],
+                "times": [
+                    "22:02:37"
+                ]
+            }
+        },
+        {
+            "business_id": "-1wzk43IZ5D9Ysu6kzb5xA",
+            "checkin_times": {
+                "dates": [
+                    "2019-02-27"
+                ],
+                "times": [
+                    "14:03:08"
+                ]
+            }
+        },
+        {
+            "business_id": "-23R9P2eG7VTc6DVLjFKzA",
+            "checkin_times": {
+                "dates": [
+                    "2011-12-21",
+                    "2012-04-15",
+                    "2012-04-15",
+                    "2013-06-30",
+                    "2013-10-04",
+                    "2014-07-16"
+                ],
+                "times": [
+                    "19:02:51",
+                    "04:21:39",
+                    "14:23:56",
+                    "22:39:51",
+                    "20:34:13",
+                    "02:28:40"
+                ]
+            }
+        },
+        {
+            "business_id": "-26MGfikhJiTfCI-GqmzhQ",
+            "checkin_times": {
+                "dates": [
+                    "2018-06-13"
+                ],
+                "times": [
+                    "20:16:07"
+                ]
+            }
+        },
+        {
+            "business_id": "-2bLuJsMZ0WhI9daurVQNQ",
+            "checkin_times": {
+                "dates": [
+                    "2015-05-29",
+                    "2015-06-01"
+                ],
+                "times": [
+                    "16:46:17",
+                    "15:03:53"
+                ]
+            }
+        },
+        {
+            "business_id": "-2hDBMaza_ldqnZdiU06LQ",
+            "checkin_times": {
+                "dates": [
+                    "2011-10-08",
+                    "2014-08-18",
+                    "2016-01-07",
+                    "2016-10-21",
+                    "2016-12-01",
+                    "2016-12-29",
+                    "2018-07-22",
+                    "2018-09-07",
+                    "2019-03-08"
+                ],
+                "times": [
+                    "12:02:23",
+                    "02:11:11",
+                    "05:27:51",
+                    "20:15:55",
+                    "03:57:10",
+                    "01:54:42",
+                    "19:55:31",
+                    "01:42:54",
+                    "03:41:06"
+                ]
+            }
+        }
+    ]
+);
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/insert-upsert-delete/closed/use-case-2/use-case-2.3.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/insert-upsert-delete/closed/use-case-2/use-case-2.3.query.sqlpp
new file mode 100644
index 0000000..a440550
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/insert-upsert-delete/closed/use-case-2/use-case-2.3.query.sqlpp
@@ -0,0 +1,25 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+SET `compiler.arrayindex` "true";
+
+USE TestYelp;
+
+SELECT COUNT(*)
+FROM YelpCheckin C, C.checkin_times.dates D
+WHERE D > " ";
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/insert-upsert-delete/closed/use-case-2/use-case-2.4.update.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/insert-upsert-delete/closed/use-case-2/use-case-2.4.update.sqlpp
new file mode 100644
index 0000000..ea1dba3
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/insert-upsert-delete/closed/use-case-2/use-case-2.4.update.sqlpp
@@ -0,0 +1,23 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+USE TestYelp;
+
+DELETE FROM YelpCheckin C
+WHERE C.business_id != "--1UhMGODdWsrMastO9DZw";
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/insert-upsert-delete/closed/use-case-2/use-case-2.5.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/insert-upsert-delete/closed/use-case-2/use-case-2.5.query.sqlpp
new file mode 100644
index 0000000..a440550
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/insert-upsert-delete/closed/use-case-2/use-case-2.5.query.sqlpp
@@ -0,0 +1,25 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+SET `compiler.arrayindex` "true";
+
+USE TestYelp;
+
+SELECT COUNT(*)
+FROM YelpCheckin C, C.checkin_times.dates D
+WHERE D > " ";
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/insert-upsert-delete/closed/use-case-2/use-case-2.6.update.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/insert-upsert-delete/closed/use-case-2/use-case-2.6.update.sqlpp
new file mode 100644
index 0000000..9467e0d
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/insert-upsert-delete/closed/use-case-2/use-case-2.6.update.sqlpp
@@ -0,0 +1,497 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+USE TestYelp;
+
+UPSERT INTO YelpCheckin (
+    [
+        {
+            "business_id": "--1UhMGODdWsrMastO9DZw",
+            "checkin_times": {
+                "dates": [
+                    "2016-04-26",
+                    "2016-08-30",
+                    "2016-10-15",
+                    "2016-11-18",
+                    "2017-04-20",
+                    "2017-05-03",
+                    "2019-03-19"
+                ],
+                "times": [
+                    "19:49:16",
+                    "18:36:57",
+                    "02:45:18",
+                    "01:54:50",
+                    "18:39:06",
+                    "17:58:02",
+                    "22:04:48"
+                ]
+            }
+        },
+        {
+            "business_id": "--EF5N7P70J_UYBTPypYlA",
+            "checkin_times": {
+                "dates": [
+                    "2018-05-25",
+                    "2018-09-18",
+                    "2019-10-18"
+                ],
+                "times": [
+                    "19:52:07",
+                    "16:09:44",
+                    "21:29:09"
+                ]
+            }
+        },
+        {
+            "business_id": "--Ni3oJ4VOqfOEu7Sj2Vzg",
+            "checkin_times": {
+                "dates": [
+                    "2019-06-07"
+                ],
+                "times": [
+                    "17:54:58"
+                ]
+            }
+        },
+        {
+            "business_id": "--Y1Adl1YUWfYIRSd8vkmA",
+            "checkin_times": {
+                "dates": [
+                    "2011-05-03",
+                    "2011-08-23",
+                    "2014-12-04",
+                    "2016-11-16"
+                ],
+                "times": [
+                    "20:54:05",
+                    "20:49:45",
+                    "06:13:01",
+                    "19:25:55"
+                ]
+            }
+        },
+        {
+            "business_id": "--YPwqIlRJrhHkJcjY3eiA",
+            "checkin_times": {
+                "dates": [
+                    "2016-06-18",
+                    "2016-10-15"
+                ],
+                "times": [
+                    "21:35:45",
+                    "18:17:51"
+                ]
+            }
+        },
+        {
+            "business_id": "--e8PjCNhEz32pprnPhCwQ",
+            "checkin_times": {
+                "dates": [
+                    "2015-04-02"
+                ],
+                "times": [
+                    "21:45:17"
+                ]
+            }
+        },
+        {
+            "business_id": "--kinfHwmtdjz03g8B8z8Q",
+            "checkin_times": {
+                "dates": [
+                    "2014-08-27",
+                    "2015-12-19",
+                    "2018-11-27"
+                ],
+                "times": [
+                    "17:49:18",
+                    "21:30:31",
+                    "15:53:50"
+                ]
+            }
+        },
+        {
+            "business_id": "--q6datkI-f0EoVheXNEeQ",
+            "checkin_times": {
+                "dates": [
+                    "2014-01-28",
+                    "2014-11-16",
+                    "2015-11-15",
+                    "2015-11-15"
+                ],
+                "times": [
+                    "20:56:04",
+                    "16:11:58",
+                    "19:21:53",
+                    "19:33:39"
+                ]
+            }
+        },
+        {
+            "business_id": "--qvQS4MigHPykD2GV0-zw",
+            "checkin_times": {
+                "dates": [
+                    "2019-04-11"
+                ],
+                "times": [
+                    "18:30:12"
+                ]
+            }
+        },
+        {
+            "business_id": "--wIGbLEhlpl_UeAIyDmZQ",
+            "checkin_times": {
+                "dates": [
+                    "2015-06-06",
+                    "2019-03-14"
+                ],
+                "times": [
+                    "20:01:06",
+                    "22:01:52"
+                ]
+            }
+        },
+        {
+            "business_id": "-0FA-Qdi3SPYIoJz9UQw-A",
+            "checkin_times": {
+                "dates": [
+                    "2018-09-29",
+                    "2018-10-20",
+                    "2018-10-20"
+                ],
+                "times": [
+                    "18:55:17",
+                    "16:48:05",
+                    "22:20:24"
+                ]
+            }
+        },
+        {
+            "business_id": "-0Hj1hb_XW6ybWq2M7QhGA",
+            "checkin_times": {
+                "dates": [
+                    "2011-04-23",
+                    "2014-05-04",
+                    "2014-05-11",
+                    "2014-06-04",
+                    "2015-12-05",
+                    "2017-05-15"
+                ],
+                "times": [
+                    "21:11:22",
+                    "19:42:48",
+                    "19:16:08",
+                    "19:14:18",
+                    "19:22:42",
+                    "23:19:00"
+                ]
+            }
+        },
+        {
+            "business_id": "-0KMvRFwDWdVBeTpT11iHw",
+            "checkin_times": {
+                "dates": [
+                    "2012-07-13",
+                    "2016-12-24",
+                    "2017-08-31"
+                ],
+                "times": [
+                    "21:43:57",
+                    "02:27:31",
+                    "00:35:26"
+                ]
+            }
+        },
+        {
+            "business_id": "-0LPtgJC31FWMrMv317p0Q",
+            "checkin_times": {
+                "dates": [
+                    "2013-04-13",
+                    "2013-08-19",
+                    "2013-10-04"
+                ],
+                "times": [
+                    "12:35:33",
+                    "23:35:49",
+                    "19:14:56"
+                ]
+            }
+        },
+        {
+            "business_id": "-0M3o2uWBnQZwd3hmfEwuw",
+            "checkin_times": {
+                "dates": [
+                    "2016-09-10",
+                    "2018-09-08",
+                    "2019-09-13"
+                ],
+                "times": [
+                    "19:26:19",
+                    "14:15:37",
+                    "22:47:25"
+                ]
+            }
+        },
+        {
+            "business_id": "-0RRiWDtfnS16AKCtfvBZg",
+            "checkin_times": {
+                "dates": [
+                    "2017-05-19",
+                    "2017-05-19",
+                    "2017-08-28",
+                    "2017-09-20",
+                    "2017-10-01",
+                    "2017-10-01",
+                    "2017-12-27"
+                ],
+                "times": [
+                    "14:30:16",
+                    "14:30:25",
+                    "15:49:37",
+                    "20:19:51",
+                    "16:31:05",
+                    "16:56:27",
+                    "23:33:20"
+                ]
+            }
+        },
+        {
+            "business_id": "-0Soj75v-XoRcf2ERr8Bmg",
+            "checkin_times": {
+                "dates": [
+                    "2019-06-05"
+                ],
+                "times": [
+                    "18:22:49"
+                ]
+            }
+        },
+        {
+            "business_id": "-0ZumLlFjMh4ZW1z2nXGug",
+            "checkin_times": {
+                "dates": [
+                    "2011-09-24",
+                    "2014-03-10",
+                    "2015-05-27",
+                    "2015-08-29",
+                    "2018-03-16"
+                ],
+                "times": [
+                    "21:37:32",
+                    "20:20:07",
+                    "00:40:24",
+                    "17:58:15",
+                    "15:03:26"
+                ]
+            }
+        },
+        {
+            "business_id": "-0aOudcaAyac0VJbMX-L1g",
+            "checkin_times": {
+                "dates": [
+                    "2015-03-16",
+                    "2015-12-21",
+                    "2016-10-28",
+                    "2016-10-28"
+                ],
+                "times": [
+                    "23:51:16",
+                    "04:48:01",
+                    "20:22:42",
+                    "20:23:00"
+                ]
+            }
+        },
+        {
+            "business_id": "-0b86isaXMY0v4g-V8GZ9Q",
+            "checkin_times": {
+                "dates": [
+                    "2013-10-22",
+                    "2014-11-21"
+                ],
+                "times": [
+                    "16:49:21",
+                    "17:39:24"
+                ]
+            }
+        },
+        {
+            "business_id": "-0d-BfFSU0bwLcnMaGRxYw",
+            "checkin_times": {
+                "dates": [
+                    "2014-08-07",
+                    "2014-09-16",
+                    "2014-10-12",
+                    "2015-07-21",
+                    "2015-07-21"
+                ],
+                "times": [
+                    "18:30:48",
+                    "20:41:45",
+                    "23:22:27",
+                    "20:43:56",
+                    "20:45:07"
+                ]
+            }
+        },
+        {
+            "business_id": "-0jz6c3C6i7RG7Ag22K-Pg",
+            "checkin_times": {
+                "dates": [
+                    "2015-05-02",
+                    "2015-05-06",
+                    "2015-09-26"
+                ],
+                "times": [
+                    "19:49:05",
+                    "03:52:18",
+                    "01:13:19"
+                ]
+            }
+        },
+        {
+            "business_id": "-0y3MZU2oYP8r1ruDP1bfQ",
+            "checkin_times": {
+                "dates": [
+                    "2015-04-11",
+                    "2015-11-21",
+                    "2016-05-06",
+                    "2017-08-09",
+                    "2017-10-21"
+                ],
+                "times": [
+                    "13:14:14",
+                    "16:05:56",
+                    "14:10:04",
+                    "15:15:10",
+                    "15:12:56"
+                ]
+            }
+        },
+        {
+            "business_id": "-1BPe8UjF2_l3nVk-DFUjA",
+            "checkin_times": {
+                "dates": [
+                    "2015-12-03",
+                    "2016-03-17",
+                    "2016-11-02"
+                ],
+                "times": [
+                    "18:44:00",
+                    "18:19:21",
+                    "15:58:38"
+                ]
+            }
+        },
+        {
+            "business_id": "-1E2CQu_38mkghvmZgCCRw",
+            "checkin_times": {
+                "dates": [
+                    "2019-04-04"
+                ],
+                "times": [
+                    "22:02:37"
+                ]
+            }
+        },
+        {
+            "business_id": "-1wzk43IZ5D9Ysu6kzb5xA",
+            "checkin_times": {
+                "dates": [
+                    "2019-02-27"
+                ],
+                "times": [
+                    "14:03:08"
+                ]
+            }
+        },
+        {
+            "business_id": "-23R9P2eG7VTc6DVLjFKzA",
+            "checkin_times": {
+                "dates": [
+                    "2011-12-21",
+                    "2012-04-15",
+                    "2012-04-15",
+                    "2013-06-30",
+                    "2013-10-04",
+                    "2014-07-16"
+                ],
+                "times": [
+                    "19:02:51",
+                    "04:21:39",
+                    "14:23:56",
+                    "22:39:51",
+                    "20:34:13",
+                    "02:28:40"
+                ]
+            }
+        },
+        {
+            "business_id": "-26MGfikhJiTfCI-GqmzhQ",
+            "checkin_times": {
+                "dates": [
+                    "2018-06-13"
+                ],
+                "times": [
+                    "20:16:07"
+                ]
+            }
+        },
+        {
+            "business_id": "-2bLuJsMZ0WhI9daurVQNQ",
+            "checkin_times": {
+                "dates": [
+                    "2015-05-29",
+                    "2015-06-01"
+                ],
+                "times": [
+                    "16:46:17",
+                    "15:03:53"
+                ]
+            }
+        },
+        {
+            "business_id": "-2hDBMaza_ldqnZdiU06LQ",
+            "checkin_times": {
+                "dates": [
+                    "2011-10-08",
+                    "2014-08-18",
+                    "2016-01-07",
+                    "2016-10-21",
+                    "2016-12-01",
+                    "2016-12-29",
+                    "2018-07-22",
+                    "2018-09-07",
+                    "2019-03-08"
+                ],
+                "times": [
+                    "12:02:23",
+                    "02:11:11",
+                    "05:27:51",
+                    "20:15:55",
+                    "03:57:10",
+                    "01:54:42",
+                    "19:55:31",
+                    "01:42:54",
+                    "03:41:06"
+                ]
+            }
+        }
+    ]
+);
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/insert-upsert-delete/closed/use-case-2/use-case-2.7.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/insert-upsert-delete/closed/use-case-2/use-case-2.7.query.sqlpp
new file mode 100644
index 0000000..a440550
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/insert-upsert-delete/closed/use-case-2/use-case-2.7.query.sqlpp
@@ -0,0 +1,25 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+SET `compiler.arrayindex` "true";
+
+USE TestYelp;
+
+SELECT COUNT(*)
+FROM YelpCheckin C, C.checkin_times.dates D
+WHERE D > " ";
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/insert-upsert-delete/closed/use-case-3/use-case-3.1.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/insert-upsert-delete/closed/use-case-3/use-case-3.1.ddl.sqlpp
new file mode 100644
index 0000000..d65a2eb
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/insert-upsert-delete/closed/use-case-3/use-case-3.1.ddl.sqlpp
@@ -0,0 +1,41 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+/*
+ * Description: Verify the following DML actions for an array->record->atomic index:
+ *     1) Insert into an empty index (non bulk-load operation).
+ *     2) Delete all-but-one entry from the index.
+ *     3) Upsert all entries into the index.
+ */
+
+DROP DATAVERSE TestYelp IF EXISTS;
+CREATE DATAVERSE TestYelp;
+USE TestYelp;
+
+CREATE TYPE CheckinType AS {
+    business_id: string,
+    checkin_times: [{
+        date: string,
+        time: string
+    }]
+};
+
+CREATE DATASET YelpCheckin(CheckinType) PRIMARY KEY business_id;
+CREATE INDEX IdxYelpCheckinDates ON YelpCheckin (UNNEST checkin_times SELECT date);
+CREATE INDEX IdxYelpCheckinTimes ON YelpCheckin (UNNEST checkin_times SELECT time);
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/insert-upsert-delete/closed/use-case-3/use-case-3.2.update.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/insert-upsert-delete/closed/use-case-3/use-case-3.2.update.sqlpp
new file mode 100644
index 0000000..4fd7c64
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/insert-upsert-delete/closed/use-case-3/use-case-3.2.update.sqlpp
@@ -0,0 +1,276 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+USE TestYelp;
+
+INSERT INTO YelpCheckin (
+    [
+        {
+            "business_id": "--1UhMGODdWsrMastO9DZw",
+            "checkin_times": [
+                {"date": "2016-04-26", "time": "19:49:16"},
+                {"date": "2016-08-30", "time": "18:36:57"},
+                {"date": "2016-10-15", "time": "02:45:18"},
+                {"date": "2016-11-18", "time": "01:54:50"},
+                {"date": "2017-04-20", "time": "18:39:06"},
+                {"date": "2017-05-03", "time": "17:58:02"},
+                {"date": "2019-03-19", "time": "22:04:48"}
+             ]
+        },
+        {
+            "business_id": "--EF5N7P70J_UYBTPypYlA",
+            "checkin_times": [
+                {"date": "2018-05-25", "time": "19:52:07"},
+                {"date": "2018-09-18", "time": "16:09:44"},
+                {"date": "2019-10-18", "time": "21:29:09"}
+             ]
+        },
+        {
+            "business_id": "--Ni3oJ4VOqfOEu7Sj2Vzg",
+            "checkin_times": [
+                {"date": "2019-06-07", "time": "17:54:58"}
+             ]
+        },
+        {
+            "business_id": "--Y1Adl1YUWfYIRSd8vkmA",
+            "checkin_times": [
+                {"date": "2011-05-03", "time": "20:54:05"},
+                {"date": "2011-08-23", "time": "20:49:45"},
+                {"date": "2014-12-04", "time": "06:13:01"},
+                {"date": "2016-11-16", "time": "19:25:55"}
+             ]
+        },
+        {
+            "business_id": "--YPwqIlRJrhHkJcjY3eiA",
+            "checkin_times": [
+                {"date": "2016-06-18", "time": "21:35:45"},
+                {"date": "2016-10-15", "time": "18:17:51"}
+             ]
+        },
+        {
+            "business_id": "--e8PjCNhEz32pprnPhCwQ",
+            "checkin_times": [
+                {"date": "2015-04-02", "time": "21:45:17"}
+             ]
+        },
+        {
+            "business_id": "--kinfHwmtdjz03g8B8z8Q",
+            "checkin_times": [
+                {"date": "2014-08-27", "time": "17:49:18"},
+                {"date": "2015-12-19", "time": "21:30:31"},
+                {"date": "2018-11-27", "time": "15:53:50"}
+             ]
+        },
+        {
+            "business_id": "--q6datkI-f0EoVheXNEeQ",
+            "checkin_times": [
+                {"date": "2014-01-28", "time": "20:56:04"},
+                {"date": "2014-11-16", "time": "16:11:58"},
+                {"date": "2015-11-15", "time": "19:21:53"},
+                {"date": "2015-11-15", "time": "19:33:39"}
+             ]
+        },
+        {
+            "business_id": "--qvQS4MigHPykD2GV0-zw",
+            "checkin_times": [
+                {"date": "2019-04-11", "time": "18:30:12"}
+             ]
+        },
+        {
+            "business_id": "--wIGbLEhlpl_UeAIyDmZQ",
+            "checkin_times": [
+                {"date": "2015-06-06", "time": "20:01:06"},
+                {"date": "2019-03-14", "time": "22:01:52"}
+             ]
+        },
+        {
+            "business_id": "-0FA-Qdi3SPYIoJz9UQw-A",
+            "checkin_times": [
+                {"date": "2018-09-29", "time": "18:55:17"},
+                {"date": "2018-10-20", "time": "16:48:05"},
+                {"date": "2018-10-20", "time": "22:20:24"}
+             ]
+        },
+        {
+            "business_id": "-0Hj1hb_XW6ybWq2M7QhGA",
+            "checkin_times": [
+                {"date": "2011-04-23", "time": "21:11:22"},
+                {"date": "2014-05-04", "time": "19:42:48"},
+                {"date": "2014-05-11", "time": "19:16:08"},
+                {"date": "2014-06-04", "time": "19:14:18"},
+                {"date": "2015-12-05", "time": "19:22:42"},
+                {"date": "2017-05-15", "time": "23:19:00"}
+             ]
+        },
+        {
+            "business_id": "-0KMvRFwDWdVBeTpT11iHw",
+            "checkin_times": [
+                {"date": "2012-07-13", "time": "21:43:57"},
+                {"date": "2016-12-24", "time": "02:27:31"},
+                {"date": "2017-08-31", "time": "00:35:26"}
+             ]
+        },
+        {
+            "business_id": "-0LPtgJC31FWMrMv317p0Q",
+            "checkin_times": [
+                {"date": "2013-04-13", "time": "12:35:33"},
+                {"date": "2013-08-19", "time": "23:35:49"},
+                {"date": "2013-10-04", "time": "19:14:56"}
+             ]
+        },
+        {
+            "business_id": "-0M3o2uWBnQZwd3hmfEwuw",
+            "checkin_times": [
+                {"date": "2016-09-10", "time": "19:26:19"},
+                {"date": "2018-09-08", "time": "14:15:37"},
+                {"date": "2019-09-13", "time": "22:47:25"}
+             ]
+        },
+        {
+            "business_id": "-0RRiWDtfnS16AKCtfvBZg",
+            "checkin_times": [
+                {"date": "2017-05-19", "time": "14:30:16"},
+                {"date": "2017-05-19", "time": "14:30:25"},
+                {"date": "2017-08-28", "time": "15:49:37"},
+                {"date": "2017-09-20", "time": "20:19:51"},
+                {"date": "2017-10-01", "time": "16:31:05"},
+                {"date": "2017-10-01", "time": "16:56:27"},
+                {"date": "2017-12-27", "time": "23:33:20"}
+             ]
+        },
+        {
+            "business_id": "-0Soj75v-XoRcf2ERr8Bmg",
+            "checkin_times": [
+                {"date": "2019-06-05", "time": "18:22:49"}
+             ]
+        },
+        {
+            "business_id": "-0ZumLlFjMh4ZW1z2nXGug",
+            "checkin_times": [
+                {"date": "2011-09-24", "time": "21:37:32"},
+                {"date": "2014-03-10", "time": "20:20:07"},
+                {"date": "2015-05-27", "time": "00:40:24"},
+                {"date": "2015-08-29", "time": "17:58:15"},
+                {"date": "2018-03-16", "time": "15:03:26"}
+             ]
+        },
+        {
+            "business_id": "-0aOudcaAyac0VJbMX-L1g",
+            "checkin_times": [
+                {"date": "2015-03-16", "time": "23:51:16"},
+                {"date": "2015-12-21", "time": "04:48:01"},
+                {"date": "2016-10-28", "time": "20:22:42"},
+                {"date": "2016-10-28", "time": "20:23:00"}
+             ]
+        },
+        {
+            "business_id": "-0b86isaXMY0v4g-V8GZ9Q",
+            "checkin_times": [
+                {"date": "2013-10-22", "time": "16:49:21"},
+                {"date": "2014-11-21", "time": "17:39:24"}
+             ]
+        },
+        {
+            "business_id": "-0d-BfFSU0bwLcnMaGRxYw",
+            "checkin_times": [
+                {"date": "2014-08-07", "time": "18:30:48"},
+                {"date": "2014-09-16", "time": "20:41:45"},
+                {"date": "2014-10-12", "time": "23:22:27"},
+                {"date": "2015-07-21", "time": "20:43:56"},
+                {"date": "2015-07-21", "time": "20:45:07"}
+             ]
+        },
+        {
+            "business_id": "-0jz6c3C6i7RG7Ag22K-Pg",
+            "checkin_times": [
+                {"date": "2015-05-02", "time": "19:49:05"},
+                {"date": "2015-05-06", "time": "03:52:18"},
+                {"date": "2015-09-26", "time": "01:13:19"}
+             ]
+        },
+        {
+            "business_id": "-0y3MZU2oYP8r1ruDP1bfQ",
+            "checkin_times": [
+                {"date": "2015-04-11", "time": "13:14:14"},
+                {"date": "2015-11-21", "time": "16:05:56"},
+                {"date": "2016-05-06", "time": "14:10:04"},
+                {"date": "2017-08-09", "time": "15:15:10"},
+                {"date": "2017-10-21", "time": "15:12:56"}
+             ]
+        },
+        {
+            "business_id": "-1BPe8UjF2_l3nVk-DFUjA",
+            "checkin_times": [
+                {"date": "2015-12-03", "time": "18:44:00"},
+                {"date": "2016-03-17", "time": "18:19:21"},
+                {"date": "2016-11-02", "time": "15:58:38"}
+             ]
+        },
+        {
+            "business_id": "-1E2CQu_38mkghvmZgCCRw",
+            "checkin_times": [
+                {"date": "2019-04-04", "time": "22:02:37"}
+             ]
+        },
+        {
+            "business_id": "-1wzk43IZ5D9Ysu6kzb5xA",
+            "checkin_times": [
+                {"date": "2019-02-27", "time": "14:03:08"}
+             ]
+        },
+        {
+            "business_id": "-23R9P2eG7VTc6DVLjFKzA",
+            "checkin_times": [
+                {"date": "2011-12-21", "time": "19:02:51"},
+                {"date": "2012-04-15", "time": "04:21:39"},
+                {"date": "2012-04-15", "time": "14:23:56"},
+                {"date": "2013-06-30", "time": "22:39:51"},
+                {"date": "2013-10-04", "time": "20:34:13"},
+                {"date": "2014-07-16", "time": "02:28:40"}
+             ]
+        },
+        {
+            "business_id": "-26MGfikhJiTfCI-GqmzhQ",
+            "checkin_times": [
+                {"date": "2018-06-13", "time": "20:16:07"}
+             ]
+        },
+        {
+            "business_id": "-2bLuJsMZ0WhI9daurVQNQ",
+            "checkin_times": [
+                {"date": "2015-05-29", "time": "16:46:17"},
+                {"date": "2015-06-01", "time": "15:03:53"}
+             ]
+        },
+        {
+            "business_id": "-2hDBMaza_ldqnZdiU06LQ",
+            "checkin_times": [
+                {"date": "2011-10-08", "time": "12:02:23"},
+                {"date": "2014-08-18", "time": "02:11:11"},
+                {"date": "2016-01-07", "time": "05:27:51"},
+                {"date": "2016-10-21", "time": "20:15:55"},
+                {"date": "2016-12-01", "time": "03:57:10"},
+                {"date": "2016-12-29", "time": "01:54:42"},
+                {"date": "2018-07-22", "time": "19:55:31"},
+                {"date": "2018-09-07", "time": "01:42:54"},
+                {"date": "2019-03-08", "time": "03:41:06"}
+             ]
+        }
+    ]
+);
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/insert-upsert-delete/closed/use-case-3/use-case-3.3.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/insert-upsert-delete/closed/use-case-3/use-case-3.3.query.sqlpp
new file mode 100644
index 0000000..2757b02
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/insert-upsert-delete/closed/use-case-3/use-case-3.3.query.sqlpp
@@ -0,0 +1,25 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+SET `compiler.arrayindex` "true";
+
+USE TestYelp;
+
+SELECT COUNT(*)
+FROM YelpCheckin C, C.checkin_times D
+WHERE D.date > " ";
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/insert-upsert-delete/closed/use-case-3/use-case-3.4.update.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/insert-upsert-delete/closed/use-case-3/use-case-3.4.update.sqlpp
new file mode 100644
index 0000000..ea1dba3
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/insert-upsert-delete/closed/use-case-3/use-case-3.4.update.sqlpp
@@ -0,0 +1,23 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+USE TestYelp;
+
+DELETE FROM YelpCheckin C
+WHERE C.business_id != "--1UhMGODdWsrMastO9DZw";
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/insert-upsert-delete/closed/use-case-3/use-case-3.5.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/insert-upsert-delete/closed/use-case-3/use-case-3.5.query.sqlpp
new file mode 100644
index 0000000..2757b02
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/insert-upsert-delete/closed/use-case-3/use-case-3.5.query.sqlpp
@@ -0,0 +1,25 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+SET `compiler.arrayindex` "true";
+
+USE TestYelp;
+
+SELECT COUNT(*)
+FROM YelpCheckin C, C.checkin_times D
+WHERE D.date > " ";
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/insert-upsert-delete/closed/use-case-3/use-case-3.6.update.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/insert-upsert-delete/closed/use-case-3/use-case-3.6.update.sqlpp
new file mode 100644
index 0000000..b3f8466
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/insert-upsert-delete/closed/use-case-3/use-case-3.6.update.sqlpp
@@ -0,0 +1,276 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+USE TestYelp;
+
+UPSERT INTO YelpCheckin (
+    [
+        {
+            "business_id": "--1UhMGODdWsrMastO9DZw",
+            "checkin_times": [
+                {"date": "2016-04-26", "time": "19:49:16"},
+                {"date": "2016-08-30", "time": "18:36:57"},
+                {"date": "2016-10-15", "time": "02:45:18"},
+                {"date": "2016-11-18", "time": "01:54:50"},
+                {"date": "2017-04-20", "time": "18:39:06"},
+                {"date": "2017-05-03", "time": "17:58:02"},
+                {"date": "2019-03-19", "time": "22:04:48"}
+             ]
+        },
+        {
+            "business_id": "--EF5N7P70J_UYBTPypYlA",
+            "checkin_times": [
+                {"date": "2018-05-25", "time": "19:52:07"},
+                {"date": "2018-09-18", "time": "16:09:44"},
+                {"date": "2019-10-18", "time": "21:29:09"}
+             ]
+        },
+        {
+            "business_id": "--Ni3oJ4VOqfOEu7Sj2Vzg",
+            "checkin_times": [
+                {"date": "2019-06-07", "time": "17:54:58"}
+             ]
+        },
+        {
+            "business_id": "--Y1Adl1YUWfYIRSd8vkmA",
+            "checkin_times": [
+                {"date": "2011-05-03", "time": "20:54:05"},
+                {"date": "2011-08-23", "time": "20:49:45"},
+                {"date": "2014-12-04", "time": "06:13:01"},
+                {"date": "2016-11-16", "time": "19:25:55"}
+             ]
+        },
+        {
+            "business_id": "--YPwqIlRJrhHkJcjY3eiA",
+            "checkin_times": [
+                {"date": "2016-06-18", "time": "21:35:45"},
+                {"date": "2016-10-15", "time": "18:17:51"}
+             ]
+        },
+        {
+            "business_id": "--e8PjCNhEz32pprnPhCwQ",
+            "checkin_times": [
+                {"date": "2015-04-02", "time": "21:45:17"}
+             ]
+        },
+        {
+            "business_id": "--kinfHwmtdjz03g8B8z8Q",
+            "checkin_times": [
+                {"date": "2014-08-27", "time": "17:49:18"},
+                {"date": "2015-12-19", "time": "21:30:31"},
+                {"date": "2018-11-27", "time": "15:53:50"}
+             ]
+        },
+        {
+            "business_id": "--q6datkI-f0EoVheXNEeQ",
+            "checkin_times": [
+                {"date": "2014-01-28", "time": "20:56:04"},
+                {"date": "2014-11-16", "time": "16:11:58"},
+                {"date": "2015-11-15", "time": "19:21:53"},
+                {"date": "2015-11-15", "time": "19:33:39"}
+             ]
+        },
+        {
+            "business_id": "--qvQS4MigHPykD2GV0-zw",
+            "checkin_times": [
+                {"date": "2019-04-11", "time": "18:30:12"}
+             ]
+        },
+        {
+            "business_id": "--wIGbLEhlpl_UeAIyDmZQ",
+            "checkin_times": [
+                {"date": "2015-06-06", "time": "20:01:06"},
+                {"date": "2019-03-14", "time": "22:01:52"}
+             ]
+        },
+        {
+            "business_id": "-0FA-Qdi3SPYIoJz9UQw-A",
+            "checkin_times": [
+                {"date": "2018-09-29", "time": "18:55:17"},
+                {"date": "2018-10-20", "time": "16:48:05"},
+                {"date": "2018-10-20", "time": "22:20:24"}
+             ]
+        },
+        {
+            "business_id": "-0Hj1hb_XW6ybWq2M7QhGA",
+            "checkin_times": [
+                {"date": "2011-04-23", "time": "21:11:22"},
+                {"date": "2014-05-04", "time": "19:42:48"},
+                {"date": "2014-05-11", "time": "19:16:08"},
+                {"date": "2014-06-04", "time": "19:14:18"},
+                {"date": "2015-12-05", "time": "19:22:42"},
+                {"date": "2017-05-15", "time": "23:19:00"}
+             ]
+        },
+        {
+            "business_id": "-0KMvRFwDWdVBeTpT11iHw",
+            "checkin_times": [
+                {"date": "2012-07-13", "time": "21:43:57"},
+                {"date": "2016-12-24", "time": "02:27:31"},
+                {"date": "2017-08-31", "time": "00:35:26"}
+             ]
+        },
+        {
+            "business_id": "-0LPtgJC31FWMrMv317p0Q",
+            "checkin_times": [
+                {"date": "2013-04-13", "time": "12:35:33"},
+                {"date": "2013-08-19", "time": "23:35:49"},
+                {"date": "2013-10-04", "time": "19:14:56"}
+             ]
+        },
+        {
+            "business_id": "-0M3o2uWBnQZwd3hmfEwuw",
+            "checkin_times": [
+                {"date": "2016-09-10", "time": "19:26:19"},
+                {"date": "2018-09-08", "time": "14:15:37"},
+                {"date": "2019-09-13", "time": "22:47:25"}
+             ]
+        },
+        {
+            "business_id": "-0RRiWDtfnS16AKCtfvBZg",
+            "checkin_times": [
+                {"date": "2017-05-19", "time": "14:30:16"},
+                {"date": "2017-05-19", "time": "14:30:25"},
+                {"date": "2017-08-28", "time": "15:49:37"},
+                {"date": "2017-09-20", "time": "20:19:51"},
+                {"date": "2017-10-01", "time": "16:31:05"},
+                {"date": "2017-10-01", "time": "16:56:27"},
+                {"date": "2017-12-27", "time": "23:33:20"}
+             ]
+        },
+        {
+            "business_id": "-0Soj75v-XoRcf2ERr8Bmg",
+            "checkin_times": [
+                {"date": "2019-06-05", "time": "18:22:49"}
+             ]
+        },
+        {
+            "business_id": "-0ZumLlFjMh4ZW1z2nXGug",
+            "checkin_times": [
+                {"date": "2011-09-24", "time": "21:37:32"},
+                {"date": "2014-03-10", "time": "20:20:07"},
+                {"date": "2015-05-27", "time": "00:40:24"},
+                {"date": "2015-08-29", "time": "17:58:15"},
+                {"date": "2018-03-16", "time": "15:03:26"}
+             ]
+        },
+        {
+            "business_id": "-0aOudcaAyac0VJbMX-L1g",
+            "checkin_times": [
+                {"date": "2015-03-16", "time": "23:51:16"},
+                {"date": "2015-12-21", "time": "04:48:01"},
+                {"date": "2016-10-28", "time": "20:22:42"},
+                {"date": "2016-10-28", "time": "20:23:00"}
+             ]
+        },
+        {
+            "business_id": "-0b86isaXMY0v4g-V8GZ9Q",
+            "checkin_times": [
+                {"date": "2013-10-22", "time": "16:49:21"},
+                {"date": "2014-11-21", "time": "17:39:24"}
+             ]
+        },
+        {
+            "business_id": "-0d-BfFSU0bwLcnMaGRxYw",
+            "checkin_times": [
+                {"date": "2014-08-07", "time": "18:30:48"},
+                {"date": "2014-09-16", "time": "20:41:45"},
+                {"date": "2014-10-12", "time": "23:22:27"},
+                {"date": "2015-07-21", "time": "20:43:56"},
+                {"date": "2015-07-21", "time": "20:45:07"}
+             ]
+        },
+        {
+            "business_id": "-0jz6c3C6i7RG7Ag22K-Pg",
+            "checkin_times": [
+                {"date": "2015-05-02", "time": "19:49:05"},
+                {"date": "2015-05-06", "time": "03:52:18"},
+                {"date": "2015-09-26", "time": "01:13:19"}
+             ]
+        },
+        {
+            "business_id": "-0y3MZU2oYP8r1ruDP1bfQ",
+            "checkin_times": [
+                {"date": "2015-04-11", "time": "13:14:14"},
+                {"date": "2015-11-21", "time": "16:05:56"},
+                {"date": "2016-05-06", "time": "14:10:04"},
+                {"date": "2017-08-09", "time": "15:15:10"},
+                {"date": "2017-10-21", "time": "15:12:56"}
+             ]
+        },
+        {
+            "business_id": "-1BPe8UjF2_l3nVk-DFUjA",
+            "checkin_times": [
+                {"date": "2015-12-03", "time": "18:44:00"},
+                {"date": "2016-03-17", "time": "18:19:21"},
+                {"date": "2016-11-02", "time": "15:58:38"}
+             ]
+        },
+        {
+            "business_id": "-1E2CQu_38mkghvmZgCCRw",
+            "checkin_times": [
+                {"date": "2019-04-04", "time": "22:02:37"}
+             ]
+        },
+        {
+            "business_id": "-1wzk43IZ5D9Ysu6kzb5xA",
+            "checkin_times": [
+                {"date": "2019-02-27", "time": "14:03:08"}
+             ]
+        },
+        {
+            "business_id": "-23R9P2eG7VTc6DVLjFKzA",
+            "checkin_times": [
+                {"date": "2011-12-21", "time": "19:02:51"},
+                {"date": "2012-04-15", "time": "04:21:39"},
+                {"date": "2012-04-15", "time": "14:23:56"},
+                {"date": "2013-06-30", "time": "22:39:51"},
+                {"date": "2013-10-04", "time": "20:34:13"},
+                {"date": "2014-07-16", "time": "02:28:40"}
+             ]
+        },
+        {
+            "business_id": "-26MGfikhJiTfCI-GqmzhQ",
+            "checkin_times": [
+                {"date": "2018-06-13", "time": "20:16:07"}
+             ]
+        },
+        {
+            "business_id": "-2bLuJsMZ0WhI9daurVQNQ",
+            "checkin_times": [
+                {"date": "2015-05-29", "time": "16:46:17"},
+                {"date": "2015-06-01", "time": "15:03:53"}
+             ]
+        },
+        {
+            "business_id": "-2hDBMaza_ldqnZdiU06LQ",
+            "checkin_times": [
+                {"date": "2011-10-08", "time": "12:02:23"},
+                {"date": "2014-08-18", "time": "02:11:11"},
+                {"date": "2016-01-07", "time": "05:27:51"},
+                {"date": "2016-10-21", "time": "20:15:55"},
+                {"date": "2016-12-01", "time": "03:57:10"},
+                {"date": "2016-12-29", "time": "01:54:42"},
+                {"date": "2018-07-22", "time": "19:55:31"},
+                {"date": "2018-09-07", "time": "01:42:54"},
+                {"date": "2019-03-08", "time": "03:41:06"}
+             ]
+        }
+    ]
+);
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/insert-upsert-delete/closed/use-case-3/use-case-3.7.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/insert-upsert-delete/closed/use-case-3/use-case-3.7.query.sqlpp
new file mode 100644
index 0000000..2757b02
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/insert-upsert-delete/closed/use-case-3/use-case-3.7.query.sqlpp
@@ -0,0 +1,25 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+SET `compiler.arrayindex` "true";
+
+USE TestYelp;
+
+SELECT COUNT(*)
+FROM YelpCheckin C, C.checkin_times D
+WHERE D.date > " ";
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/insert-upsert-delete/closed/use-case-4/use-case-4.1.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/insert-upsert-delete/closed/use-case-4/use-case-4.1.ddl.sqlpp
new file mode 100644
index 0000000..590c9d3
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/insert-upsert-delete/closed/use-case-4/use-case-4.1.ddl.sqlpp
@@ -0,0 +1,41 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+/*
+ * Description: Verify the following DML actions for an array->record->array->atomic index:
+ *     1) Insert into an empty index (non bulk-load operation).
+ *     2) Delete all-but-one entry from the index.
+ *     3) Upsert all entries into the index.
+ */
+
+DROP DATAVERSE TestYelp IF EXISTS;
+CREATE DATAVERSE TestYelp;
+USE TestYelp;
+
+CREATE TYPE CheckinType AS {
+    business_id: string,
+    checkin_times: [{
+        dates: [string],
+        times: [string]
+    }]
+};
+
+CREATE DATASET YelpCheckin(CheckinType) PRIMARY KEY business_id;
+CREATE INDEX IdxYelpCheckinDates ON YelpCheckin (UNNEST checkin_times UNNEST dates);
+CREATE INDEX IdxYelpCheckinTimes ON YelpCheckin (UNNEST checkin_times UNNEST times);
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/insert-upsert-delete/closed/use-case-4/use-case-4.2.update.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/insert-upsert-delete/closed/use-case-4/use-case-4.2.update.sqlpp
new file mode 100644
index 0000000..0360912
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/insert-upsert-delete/closed/use-case-4/use-case-4.2.update.sqlpp
@@ -0,0 +1,425 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+USE TestYelp;
+
+INSERT INTO YelpCheckin (
+      [
+        {
+            "business_id": "--1UhMGODdWsrMastO9DZw",
+            "checkin_times": [
+                {
+                    "dates": ["2016-04-26", "2016-08-30", "2016-10-15", "2016-11-18"],
+                    "times": ["19:49:16", "18:36:57", "02:45:18", "01:54:50"]
+                },
+                {
+                    "dates": ["2017-04-20", "2017-05-03"],
+                    "times": ["18:39:06", "17:58:02"]
+                },
+                {
+                    "dates": ["2019-03-19"],
+                    "times": ["22:04:48"]
+                }
+             ]
+        },
+        {
+            "business_id": "--EF5N7P70J_UYBTPypYlA",
+            "checkin_times": [
+                {
+                    "dates": ["2018-05-25", "2018-09-18"],
+                    "times": ["19:52:07", "16:09:44"]
+                },
+                {
+                    "dates": ["2019-10-18"],
+                    "times": ["21:29:09"]
+                }
+             ]
+        },
+        {
+            "business_id": "--Ni3oJ4VOqfOEu7Sj2Vzg",
+            "checkin_times": [
+                {
+                    "dates": ["2019-06-07"],
+                    "times": ["17:54:58"]
+                }
+             ]
+        },
+        {
+            "business_id": "--Y1Adl1YUWfYIRSd8vkmA",
+            "checkin_times": [
+                {
+                    "dates": ["2011-05-03", "2011-08-23"],
+                    "times": ["20:54:05", "20:49:45"]
+                },
+                {
+                    "dates": ["2014-12-04"],
+                    "times": ["06:13:01"]
+                },
+                {
+                    "dates": ["2016-11-16"],
+                    "times": ["19:25:55"]
+                }
+             ]
+        },
+        {
+            "business_id": "--YPwqIlRJrhHkJcjY3eiA",
+            "checkin_times": [
+               {
+                    "dates": ["2016-06-18", "2016-10-15"],
+                    "times": ["21:35:45", "18:17:51"]
+                }
+             ]
+        },
+        {
+            "business_id": "--e8PjCNhEz32pprnPhCwQ",
+            "checkin_times": [
+                {
+                    "dates": ["2015-04-02"],
+                    "times": ["21:45:17"]
+                }
+             ]
+        },
+        {
+            "business_id": "--kinfHwmtdjz03g8B8z8Q",
+            "checkin_times": [
+                {
+                    "dates": ["2014-08-27"],
+                    "times": ["17:49:18"]
+                },
+                {
+                    "dates": ["2015-12-19"],
+                    "times": ["21:30:31"]
+                },
+                {
+                    "dates": ["2018-11-27"],
+                    "times": ["15:53:50"]
+                }
+             ]
+        },
+        {
+            "business_id": "--q6datkI-f0EoVheXNEeQ",
+            "checkin_times": [
+                {
+                    "dates": ["2014-01-28", "2014-11-16"],
+                    "times": ["20:56:04", "16:11:58"]
+                },
+                {
+                    "dates": ["2015-11-15", "2015-11-15"],
+                    "times": ["19:21:53", "19:33:39"]
+                }
+             ]
+        },
+        {
+            "business_id": "--qvQS4MigHPykD2GV0-zw",
+            "checkin_times": [
+                {
+                    "dates": ["2019-04-11"],
+                    "times": ["18:30:12"]
+                }
+             ]
+        },
+        {
+            "business_id": "--wIGbLEhlpl_UeAIyDmZQ",
+            "checkin_times": [
+                {
+                    "dates": ["2015-06-06"],
+                    "times": ["20:01:06"]
+                },
+                {
+                    "dates": ["2019-03-14"],
+                    "times": ["22:01:52"]
+                }
+             ]
+        },
+        {
+            "business_id": "-0FA-Qdi3SPYIoJz9UQw-A",
+            "checkin_times": [
+                {
+                    "dates": ["2018-09-29", "2018-10-20", "2018-10-20"],
+                    "times": ["18:55:17", "16:48:05", "22:20:24"]
+                }
+             ]
+        },
+        {
+            "business_id": "-0Hj1hb_XW6ybWq2M7QhGA",
+            "checkin_times": [
+                {
+                    "dates": ["2011-04-23"],
+                    "times": ["21:11:22"]
+                },
+                {
+                    "dates": ["2014-05-04", "2014-05-11", "2014-06-04"],
+                    "times": ["19:42:48", "19:16:08", "19:14:18"]
+                },
+                {
+                    "dates": ["2015-12-05"],
+                    "times": ["19:22:42"]
+                },
+                {
+                    "dates": ["2017-05-15"],
+                    "times": ["23:19:00"]
+                }
+             ]
+        },
+        {
+            "business_id": "-0KMvRFwDWdVBeTpT11iHw",
+            "checkin_times": [
+                {
+                    "dates": ["2012-07-13"],
+                    "times": ["21:43:57"]
+                },
+                {
+                    "dates": ["2016-12-24"],
+                    "times": ["02:27:31"]
+                },
+                {
+                    "dates": ["2017-08-31"],
+                    "times": ["00:35:26"]
+                }
+             ]
+        },
+        {
+            "business_id": "-0LPtgJC31FWMrMv317p0Q",
+            "checkin_times": [
+                {
+                    "dates": ["2013-04-13", "2013-08-19", "2013-10-04"],
+                    "times": ["12:35:33", "23:35:49", "19:14:56"]
+                }
+
+             ]
+        },
+        {
+            "business_id": "-0M3o2uWBnQZwd3hmfEwuw",
+            "checkin_times": [
+                {
+                    "dates": ["2016-09-10"],
+                    "times": ["19:26:19"]
+                },
+                {
+                    "dates": ["2018-09-08"],
+                    "times": ["14:15:37"]
+                },
+                {
+                    "dates": ["2019-09-13"],
+                    "times": ["22:47:25"]
+                }
+             ]
+        },
+        {
+            "business_id": "-0RRiWDtfnS16AKCtfvBZg",
+            "checkin_times": [
+                 {
+                    "dates": ["2017-05-19", "2017-05-19", "2017-08-28", "2017-09-20", "2017-10-01", "2017-10-01", "2017-12-27"],
+                    "times": ["14:30:16", "14:30:25", "15:49:37", "20:19:51", "16:31:05", "16:56:27", "23:33:20"]
+                }
+             ]
+        },
+        {
+            "business_id": "-0Soj75v-XoRcf2ERr8Bmg",
+            "checkin_times": [
+                {
+                    "dates": ["2019-06-05"],
+                    "times": ["18:22:49"]
+                }
+             ]
+        },
+        {
+            "business_id": "-0ZumLlFjMh4ZW1z2nXGug",
+            "checkin_times": [
+                {
+                    "dates": ["2011-09-24"],
+                    "times": ["21:37:32"]
+                },
+                {
+                    "dates": ["2014-03-10"],
+                    "times": ["20:20:07"]
+                },
+                {
+                    "dates": ["2015-05-27", "2015-08-29"],
+                    "times": ["00:40:24", "17:58:15"]
+                },
+                {
+                    "dates": ["2018-03-16"],
+                    "times": ["15:03:26"]
+                }
+             ]
+        },
+        {
+            "business_id": "-0aOudcaAyac0VJbMX-L1g",
+            "checkin_times": [
+                {
+                    "dates": ["2015-03-16", "2015-12-21"],
+                    "times": ["23:51:16", "04:48:01"]
+                },
+                {
+                    "dates": ["2016-10-28", "2016-10-28"],
+                    "times": ["20:22:42", "20:23:00"]
+                }
+             ]
+        },
+        {
+            "business_id": "-0b86isaXMY0v4g-V8GZ9Q",
+            "checkin_times": [
+                {
+                    "dates": ["2013-10-22"],
+                    "times": ["16:49:21"]
+                },
+                {
+                    "dates": ["2014-11-21"],
+                    "times": ["17:39:24"]
+                }
+             ]
+        },
+        {
+            "business_id": "-0d-BfFSU0bwLcnMaGRxYw",
+            "checkin_times": [
+                {
+                    "dates": ["2014-08-07", "2014-09-16", "2014-10-12"],
+                    "times": ["18:30:48", "20:41:45", "23:22:27"]
+                },
+                {
+                    "dates": ["2015-07-21", "2015-07-21"],
+                    "times": ["20:43:56", "20:45:07"]
+                }
+             ]
+        },
+        {
+            "business_id": "-0jz6c3C6i7RG7Ag22K-Pg",
+            "checkin_times": [
+                {
+                    "dates": ["2015-05-02", "2015-05-06", "2015-09-26"],
+                    "times": ["19:49:05", "03:52:18", "01:13:19"]
+                }
+
+             ]
+        },
+        {
+            "business_id": "-0y3MZU2oYP8r1ruDP1bfQ",
+            "checkin_times": [
+                {
+                    "dates": ["2015-04-11", "2015-11-21"],
+                    "times": ["13:14:14", "16:05:56"]
+                },
+                {
+                    "dates": ["2016-05-06"],
+                    "times": ["14:10:04"]
+                },
+                {
+                    "dates": ["2017-08-09", "2017-10-21"],
+                    "times": ["15:15:10", "15:12:56"]
+                }
+             ]
+        },
+        {
+            "business_id": "-1BPe8UjF2_l3nVk-DFUjA",
+            "checkin_times": [
+                {
+                    "dates": ["2015-12-03"],
+                    "times": ["18:44:00"]
+                },
+                {
+                    "dates": ["2016-03-17", "2016-11-02"],
+                    "times": ["18:19:21", "15:58:38"]
+                }
+             ]
+        },
+        {
+            "business_id": "-1E2CQu_38mkghvmZgCCRw",
+            "checkin_times": [
+                {
+                    "dates": ["2019-04-04"],
+                    "times": ["22:02:37"]
+                }
+             ]
+        },
+        {
+            "business_id": "-1wzk43IZ5D9Ysu6kzb5xA",
+            "checkin_times": [
+                {
+                    "dates": ["2019-02-27"],
+                    "times": ["14:03:08"]
+                }
+             ]
+        },
+        {
+            "business_id": "-23R9P2eG7VTc6DVLjFKzA",
+            "checkin_times": [
+                {
+                    "dates": ["2011-12-21"],
+                    "times": ["19:02:51"]
+                },
+                {
+                    "dates": ["2012-04-15", "2012-04-15"],
+                    "times": ["04:21:39", "14:23:56"]
+                },
+                {
+                    "dates": ["2013-06-30", "2013-10-04"],
+                    "times": ["22:39:51", "20:34:13"]
+                },
+                {
+                    "dates": ["2014-07-16"],
+                    "times": ["02:28:40"]
+                }
+             ]
+        },
+        {
+            "business_id": "-26MGfikhJiTfCI-GqmzhQ",
+            "checkin_times": [
+                {
+                    "dates": ["2018-06-13"],
+                    "times": ["20:16:07"]
+                }
+             ]
+        },
+        {
+            "business_id": "-2bLuJsMZ0WhI9daurVQNQ",
+            "checkin_times": [
+                {
+                    "dates": ["2015-05-29", "2015-06-01"],
+                    "times": ["16:46:17", "15:03:53"]
+                }
+             ]
+        },
+        {
+            "business_id": "-2hDBMaza_ldqnZdiU06LQ",
+            "checkin_times": [
+                {
+                    "dates": ["2011-10-08"],
+                    "times": ["12:02:23"]
+                },
+                {
+                    "dates": ["2014-08-18"],
+                    "times": ["02:11:11"]
+                },
+                {
+                    "dates": ["2016-01-07", "2016-10-21", "2016-12-01", "2016-12-29"],
+                    "times": ["05:27:51", "20:15:55", "03:57:10", "01:54:42"]
+                },
+                {
+                    "dates": ["2018-07-22", "2018-09-07"],
+                    "times": ["19:55:31", "01:42:54"]
+                },
+                {
+                    "dates": ["2019-03-08"],
+                    "times": ["03:41:06"]
+                }
+             ]
+        }
+    ]
+);
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/insert-upsert-delete/closed/use-case-4/use-case-4.3.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/insert-upsert-delete/closed/use-case-4/use-case-4.3.query.sqlpp
new file mode 100644
index 0000000..3406a35
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/insert-upsert-delete/closed/use-case-4/use-case-4.3.query.sqlpp
@@ -0,0 +1,25 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+SET `compiler.arrayindex` "true";
+
+USE TestYelp;
+
+SELECT COUNT(*)
+FROM YelpCheckin C, C.checkin_times CT, CT.dates D
+WHERE D > " ";
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/insert-upsert-delete/closed/use-case-4/use-case-4.4.update.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/insert-upsert-delete/closed/use-case-4/use-case-4.4.update.sqlpp
new file mode 100644
index 0000000..ea1dba3
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/insert-upsert-delete/closed/use-case-4/use-case-4.4.update.sqlpp
@@ -0,0 +1,23 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+USE TestYelp;
+
+DELETE FROM YelpCheckin C
+WHERE C.business_id != "--1UhMGODdWsrMastO9DZw";
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/insert-upsert-delete/closed/use-case-4/use-case-4.5.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/insert-upsert-delete/closed/use-case-4/use-case-4.5.query.sqlpp
new file mode 100644
index 0000000..3406a35
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/insert-upsert-delete/closed/use-case-4/use-case-4.5.query.sqlpp
@@ -0,0 +1,25 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+SET `compiler.arrayindex` "true";
+
+USE TestYelp;
+
+SELECT COUNT(*)
+FROM YelpCheckin C, C.checkin_times CT, CT.dates D
+WHERE D > " ";
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/insert-upsert-delete/closed/use-case-4/use-case-4.6.update.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/insert-upsert-delete/closed/use-case-4/use-case-4.6.update.sqlpp
new file mode 100644
index 0000000..b5b6c72
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/insert-upsert-delete/closed/use-case-4/use-case-4.6.update.sqlpp
@@ -0,0 +1,425 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+USE TestYelp;
+
+UPSERT INTO YelpCheckin (
+      [
+        {
+            "business_id": "--1UhMGODdWsrMastO9DZw",
+            "checkin_times": [
+                {
+                    "dates": ["2016-04-26", "2016-08-30", "2016-10-15", "2016-11-18"],
+                    "times": ["19:49:16", "18:36:57", "02:45:18", "01:54:50"]
+                },
+                {
+                    "dates": ["2017-04-20", "2017-05-03"],
+                    "times": ["18:39:06", "17:58:02"]
+                },
+                {
+                    "dates": ["2019-03-19"],
+                    "times": ["22:04:48"]
+                }
+             ]
+        },
+        {
+            "business_id": "--EF5N7P70J_UYBTPypYlA",
+            "checkin_times": [
+                {
+                    "dates": ["2018-05-25", "2018-09-18"],
+                    "times": ["19:52:07", "16:09:44"]
+                },
+                {
+                    "dates": ["2019-10-18"],
+                    "times": ["21:29:09"]
+                }
+             ]
+        },
+        {
+            "business_id": "--Ni3oJ4VOqfOEu7Sj2Vzg",
+            "checkin_times": [
+                {
+                    "dates": ["2019-06-07"],
+                    "times": ["17:54:58"]
+                }
+             ]
+        },
+        {
+            "business_id": "--Y1Adl1YUWfYIRSd8vkmA",
+            "checkin_times": [
+                {
+                    "dates": ["2011-05-03", "2011-08-23"],
+                    "times": ["20:54:05", "20:49:45"]
+                },
+                {
+                    "dates": ["2014-12-04"],
+                    "times": ["06:13:01"]
+                },
+                {
+                    "dates": ["2016-11-16"],
+                    "times": ["19:25:55"]
+                }
+             ]
+        },
+        {
+            "business_id": "--YPwqIlRJrhHkJcjY3eiA",
+            "checkin_times": [
+               {
+                    "dates": ["2016-06-18", "2016-10-15"],
+                    "times": ["21:35:45", "18:17:51"]
+                }
+             ]
+        },
+        {
+            "business_id": "--e8PjCNhEz32pprnPhCwQ",
+            "checkin_times": [
+                {
+                    "dates": ["2015-04-02"],
+                    "times": ["21:45:17"]
+                }
+             ]
+        },
+        {
+            "business_id": "--kinfHwmtdjz03g8B8z8Q",
+            "checkin_times": [
+                {
+                    "dates": ["2014-08-27"],
+                    "times": ["17:49:18"]
+                },
+                {
+                    "dates": ["2015-12-19"],
+                    "times": ["21:30:31"]
+                },
+                {
+                    "dates": ["2018-11-27"],
+                    "times": ["15:53:50"]
+                }
+             ]
+        },
+        {
+            "business_id": "--q6datkI-f0EoVheXNEeQ",
+            "checkin_times": [
+                {
+                    "dates": ["2014-01-28", "2014-11-16"],
+                    "times": ["20:56:04", "16:11:58"]
+                },
+                {
+                    "dates": ["2015-11-15", "2015-11-15"],
+                    "times": ["19:21:53", "19:33:39"]
+                }
+             ]
+        },
+        {
+            "business_id": "--qvQS4MigHPykD2GV0-zw",
+            "checkin_times": [
+                {
+                    "dates": ["2019-04-11"],
+                    "times": ["18:30:12"]
+                }
+             ]
+        },
+        {
+            "business_id": "--wIGbLEhlpl_UeAIyDmZQ",
+            "checkin_times": [
+                {
+                    "dates": ["2015-06-06"],
+                    "times": ["20:01:06"]
+                },
+                {
+                    "dates": ["2019-03-14"],
+                    "times": ["22:01:52"]
+                }
+             ]
+        },
+        {
+            "business_id": "-0FA-Qdi3SPYIoJz9UQw-A",
+            "checkin_times": [
+                {
+                    "dates": ["2018-09-29", "2018-10-20", "2018-10-20"],
+                    "times": ["18:55:17", "16:48:05", "22:20:24"]
+                }
+             ]
+        },
+        {
+            "business_id": "-0Hj1hb_XW6ybWq2M7QhGA",
+            "checkin_times": [
+                {
+                    "dates": ["2011-04-23"],
+                    "times": ["21:11:22"]
+                },
+                {
+                    "dates": ["2014-05-04", "2014-05-11", "2014-06-04"],
+                    "times": ["19:42:48", "19:16:08", "19:14:18"]
+                },
+                {
+                    "dates": ["2015-12-05"],
+                    "times": ["19:22:42"]
+                },
+                {
+                    "dates": ["2017-05-15"],
+                    "times": ["23:19:00"]
+                }
+             ]
+        },
+        {
+            "business_id": "-0KMvRFwDWdVBeTpT11iHw",
+            "checkin_times": [
+                {
+                    "dates": ["2012-07-13"],
+                    "times": ["21:43:57"]
+                },
+                {
+                    "dates": ["2016-12-24"],
+                    "times": ["02:27:31"]
+                },
+                {
+                    "dates": ["2017-08-31"],
+                    "times": ["00:35:26"]
+                }
+             ]
+        },
+        {
+            "business_id": "-0LPtgJC31FWMrMv317p0Q",
+            "checkin_times": [
+                {
+                    "dates": ["2013-04-13", "2013-08-19", "2013-10-04"],
+                    "times": ["12:35:33", "23:35:49", "19:14:56"]
+                }
+
+             ]
+        },
+        {
+            "business_id": "-0M3o2uWBnQZwd3hmfEwuw",
+            "checkin_times": [
+                {
+                    "dates": ["2016-09-10"],
+                    "times": ["19:26:19"]
+                },
+                {
+                    "dates": ["2018-09-08"],
+                    "times": ["14:15:37"]
+                },
+                {
+                    "dates": ["2019-09-13"],
+                    "times": ["22:47:25"]
+                }
+             ]
+        },
+        {
+            "business_id": "-0RRiWDtfnS16AKCtfvBZg",
+            "checkin_times": [
+                 {
+                    "dates": ["2017-05-19", "2017-05-19", "2017-08-28", "2017-09-20", "2017-10-01", "2017-10-01", "2017-12-27"],
+                    "times": ["14:30:16", "14:30:25", "15:49:37", "20:19:51", "16:31:05", "16:56:27", "23:33:20"]
+                }
+             ]
+        },
+        {
+            "business_id": "-0Soj75v-XoRcf2ERr8Bmg",
+            "checkin_times": [
+                {
+                    "dates": ["2019-06-05"],
+                    "times": ["18:22:49"]
+                }
+             ]
+        },
+        {
+            "business_id": "-0ZumLlFjMh4ZW1z2nXGug",
+            "checkin_times": [
+                {
+                    "dates": ["2011-09-24"],
+                    "times": ["21:37:32"]
+                },
+                {
+                    "dates": ["2014-03-10"],
+                    "times": ["20:20:07"]
+                },
+                {
+                    "dates": ["2015-05-27", "2015-08-29"],
+                    "times": ["00:40:24", "17:58:15"]
+                },
+                {
+                    "dates": ["2018-03-16"],
+                    "times": ["15:03:26"]
+                }
+             ]
+        },
+        {
+            "business_id": "-0aOudcaAyac0VJbMX-L1g",
+            "checkin_times": [
+                {
+                    "dates": ["2015-03-16", "2015-12-21"],
+                    "times": ["23:51:16", "04:48:01"]
+                },
+                {
+                    "dates": ["2016-10-28", "2016-10-28"],
+                    "times": ["20:22:42", "20:23:00"]
+                }
+             ]
+        },
+        {
+            "business_id": "-0b86isaXMY0v4g-V8GZ9Q",
+            "checkin_times": [
+                {
+                    "dates": ["2013-10-22"],
+                    "times": ["16:49:21"]
+                },
+                {
+                    "dates": ["2014-11-21"],
+                    "times": ["17:39:24"]
+                }
+             ]
+        },
+        {
+            "business_id": "-0d-BfFSU0bwLcnMaGRxYw",
+            "checkin_times": [
+                {
+                    "dates": ["2014-08-07", "2014-09-16", "2014-10-12"],
+                    "times": ["18:30:48", "20:41:45", "23:22:27"]
+                },
+                {
+                    "dates": ["2015-07-21", "2015-07-21"],
+                    "times": ["20:43:56", "20:45:07"]
+                }
+             ]
+        },
+        {
+            "business_id": "-0jz6c3C6i7RG7Ag22K-Pg",
+            "checkin_times": [
+                {
+                    "dates": ["2015-05-02", "2015-05-06", "2015-09-26"],
+                    "times": ["19:49:05", "03:52:18", "01:13:19"]
+                }
+
+             ]
+        },
+        {
+            "business_id": "-0y3MZU2oYP8r1ruDP1bfQ",
+            "checkin_times": [
+                {
+                    "dates": ["2015-04-11", "2015-11-21"],
+                    "times": ["13:14:14", "16:05:56"]
+                },
+                {
+                    "dates": ["2016-05-06"],
+                    "times": ["14:10:04"]
+                },
+                {
+                    "dates": ["2017-08-09", "2017-10-21"],
+                    "times": ["15:15:10", "15:12:56"]
+                }
+             ]
+        },
+        {
+            "business_id": "-1BPe8UjF2_l3nVk-DFUjA",
+            "checkin_times": [
+                {
+                    "dates": ["2015-12-03"],
+                    "times": ["18:44:00"]
+                },
+                {
+                    "dates": ["2016-03-17", "2016-11-02"],
+                    "times": ["18:19:21", "15:58:38"]
+                }
+             ]
+        },
+        {
+            "business_id": "-1E2CQu_38mkghvmZgCCRw",
+            "checkin_times": [
+                {
+                    "dates": ["2019-04-04"],
+                    "times": ["22:02:37"]
+                }
+             ]
+        },
+        {
+            "business_id": "-1wzk43IZ5D9Ysu6kzb5xA",
+            "checkin_times": [
+                {
+                    "dates": ["2019-02-27"],
+                    "times": ["14:03:08"]
+                }
+             ]
+        },
+        {
+            "business_id": "-23R9P2eG7VTc6DVLjFKzA",
+            "checkin_times": [
+                {
+                    "dates": ["2011-12-21"],
+                    "times": ["19:02:51"]
+                },
+                {
+                    "dates": ["2012-04-15", "2012-04-15"],
+                    "times": ["04:21:39", "14:23:56"]
+                },
+                {
+                    "dates": ["2013-06-30", "2013-10-04"],
+                    "times": ["22:39:51", "20:34:13"]
+                },
+                {
+                    "dates": ["2014-07-16"],
+                    "times": ["02:28:40"]
+                }
+             ]
+        },
+        {
+            "business_id": "-26MGfikhJiTfCI-GqmzhQ",
+            "checkin_times": [
+                {
+                    "dates": ["2018-06-13"],
+                    "times": ["20:16:07"]
+                }
+             ]
+        },
+        {
+            "business_id": "-2bLuJsMZ0WhI9daurVQNQ",
+            "checkin_times": [
+                {
+                    "dates": ["2015-05-29", "2015-06-01"],
+                    "times": ["16:46:17", "15:03:53"]
+                }
+             ]
+        },
+        {
+            "business_id": "-2hDBMaza_ldqnZdiU06LQ",
+            "checkin_times": [
+                {
+                    "dates": ["2011-10-08"],
+                    "times": ["12:02:23"]
+                },
+                {
+                    "dates": ["2014-08-18"],
+                    "times": ["02:11:11"]
+                },
+                {
+                    "dates": ["2016-01-07", "2016-10-21", "2016-12-01", "2016-12-29"],
+                    "times": ["05:27:51", "20:15:55", "03:57:10", "01:54:42"]
+                },
+                {
+                    "dates": ["2018-07-22", "2018-09-07"],
+                    "times": ["19:55:31", "01:42:54"]
+                },
+                {
+                    "dates": ["2019-03-08"],
+                    "times": ["03:41:06"]
+                }
+             ]
+        }
+    ]
+);
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/insert-upsert-delete/closed/use-case-4/use-case-4.7.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/insert-upsert-delete/closed/use-case-4/use-case-4.7.query.sqlpp
new file mode 100644
index 0000000..3406a35
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/insert-upsert-delete/closed/use-case-4/use-case-4.7.query.sqlpp
@@ -0,0 +1,25 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+SET `compiler.arrayindex` "true";
+
+USE TestYelp;
+
+SELECT COUNT(*)
+FROM YelpCheckin C, C.checkin_times CT, CT.dates D
+WHERE D > " ";
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/insert-upsert-delete/closed/with-additional-atomic-index/with-additional-atomic-index.1.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/insert-upsert-delete/closed/with-additional-atomic-index/with-additional-atomic-index.1.ddl.sqlpp
new file mode 100644
index 0000000..1697fb3
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/insert-upsert-delete/closed/with-additional-atomic-index/with-additional-atomic-index.1.ddl.sqlpp
@@ -0,0 +1,39 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+/*
+ * Description: Verify the following DML actions for an array->atomic index, with an additional atomic index:
+ *     1) Insert into an empty index (non bulk-load operation), additionally with two records that have no array-index qualifying entries.
+ *     2) Delete all-but-one entry from the index.
+ *     3) Upsert all *original* (all records have qualifying array-index entries now) entries into the index.
+ */
+
+DROP DATAVERSE TestYelp IF EXISTS;
+CREATE DATAVERSE TestYelp;
+USE TestYelp;
+
+CREATE TYPE CheckinType AS {
+    checkin_id: int,
+    business_id: string,
+    dates: [string]
+};
+
+CREATE DATASET YelpCheckin(CheckinType) PRIMARY KEY checkin_id;
+CREATE INDEX IdxYelpCheckinDates ON YelpCheckin (UNNEST dates);
+CREATE INDEX IdxYelpCheckinBusinessID ON YelpCheckin (business_id);
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/insert-upsert-delete/closed/with-additional-atomic-index/with-additional-atomic-index.2.update.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/insert-upsert-delete/closed/with-additional-atomic-index/with-additional-atomic-index.2.update.sqlpp
new file mode 100644
index 0000000..8ee1570
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/insert-upsert-delete/closed/with-additional-atomic-index/with-additional-atomic-index.2.update.sqlpp
@@ -0,0 +1,302 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+USE TestYelp;
+
+INSERT INTO YelpCheckin (
+    [
+        {
+            "checkin_id": 1,
+            "business_id": "--1UhMGODdWsrMastO9DZw",
+            "dates": [
+                "2016-04-26 19:49:16",
+                "2016-08-30 18:36:57",
+                "2016-10-15 02:45:18",
+                "2016-11-18 01:54:50",
+                "2017-04-20 18:39:06",
+                "2017-05-03 17:58:02",
+                "2019-03-19 22:04:48"
+            ]
+        },
+        {
+            "checkin_id": 2,
+            "business_id": "--EF5N7P70J_UYBTPypYlA",
+            "dates": [
+                "2018-05-25 19:52:07",
+                "2018-09-18 16:09:44",
+                "2019-10-18 21:29:09"
+            ]
+        },
+        {
+            "checkin_id": 3,
+            "business_id": "--Ni3oJ4VOqfOEu7Sj2Vzg",
+            "dates": [
+                "2019-06-07 17:54:58"
+            ]
+        },
+        {
+            "checkin_id": 4,
+            "business_id": "--Y1Adl1YUWfYIRSd8vkmA",
+            "dates": [
+                "2011-05-03 20:54:05",
+                "2011-08-23 20:49:45",
+                "2014-12-04 06:13:01",
+                "2016-11-16 19:25:55"
+            ]
+        },
+        {
+            "checkin_id": 5,
+            "business_id": "--YPwqIlRJrhHkJcjY3eiA",
+            "dates": [
+                "2016-06-18 21:35:45",
+                "2016-10-15 18:17:51"
+            ]
+        },
+        {
+            "checkin_id": 6,
+            "business_id": "--e8PjCNhEz32pprnPhCwQ",
+            "dates": [
+                "2015-04-02 21:45:17"
+            ]
+        },
+        {
+            "checkin_id": 7,
+            "business_id": "--kinfHwmtdjz03g8B8z8Q",
+            "dates": [
+                "2014-08-27 17:49:18",
+                "2015-12-19 21:30:31",
+                "2018-11-27 15:53:50"
+            ]
+        },
+        {
+            "checkin_id": 8,
+            "business_id": "--q6datkI-f0EoVheXNEeQ",
+            "dates": [
+                "2014-01-28 20:56:04",
+                "2014-11-16 16:11:58",
+                "2015-11-15 19:21:53",
+                "2015-11-15 19:33:39"
+            ]
+        },
+        {
+            "checkin_id": 9,
+            "business_id": "--qvQS4MigHPykD2GV0-zw",
+            "dates": [
+                "2019-04-11 18:30:12"
+            ]
+        },
+        {
+            "checkin_id": 10,
+            "business_id": "--wIGbLEhlpl_UeAIyDmZQ",
+            "dates": [
+                "2015-06-06 20:01:06",
+                "2019-03-14 22:01:52"
+            ]
+        },
+        {
+            "checkin_id": 11,
+            "business_id": "-0FA-Qdi3SPYIoJz9UQw-A",
+            "dates": [
+                "2018-09-29 18:55:17",
+                "2018-10-20 16:48:05",
+                "2018-10-20 22:20:24"
+            ]
+        },
+        {
+            "checkin_id": 12,
+            "business_id": "-0Hj1hb_XW6ybWq2M7QhGA",
+            "dates": [
+                "2011-04-23 21:11:22",
+                "2014-05-04 19:42:48",
+                "2014-05-11 19:16:08",
+                "2014-06-04 19:14:18",
+                "2015-12-05 19:22:42",
+                "2017-05-15 23:19:00"
+            ]
+        },
+        {
+            "checkin_id": 13,
+            "business_id": "-0KMvRFwDWdVBeTpT11iHw",
+            "dates": [
+                "2012-07-13 21:43:57",
+                "2016-12-24 02:27:31",
+                "2017-08-31 00:35:26"
+            ]
+        },
+        {
+            "checkin_id": 14,
+            "business_id": "-0LPtgJC31FWMrMv317p0Q",
+            "dates": [
+                "2013-04-13 12:35:33",
+                "2013-08-19 23:35:49",
+                "2013-10-04 19:14:56"
+            ]
+        },
+        {
+            "checkin_id": 15,
+            "business_id": "-0M3o2uWBnQZwd3hmfEwuw",
+            "dates": [
+                "2016-09-10 19:26:19",
+                "2018-09-08 14:15:37",
+                "2019-09-13 22:47:25"
+            ]
+        },
+        {
+            "checkin_id": 16,
+            "business_id": "-0RRiWDtfnS16AKCtfvBZg",
+            "dates": [
+                "2017-05-19 14:30:16",
+                "2017-05-19 14:30:25",
+                "2017-08-28 15:49:37",
+                "2017-09-20 20:19:51",
+                "2017-10-01 16:31:05",
+                "2017-10-01 16:56:27",
+                "2017-12-27 23:33:20"
+            ]
+        },
+        {
+            "checkin_id": 17,
+            "business_id": "-0Soj75v-XoRcf2ERr8Bmg",
+            "dates": [
+                "2019-06-05 18:22:49"
+            ]
+        },
+        {
+            "checkin_id": 18,
+            "business_id": "-0ZumLlFjMh4ZW1z2nXGug",
+            "dates": [
+                "2011-09-24 21:37:32",
+                "2014-03-10 20:20:07",
+                "2015-05-27 00:40:24",
+                "2015-08-29 17:58:15",
+                "2018-03-16 15:03:26"
+            ]
+        },
+        {
+            "checkin_id": 19,
+            "business_id": "-0aOudcaAyac0VJbMX-L1g",
+            "dates": [
+                "2015-03-16 23:51:16",
+                "2015-12-21 04:48:01",
+                "2016-10-28 20:22:42",
+                "2016-10-28 20:23:00"
+            ]
+        },
+        {
+            "checkin_id": 20,
+            "business_id": "-0b86isaXMY0v4g-V8GZ9Q",
+            "dates": [
+                "2013-10-22 16:49:21",
+                "2014-11-21 17:39:24"
+            ]
+        },
+        {
+            "checkin_id": 21,
+            "business_id": "-0d-BfFSU0bwLcnMaGRxYw",
+            "dates": [
+                "2014-08-07 18:30:48",
+                "2014-09-16 20:41:45",
+                "2014-10-12 23:22:27",
+                "2015-07-21 20:43:56",
+                "2015-07-21 20:45:07"
+            ]
+        },
+        {
+            "checkin_id": 22,
+            "business_id": "-0jz6c3C6i7RG7Ag22K-Pg",
+            "dates": [
+                "2015-05-02 19:49:05",
+                "2015-05-06 03:52:18",
+                "2015-09-26 01:13:19"
+            ]
+        },
+        {
+            "checkin_id": 23,
+            "business_id": "-0y3MZU2oYP8r1ruDP1bfQ",
+            "dates": [
+                "2015-04-11 13:14:14",
+                "2015-11-21 16:05:56",
+                "2016-05-06 14:10:04",
+                "2017-08-09 15:15:10",
+                "2017-10-21 15:12:56"
+            ]
+        },
+        {
+            "checkin_id": 24,
+            "business_id": "-1BPe8UjF2_l3nVk-DFUjA",
+            "dates": [
+                "2015-12-03 18:44:00",
+                "2016-03-17 18:19:21",
+                "2016-11-02 15:58:38"
+            ]
+        },
+        {
+            "checkin_id": 25,
+            "business_id": "-1E2CQu_38mkghvmZgCCRw",
+            "dates": []
+        },
+        {
+            "checkin_id": 26,
+            "business_id": "-1wzk43IZ5D9Ysu6kzb5xA",
+            "dates": []
+        },
+        {
+            "checkin_id": 27,
+            "business_id": "-23R9P2eG7VTc6DVLjFKzA",
+            "dates": [
+                "2011-12-21 19:02:51",
+                "2012-04-15 04:21:39",
+                "2012-04-15 14:23:56",
+                "2013-06-30 22:39:51",
+                "2013-10-04 20:34:13",
+                "2014-07-16 02:28:40"
+            ]
+        },
+        {
+            "checkin_id": 28,
+            "business_id": "-26MGfikhJiTfCI-GqmzhQ",
+            "dates": [
+                "2018-06-13 20:16:07"
+            ]
+        },
+        {
+            "checkin_id": 29,
+            "business_id": "-2bLuJsMZ0WhI9daurVQNQ",
+            "dates": [
+                "2015-05-29 16:46:17",
+                "2015-06-01 15:03:53"
+            ]
+        },
+        {
+            "checkin_id": 30,
+            "business_id": "-2hDBMaza_ldqnZdiU06LQ",
+            "dates": [
+                "2011-10-08 12:02:23",
+                "2014-08-18 02:11:11",
+                "2016-01-07 05:27:51",
+                "2016-10-21 20:15:55",
+                "2016-12-01 03:57:10",
+                "2016-12-29 01:54:42",
+                "2018-07-22 19:55:31",
+                "2018-09-07 01:42:54",
+                "2019-03-08 03:41:06"
+            ]
+        }
+    ]
+);
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/insert-upsert-delete/closed/with-additional-atomic-index/with-additional-atomic-index.3.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/insert-upsert-delete/closed/with-additional-atomic-index/with-additional-atomic-index.3.query.sqlpp
new file mode 100644
index 0000000..69fe7cc
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/insert-upsert-delete/closed/with-additional-atomic-index/with-additional-atomic-index.3.query.sqlpp
@@ -0,0 +1,25 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+SET `compiler.arrayindex` "true";
+
+USE TestYelp;
+
+SELECT COUNT(*)
+FROM YelpCheckin C, C.dates D
+WHERE D > " ";
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/insert-upsert-delete/closed/with-additional-atomic-index/with-additional-atomic-index.4.update.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/insert-upsert-delete/closed/with-additional-atomic-index/with-additional-atomic-index.4.update.sqlpp
new file mode 100644
index 0000000..ea1dba3
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/insert-upsert-delete/closed/with-additional-atomic-index/with-additional-atomic-index.4.update.sqlpp
@@ -0,0 +1,23 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+USE TestYelp;
+
+DELETE FROM YelpCheckin C
+WHERE C.business_id != "--1UhMGODdWsrMastO9DZw";
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/insert-upsert-delete/closed/with-additional-atomic-index/with-additional-atomic-index.5.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/insert-upsert-delete/closed/with-additional-atomic-index/with-additional-atomic-index.5.query.sqlpp
new file mode 100644
index 0000000..69fe7cc
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/insert-upsert-delete/closed/with-additional-atomic-index/with-additional-atomic-index.5.query.sqlpp
@@ -0,0 +1,25 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+SET `compiler.arrayindex` "true";
+
+USE TestYelp;
+
+SELECT COUNT(*)
+FROM YelpCheckin C, C.dates D
+WHERE D > " ";
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/insert-upsert-delete/closed/with-additional-atomic-index/with-additional-atomic-index.6.update.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/insert-upsert-delete/closed/with-additional-atomic-index/with-additional-atomic-index.6.update.sqlpp
new file mode 100644
index 0000000..965ccc8
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/insert-upsert-delete/closed/with-additional-atomic-index/with-additional-atomic-index.6.update.sqlpp
@@ -0,0 +1,306 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+USE TestYelp;
+
+UPSERT INTO YelpCheckin (
+    [
+        {
+            "checkin_id": 1,
+            "business_id": "--1UhMGODdWsrMastO9DZw",
+            "dates": [
+                "2016-04-26 19:49:16",
+                "2016-08-30 18:36:57",
+                "2016-10-15 02:45:18",
+                "2016-11-18 01:54:50",
+                "2017-04-20 18:39:06",
+                "2017-05-03 17:58:02",
+                "2019-03-19 22:04:48"
+            ]
+        },
+        {
+            "checkin_id": 2,
+            "business_id": "--EF5N7P70J_UYBTPypYlA",
+            "dates": [
+                "2018-05-25 19:52:07",
+                "2018-09-18 16:09:44",
+                "2019-10-18 21:29:09"
+            ]
+        },
+        {
+            "checkin_id": 3,
+            "business_id": "--Ni3oJ4VOqfOEu7Sj2Vzg",
+            "dates": [
+                "2019-06-07 17:54:58"
+            ]
+        },
+        {
+            "checkin_id": 4,
+            "business_id": "--Y1Adl1YUWfYIRSd8vkmA",
+            "dates": [
+                "2011-05-03 20:54:05",
+                "2011-08-23 20:49:45",
+                "2014-12-04 06:13:01",
+                "2016-11-16 19:25:55"
+            ]
+        },
+        {
+            "checkin_id": 5,
+            "business_id": "--YPwqIlRJrhHkJcjY3eiA",
+            "dates": [
+                "2016-06-18 21:35:45",
+                "2016-10-15 18:17:51"
+            ]
+        },
+        {
+            "checkin_id": 6,
+            "business_id": "--e8PjCNhEz32pprnPhCwQ",
+            "dates": [
+                "2015-04-02 21:45:17"
+            ]
+        },
+        {
+            "checkin_id": 7,
+            "business_id": "--kinfHwmtdjz03g8B8z8Q",
+            "dates": [
+                "2014-08-27 17:49:18",
+                "2015-12-19 21:30:31",
+                "2018-11-27 15:53:50"
+            ]
+        },
+        {
+            "checkin_id": 8,
+            "business_id": "--q6datkI-f0EoVheXNEeQ",
+            "dates": [
+                "2014-01-28 20:56:04",
+                "2014-11-16 16:11:58",
+                "2015-11-15 19:21:53",
+                "2015-11-15 19:33:39"
+            ]
+        },
+        {
+            "checkin_id": 9,
+            "business_id": "--qvQS4MigHPykD2GV0-zw",
+            "dates": [
+                "2019-04-11 18:30:12"
+            ]
+        },
+        {
+            "checkin_id": 10,
+            "business_id": "--wIGbLEhlpl_UeAIyDmZQ",
+            "dates": [
+                "2015-06-06 20:01:06",
+                "2019-03-14 22:01:52"
+            ]
+        },
+        {
+            "checkin_id": 11,
+            "business_id": "-0FA-Qdi3SPYIoJz9UQw-A",
+            "dates": [
+                "2018-09-29 18:55:17",
+                "2018-10-20 16:48:05",
+                "2018-10-20 22:20:24"
+            ]
+        },
+        {
+            "checkin_id": 12,
+            "business_id": "-0Hj1hb_XW6ybWq2M7QhGA",
+            "dates": [
+                "2011-04-23 21:11:22",
+                "2014-05-04 19:42:48",
+                "2014-05-11 19:16:08",
+                "2014-06-04 19:14:18",
+                "2015-12-05 19:22:42",
+                "2017-05-15 23:19:00"
+            ]
+        },
+        {
+            "checkin_id": 13,
+            "business_id": "-0KMvRFwDWdVBeTpT11iHw",
+            "dates": [
+                "2012-07-13 21:43:57",
+                "2016-12-24 02:27:31",
+                "2017-08-31 00:35:26"
+            ]
+        },
+        {
+            "checkin_id": 14,
+            "business_id": "-0LPtgJC31FWMrMv317p0Q",
+            "dates": [
+                "2013-04-13 12:35:33",
+                "2013-08-19 23:35:49",
+                "2013-10-04 19:14:56"
+            ]
+        },
+        {
+            "checkin_id": 15,
+            "business_id": "-0M3o2uWBnQZwd3hmfEwuw",
+            "dates": [
+                "2016-09-10 19:26:19",
+                "2018-09-08 14:15:37",
+                "2019-09-13 22:47:25"
+            ]
+        },
+        {
+            "checkin_id": 16,
+            "business_id": "-0RRiWDtfnS16AKCtfvBZg",
+            "dates": [
+                "2017-05-19 14:30:16",
+                "2017-05-19 14:30:25",
+                "2017-08-28 15:49:37",
+                "2017-09-20 20:19:51",
+                "2017-10-01 16:31:05",
+                "2017-10-01 16:56:27",
+                "2017-12-27 23:33:20"
+            ]
+        },
+        {
+            "checkin_id": 17,
+            "business_id": "-0Soj75v-XoRcf2ERr8Bmg",
+            "dates": [
+                "2019-06-05 18:22:49"
+            ]
+        },
+        {
+            "checkin_id": 18,
+            "business_id": "-0ZumLlFjMh4ZW1z2nXGug",
+            "dates": [
+                "2011-09-24 21:37:32",
+                "2014-03-10 20:20:07",
+                "2015-05-27 00:40:24",
+                "2015-08-29 17:58:15",
+                "2018-03-16 15:03:26"
+            ]
+        },
+        {
+            "checkin_id": 19,
+            "business_id": "-0aOudcaAyac0VJbMX-L1g",
+            "dates": [
+                "2015-03-16 23:51:16",
+                "2015-12-21 04:48:01",
+                "2016-10-28 20:22:42",
+                "2016-10-28 20:23:00"
+            ]
+        },
+        {
+            "checkin_id": 20,
+            "business_id": "-0b86isaXMY0v4g-V8GZ9Q",
+            "dates": [
+                "2013-10-22 16:49:21",
+                "2014-11-21 17:39:24"
+            ]
+        },
+        {
+            "checkin_id": 21,
+            "business_id": "-0d-BfFSU0bwLcnMaGRxYw",
+            "dates": [
+                "2014-08-07 18:30:48",
+                "2014-09-16 20:41:45",
+                "2014-10-12 23:22:27",
+                "2015-07-21 20:43:56",
+                "2015-07-21 20:45:07"
+            ]
+        },
+        {
+            "checkin_id": 22,
+            "business_id": "-0jz6c3C6i7RG7Ag22K-Pg",
+            "dates": [
+                "2015-05-02 19:49:05",
+                "2015-05-06 03:52:18",
+                "2015-09-26 01:13:19"
+            ]
+        },
+        {
+            "checkin_id": 23,
+            "business_id": "-0y3MZU2oYP8r1ruDP1bfQ",
+            "dates": [
+                "2015-04-11 13:14:14",
+                "2015-11-21 16:05:56",
+                "2016-05-06 14:10:04",
+                "2017-08-09 15:15:10",
+                "2017-10-21 15:12:56"
+            ]
+        },
+        {
+            "checkin_id": 24,
+            "business_id": "-1BPe8UjF2_l3nVk-DFUjA",
+            "dates": [
+                "2015-12-03 18:44:00",
+                "2016-03-17 18:19:21",
+                "2016-11-02 15:58:38"
+            ]
+        },
+        {
+            "checkin_id": 25,
+            "business_id": "-1E2CQu_38mkghvmZgCCRw",
+            "dates": [
+                "2019-04-04 22:02:37"
+            ]
+        },
+        {
+            "checkin_id": 26,
+            "business_id": "-1wzk43IZ5D9Ysu6kzb5xA",
+            "dates": [
+                "2019-02-27 14:03:08"
+            ]
+        },
+        {
+            "checkin_id": 27,
+            "business_id": "-23R9P2eG7VTc6DVLjFKzA",
+            "dates": [
+                "2011-12-21 19:02:51",
+                "2012-04-15 04:21:39",
+                "2012-04-15 14:23:56",
+                "2013-06-30 22:39:51",
+                "2013-10-04 20:34:13",
+                "2014-07-16 02:28:40"
+            ]
+        },
+        {
+            "checkin_id": 28,
+            "business_id": "-26MGfikhJiTfCI-GqmzhQ",
+            "dates": [
+                "2018-06-13 20:16:07"
+            ]
+        },
+        {
+            "checkin_id": 29,
+            "business_id": "-2bLuJsMZ0WhI9daurVQNQ",
+            "dates": [
+                "2015-05-29 16:46:17",
+                "2015-06-01 15:03:53"
+            ]
+        },
+        {
+            "checkin_id": 30,
+            "business_id": "-2hDBMaza_ldqnZdiU06LQ",
+            "dates": [
+                "2011-10-08 12:02:23",
+                "2014-08-18 02:11:11",
+                "2016-01-07 05:27:51",
+                "2016-10-21 20:15:55",
+                "2016-12-01 03:57:10",
+                "2016-12-29 01:54:42",
+                "2018-07-22 19:55:31",
+                "2018-09-07 01:42:54",
+                "2019-03-08 03:41:06"
+            ]
+        }
+    ]
+);
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/insert-upsert-delete/closed/with-additional-atomic-index/with-additional-atomic-index.7.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/insert-upsert-delete/closed/with-additional-atomic-index/with-additional-atomic-index.7.query.sqlpp
new file mode 100644
index 0000000..69fe7cc
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/insert-upsert-delete/closed/with-additional-atomic-index/with-additional-atomic-index.7.query.sqlpp
@@ -0,0 +1,25 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+SET `compiler.arrayindex` "true";
+
+USE TestYelp;
+
+SELECT COUNT(*)
+FROM YelpCheckin C, C.dates D
+WHERE D > " ";
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/insert-upsert-delete/closed/with-composite-sk/with-composite-sk.1.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/insert-upsert-delete/closed/with-composite-sk/with-composite-sk.1.ddl.sqlpp
new file mode 100644
index 0000000..6ff0c88
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/insert-upsert-delete/closed/with-composite-sk/with-composite-sk.1.ddl.sqlpp
@@ -0,0 +1,41 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+/*
+ * Description: Verify the following DML actions for a composite array->atomic index:
+ *     1) Insert into an empty index (non bulk-load operation).
+ *     2) Delete all-but-one entry from the index.
+ *     3) Upsert all entries into the index.
+ */
+
+DROP DATAVERSE TestYelp IF EXISTS;
+CREATE DATAVERSE TestYelp;
+USE TestYelp;
+
+CREATE TYPE CheckinType AS {
+    checkin_id: int,
+    business_id: string,
+    checkin_times: [{
+        date: string,
+        time: string
+    }]
+};
+
+CREATE DATASET YelpCheckin(CheckinType) PRIMARY KEY checkin_id;
+CREATE INDEX IdxYelpCheckinDates ON YelpCheckin (UNNEST checkin_times SELECT date, time);
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/insert-upsert-delete/closed/with-composite-sk/with-composite-sk.2.update.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/insert-upsert-delete/closed/with-composite-sk/with-composite-sk.2.update.sqlpp
new file mode 100644
index 0000000..2f7a9fd
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/insert-upsert-delete/closed/with-composite-sk/with-composite-sk.2.update.sqlpp
@@ -0,0 +1,306 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+USE TestYelp;
+
+INSERT INTO YelpCheckin (
+    [
+        {
+            "checkin_id": 1,
+            "business_id": "--1UhMGODdWsrMastO9DZw",
+            "checkin_times": [ 
+                {"date": "2016-04-26", "time": "19:49:16"},
+                {"date": "2016-08-30", "time": "18:36:57"},
+                {"date": "2016-10-15", "time": "02:45:18"},
+                {"date": "2016-11-18", "time": "01:54:50"},
+                {"date": "2017-04-20", "time": "18:39:06"},
+                {"date": "2017-05-03", "time": "17:58:02"},
+                {"date": "2019-03-19", "time": "22:04:48"}
+             ]
+        },
+        {
+            "checkin_id": 2,
+            "business_id": "--EF5N7P70J_UYBTPypYlA",
+            "checkin_times": [ 
+                {"date": "2018-05-25", "time": "19:52:07"},
+                {"date": "2018-09-18", "time": "16:09:44"},
+                {"date": "2019-10-18", "time": "21:29:09"}
+             ]
+        },
+        {
+            "checkin_id": 3,
+            "business_id": "--Ni3oJ4VOqfOEu7Sj2Vzg",
+            "checkin_times": [ 
+                {"date": "2019-06-07", "time": "17:54:58"}
+             ]
+        },
+        {
+            "checkin_id": 4,
+            "business_id": "--Y1Adl1YUWfYIRSd8vkmA",
+            "checkin_times": [ 
+                {"date": "2011-05-03", "time": "20:54:05"},
+                {"date": "2011-08-23", "time": "20:49:45"},
+                {"date": "2014-12-04", "time": "06:13:01"},
+                {"date": "2016-11-16", "time": "19:25:55"}
+             ]
+        },
+        {
+            "checkin_id": 5,
+            "business_id": "--YPwqIlRJrhHkJcjY3eiA",
+            "checkin_times": [ 
+                {"date": "2016-06-18", "time": "21:35:45"},
+                {"date": "2016-10-15", "time": "18:17:51"}
+             ]
+        },
+        {
+            "checkin_id": 6,
+            "business_id": "--e8PjCNhEz32pprnPhCwQ",
+            "checkin_times": [ 
+                {"date": "2015-04-02", "time": "21:45:17"}
+             ]
+        },
+        {
+            "checkin_id": 7,
+            "business_id": "--kinfHwmtdjz03g8B8z8Q",
+            "checkin_times": [ 
+                {"date": "2014-08-27", "time": "17:49:18"},
+                {"date": "2015-12-19", "time": "21:30:31"},
+                {"date": "2018-11-27", "time": "15:53:50"}
+             ]
+        },
+        {
+            "checkin_id": 8,
+            "business_id": "--q6datkI-f0EoVheXNEeQ",
+            "checkin_times": [ 
+                {"date": "2014-01-28", "time": "20:56:04"},
+                {"date": "2014-11-16", "time": "16:11:58"},
+                {"date": "2015-11-15", "time": "19:21:53"},
+                {"date": "2015-11-15", "time": "19:33:39"}
+             ]
+        },
+        {
+            "checkin_id": 9,
+            "business_id": "--qvQS4MigHPykD2GV0-zw",
+            "checkin_times": [ 
+                {"date": "2019-04-11", "time": "18:30:12"}
+             ]
+        },
+        {
+            "checkin_id": 10,
+            "business_id": "--wIGbLEhlpl_UeAIyDmZQ",
+            "checkin_times": [ 
+                {"date": "2015-06-06", "time": "20:01:06"},
+                {"date": "2019-03-14", "time": "22:01:52"}
+             ]
+        },
+        {
+            "checkin_id": 11,
+            "business_id": "-0FA-Qdi3SPYIoJz9UQw-A",
+            "checkin_times": [ 
+                {"date": "2018-09-29", "time": "18:55:17"},
+                {"date": "2018-10-20", "time": "16:48:05"},
+                {"date": "2018-10-20", "time": "22:20:24"}
+             ]
+        },
+        {
+            "checkin_id": 12,
+            "business_id": "-0Hj1hb_XW6ybWq2M7QhGA",
+            "checkin_times": [ 
+                {"date": "2011-04-23", "time": "21:11:22"},
+                {"date": "2014-05-04", "time": "19:42:48"},
+                {"date": "2014-05-11", "time": "19:16:08"},
+                {"date": "2014-06-04", "time": "19:14:18"},
+                {"date": "2015-12-05", "time": "19:22:42"},
+                {"date": "2017-05-15", "time": "23:19:00"}
+             ]
+        },
+        {
+            "checkin_id": 13,
+            "business_id": "-0KMvRFwDWdVBeTpT11iHw",
+            "checkin_times": [ 
+                {"date": "2012-07-13", "time": "21:43:57"},
+                {"date": "2016-12-24", "time": "02:27:31"},
+                {"date": "2017-08-31", "time": "00:35:26"}
+             ]
+        },
+        {
+            "checkin_id": 14,
+            "business_id": "-0LPtgJC31FWMrMv317p0Q",
+            "checkin_times": [ 
+                {"date": "2013-04-13", "time": "12:35:33"},
+                {"date": "2013-08-19", "time": "23:35:49"},
+                {"date": "2013-10-04", "time": "19:14:56"}
+             ]
+        },
+        {
+            "checkin_id": 15,
+            "business_id": "-0M3o2uWBnQZwd3hmfEwuw",
+            "checkin_times": [ 
+                {"date": "2016-09-10", "time": "19:26:19"},
+                {"date": "2018-09-08", "time": "14:15:37"},
+                {"date": "2019-09-13", "time": "22:47:25"}
+             ]
+        },
+        {
+            "checkin_id": 16,
+            "business_id": "-0RRiWDtfnS16AKCtfvBZg",
+            "checkin_times": [ 
+                {"date": "2017-05-19", "time": "14:30:16"},
+                {"date": "2017-05-19", "time": "14:30:25"},
+                {"date": "2017-08-28", "time": "15:49:37"},
+                {"date": "2017-09-20", "time": "20:19:51"},
+                {"date": "2017-10-01", "time": "16:31:05"},
+                {"date": "2017-10-01", "time": "16:56:27"},
+                {"date": "2017-12-27", "time": "23:33:20"}
+             ]
+        },
+        {
+            "checkin_id": 17,
+            "business_id": "-0Soj75v-XoRcf2ERr8Bmg",
+            "checkin_times": [ 
+                {"date": "2019-06-05", "time": "18:22:49"}
+             ]
+        },
+        {
+            "checkin_id": 18,
+            "business_id": "-0ZumLlFjMh4ZW1z2nXGug",
+            "checkin_times": [ 
+                {"date": "2011-09-24", "time": "21:37:32"},
+                {"date": "2014-03-10", "time": "20:20:07"},
+                {"date": "2015-05-27", "time": "00:40:24"},
+                {"date": "2015-08-29", "time": "17:58:15"},
+                {"date": "2018-03-16", "time": "15:03:26"}
+             ]
+        },
+        {
+            "checkin_id": 19,
+            "business_id": "-0aOudcaAyac0VJbMX-L1g",
+            "checkin_times": [ 
+                {"date": "2015-03-16", "time": "23:51:16"},
+                {"date": "2015-12-21", "time": "04:48:01"},
+                {"date": "2016-10-28", "time": "20:22:42"},
+                {"date": "2016-10-28", "time": "20:23:00"}
+             ]
+        },
+        {
+            "checkin_id": 20,
+            "business_id": "-0b86isaXMY0v4g-V8GZ9Q",
+            "checkin_times": [ 
+                {"date": "2013-10-22", "time": "16:49:21"},
+                {"date": "2014-11-21", "time": "17:39:24"}
+             ]
+        },
+        {
+            "checkin_id": 21,
+            "business_id": "-0d-BfFSU0bwLcnMaGRxYw",
+            "checkin_times": [ 
+                {"date": "2014-08-07", "time": "18:30:48"},
+                {"date": "2014-09-16", "time": "20:41:45"},
+                {"date": "2014-10-12", "time": "23:22:27"},
+                {"date": "2015-07-21", "time": "20:43:56"},
+                {"date": "2015-07-21", "time": "20:45:07"}
+             ]
+        },
+        {
+            "checkin_id": 22,
+            "business_id": "-0jz6c3C6i7RG7Ag22K-Pg",
+            "checkin_times": [ 
+                {"date": "2015-05-02", "time": "19:49:05"},
+                {"date": "2015-05-06", "time": "03:52:18"},
+                {"date": "2015-09-26", "time": "01:13:19"}
+             ]
+        },
+        {
+            "checkin_id": 23,
+            "business_id": "-0y3MZU2oYP8r1ruDP1bfQ",
+            "checkin_times": [ 
+                {"date": "2015-04-11", "time": "13:14:14"},
+                {"date": "2015-11-21", "time": "16:05:56"},
+                {"date": "2016-05-06", "time": "14:10:04"},
+                {"date": "2017-08-09", "time": "15:15:10"},
+                {"date": "2017-10-21", "time": "15:12:56"}
+             ]
+        },
+        {
+            "checkin_id": 24,
+            "business_id": "-1BPe8UjF2_l3nVk-DFUjA",
+            "checkin_times": [ 
+                {"date": "2015-12-03", "time": "18:44:00"},
+                {"date": "2016-03-17", "time": "18:19:21"},
+                {"date": "2016-11-02", "time": "15:58:38"}
+             ]
+        },
+        {
+            "checkin_id": 25,
+            "business_id": "-1E2CQu_38mkghvmZgCCRw",
+            "checkin_times": [ 
+                {"date": "2019-04-04", "time": "22:02:37"}
+             ]
+        },
+        {
+            "checkin_id": 26,
+            "business_id": "-1wzk43IZ5D9Ysu6kzb5xA",
+            "checkin_times": [ 
+                {"date": "2019-02-27", "time": "14:03:08"}
+             ]
+        },
+        {
+            "checkin_id": 27,
+            "business_id": "-23R9P2eG7VTc6DVLjFKzA",
+            "checkin_times": [ 
+                {"date": "2011-12-21", "time": "19:02:51"},
+                {"date": "2012-04-15", "time": "04:21:39"},
+                {"date": "2012-04-15", "time": "14:23:56"},
+                {"date": "2013-06-30", "time": "22:39:51"},
+                {"date": "2013-10-04", "time": "20:34:13"},
+                {"date": "2014-07-16", "time": "02:28:40"}
+             ]
+        },
+        {
+            "checkin_id": 28,
+            "business_id": "-26MGfikhJiTfCI-GqmzhQ",
+            "checkin_times": [ 
+                {"date": "2018-06-13", "time": "20:16:07"}
+             ]
+        },
+        {
+            "checkin_id": 29,
+            "business_id": "-2bLuJsMZ0WhI9daurVQNQ",
+            "checkin_times": [ 
+                {"date": "2015-05-29", "time": "16:46:17"},
+                {"date": "2015-06-01", "time": "15:03:53"}
+             ]
+        },
+        {
+            "checkin_id": 30,
+            "business_id": "-2hDBMaza_ldqnZdiU06LQ",
+            "checkin_times": [ 
+                {"date": "2011-10-08", "time": "12:02:23"},
+                {"date": "2014-08-18", "time": "02:11:11"},
+                {"date": "2016-01-07", "time": "05:27:51"},
+                {"date": "2016-10-21", "time": "20:15:55"},
+                {"date": "2016-12-01", "time": "03:57:10"},
+                {"date": "2016-12-29", "time": "01:54:42"},
+                {"date": "2018-07-22", "time": "19:55:31"},
+                {"date": "2018-09-07", "time": "01:42:54"},
+                {"date": "2019-03-08", "time": "03:41:06"}
+             ]
+        }
+    ]
+);
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/insert-upsert-delete/closed/with-composite-sk/with-composite-sk.3.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/insert-upsert-delete/closed/with-composite-sk/with-composite-sk.3.query.sqlpp
new file mode 100644
index 0000000..baf548c
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/insert-upsert-delete/closed/with-composite-sk/with-composite-sk.3.query.sqlpp
@@ -0,0 +1,27 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+SET `compiler.arrayindex` "true";
+
+USE TestYelp;
+
+SELECT COUNT(*)
+FROM YelpCheckin C, C.checkin_times D
+WHERE D.date > " " AND 
+      D.time > " " AND 
+      C.business_id > " ";
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/insert-upsert-delete/closed/with-composite-sk/with-composite-sk.4.update.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/insert-upsert-delete/closed/with-composite-sk/with-composite-sk.4.update.sqlpp
new file mode 100644
index 0000000..ea1dba3
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/insert-upsert-delete/closed/with-composite-sk/with-composite-sk.4.update.sqlpp
@@ -0,0 +1,23 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+USE TestYelp;
+
+DELETE FROM YelpCheckin C
+WHERE C.business_id != "--1UhMGODdWsrMastO9DZw";
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/insert-upsert-delete/closed/with-composite-sk/with-composite-sk.5.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/insert-upsert-delete/closed/with-composite-sk/with-composite-sk.5.query.sqlpp
new file mode 100644
index 0000000..baf548c
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/insert-upsert-delete/closed/with-composite-sk/with-composite-sk.5.query.sqlpp
@@ -0,0 +1,27 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+SET `compiler.arrayindex` "true";
+
+USE TestYelp;
+
+SELECT COUNT(*)
+FROM YelpCheckin C, C.checkin_times D
+WHERE D.date > " " AND 
+      D.time > " " AND 
+      C.business_id > " ";
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/insert-upsert-delete/closed/with-composite-sk/with-composite-sk.6.update.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/insert-upsert-delete/closed/with-composite-sk/with-composite-sk.6.update.sqlpp
new file mode 100644
index 0000000..5ec7879
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/insert-upsert-delete/closed/with-composite-sk/with-composite-sk.6.update.sqlpp
@@ -0,0 +1,306 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+USE TestYelp;
+
+UPSERT INTO YelpCheckin (
+    [
+        {
+            "checkin_id": 1,
+            "business_id": "--1UhMGODdWsrMastO9DZw",
+            "checkin_times": [ 
+                {"date": "2016-04-26", "time": "19:49:16"},
+                {"date": "2016-08-30", "time": "18:36:57"},
+                {"date": "2016-10-15", "time": "02:45:18"},
+                {"date": "2016-11-18", "time": "01:54:50"},
+                {"date": "2017-04-20", "time": "18:39:06"},
+                {"date": "2017-05-03", "time": "17:58:02"},
+                {"date": "2019-03-19", "time": "22:04:48"}
+             ]
+        },
+        {
+            "checkin_id": 2,
+            "business_id": "--EF5N7P70J_UYBTPypYlA",
+            "checkin_times": [ 
+                {"date": "2018-05-25", "time": "19:52:07"},
+                {"date": "2018-09-18", "time": "16:09:44"},
+                {"date": "2019-10-18", "time": "21:29:09"}
+             ]
+        },
+        {
+            "checkin_id": 3,
+            "business_id": "--Ni3oJ4VOqfOEu7Sj2Vzg",
+            "checkin_times": [ 
+                {"date": "2019-06-07", "time": "17:54:58"}
+             ]
+        },
+        {
+            "checkin_id": 4,
+            "business_id": "--Y1Adl1YUWfYIRSd8vkmA",
+            "checkin_times": [ 
+                {"date": "2011-05-03", "time": "20:54:05"},
+                {"date": "2011-08-23", "time": "20:49:45"},
+                {"date": "2014-12-04", "time": "06:13:01"},
+                {"date": "2016-11-16", "time": "19:25:55"}
+             ]
+        },
+        {
+            "checkin_id": 5,
+            "business_id": "--YPwqIlRJrhHkJcjY3eiA",
+            "checkin_times": [ 
+                {"date": "2016-06-18", "time": "21:35:45"},
+                {"date": "2016-10-15", "time": "18:17:51"}
+             ]
+        },
+        {
+            "checkin_id": 6,
+            "business_id": "--e8PjCNhEz32pprnPhCwQ",
+            "checkin_times": [ 
+                {"date": "2015-04-02", "time": "21:45:17"}
+             ]
+        },
+        {
+            "checkin_id": 7,
+            "business_id": "--kinfHwmtdjz03g8B8z8Q",
+            "checkin_times": [ 
+                {"date": "2014-08-27", "time": "17:49:18"},
+                {"date": "2015-12-19", "time": "21:30:31"},
+                {"date": "2018-11-27", "time": "15:53:50"}
+             ]
+        },
+        {
+            "checkin_id": 8,
+            "business_id": "--q6datkI-f0EoVheXNEeQ",
+            "checkin_times": [ 
+                {"date": "2014-01-28", "time": "20:56:04"},
+                {"date": "2014-11-16", "time": "16:11:58"},
+                {"date": "2015-11-15", "time": "19:21:53"},
+                {"date": "2015-11-15", "time": "19:33:39"}
+             ]
+        },
+        {
+            "checkin_id": 9,
+            "business_id": "--qvQS4MigHPykD2GV0-zw",
+            "checkin_times": [ 
+                {"date": "2019-04-11", "time": "18:30:12"}
+             ]
+        },
+        {
+            "checkin_id": 10,
+            "business_id": "--wIGbLEhlpl_UeAIyDmZQ",
+            "checkin_times": [ 
+                {"date": "2015-06-06", "time": "20:01:06"},
+                {"date": "2019-03-14", "time": "22:01:52"}
+             ]
+        },
+        {
+            "checkin_id": 11,
+            "business_id": "-0FA-Qdi3SPYIoJz9UQw-A",
+            "checkin_times": [ 
+                {"date": "2018-09-29", "time": "18:55:17"},
+                {"date": "2018-10-20", "time": "16:48:05"},
+                {"date": "2018-10-20", "time": "22:20:24"}
+             ]
+        },
+        {
+            "checkin_id": 12,
+            "business_id": "-0Hj1hb_XW6ybWq2M7QhGA",
+            "checkin_times": [ 
+                {"date": "2011-04-23", "time": "21:11:22"},
+                {"date": "2014-05-04", "time": "19:42:48"},
+                {"date": "2014-05-11", "time": "19:16:08"},
+                {"date": "2014-06-04", "time": "19:14:18"},
+                {"date": "2015-12-05", "time": "19:22:42"},
+                {"date": "2017-05-15", "time": "23:19:00"}
+             ]
+        },
+        {
+            "checkin_id": 13,
+            "business_id": "-0KMvRFwDWdVBeTpT11iHw",
+            "checkin_times": [ 
+                {"date": "2012-07-13", "time": "21:43:57"},
+                {"date": "2016-12-24", "time": "02:27:31"},
+                {"date": "2017-08-31", "time": "00:35:26"}
+             ]
+        },
+        {
+            "checkin_id": 14,
+            "business_id": "-0LPtgJC31FWMrMv317p0Q",
+            "checkin_times": [ 
+                {"date": "2013-04-13", "time": "12:35:33"},
+                {"date": "2013-08-19", "time": "23:35:49"},
+                {"date": "2013-10-04", "time": "19:14:56"}
+             ]
+        },
+        {
+            "checkin_id": 15,
+            "business_id": "-0M3o2uWBnQZwd3hmfEwuw",
+            "checkin_times": [ 
+                {"date": "2016-09-10", "time": "19:26:19"},
+                {"date": "2018-09-08", "time": "14:15:37"},
+                {"date": "2019-09-13", "time": "22:47:25"}
+             ]
+        },
+        {
+            "checkin_id": 16,
+            "business_id": "-0RRiWDtfnS16AKCtfvBZg",
+            "checkin_times": [ 
+                {"date": "2017-05-19", "time": "14:30:16"},
+                {"date": "2017-05-19", "time": "14:30:25"},
+                {"date": "2017-08-28", "time": "15:49:37"},
+                {"date": "2017-09-20", "time": "20:19:51"},
+                {"date": "2017-10-01", "time": "16:31:05"},
+                {"date": "2017-10-01", "time": "16:56:27"},
+                {"date": "2017-12-27", "time": "23:33:20"}
+             ]
+        },
+        {
+            "checkin_id": 17,
+            "business_id": "-0Soj75v-XoRcf2ERr8Bmg",
+            "checkin_times": [ 
+                {"date": "2019-06-05", "time": "18:22:49"}
+             ]
+        },
+        {
+            "checkin_id": 18,
+            "business_id": "-0ZumLlFjMh4ZW1z2nXGug",
+            "checkin_times": [ 
+                {"date": "2011-09-24", "time": "21:37:32"},
+                {"date": "2014-03-10", "time": "20:20:07"},
+                {"date": "2015-05-27", "time": "00:40:24"},
+                {"date": "2015-08-29", "time": "17:58:15"},
+                {"date": "2018-03-16", "time": "15:03:26"}
+             ]
+        },
+        {
+            "checkin_id": 19,
+            "business_id": "-0aOudcaAyac0VJbMX-L1g",
+            "checkin_times": [ 
+                {"date": "2015-03-16", "time": "23:51:16"},
+                {"date": "2015-12-21", "time": "04:48:01"},
+                {"date": "2016-10-28", "time": "20:22:42"},
+                {"date": "2016-10-28", "time": "20:23:00"}
+             ]
+        },
+        {
+            "checkin_id": 20,
+            "business_id": "-0b86isaXMY0v4g-V8GZ9Q",
+            "checkin_times": [ 
+                {"date": "2013-10-22", "time": "16:49:21"},
+                {"date": "2014-11-21", "time": "17:39:24"}
+             ]
+        },
+        {
+            "checkin_id": 21,
+            "business_id": "-0d-BfFSU0bwLcnMaGRxYw",
+            "checkin_times": [ 
+                {"date": "2014-08-07", "time": "18:30:48"},
+                {"date": "2014-09-16", "time": "20:41:45"},
+                {"date": "2014-10-12", "time": "23:22:27"},
+                {"date": "2015-07-21", "time": "20:43:56"},
+                {"date": "2015-07-21", "time": "20:45:07"}
+             ]
+        },
+        {
+            "checkin_id": 22,
+            "business_id": "-0jz6c3C6i7RG7Ag22K-Pg",
+            "checkin_times": [ 
+                {"date": "2015-05-02", "time": "19:49:05"},
+                {"date": "2015-05-06", "time": "03:52:18"},
+                {"date": "2015-09-26", "time": "01:13:19"}
+             ]
+        },
+        {
+            "checkin_id": 23,
+            "business_id": "-0y3MZU2oYP8r1ruDP1bfQ",
+            "checkin_times": [ 
+                {"date": "2015-04-11", "time": "13:14:14"},
+                {"date": "2015-11-21", "time": "16:05:56"},
+                {"date": "2016-05-06", "time": "14:10:04"},
+                {"date": "2017-08-09", "time": "15:15:10"},
+                {"date": "2017-10-21", "time": "15:12:56"}
+             ]
+        },
+        {
+            "checkin_id": 24,
+            "business_id": "-1BPe8UjF2_l3nVk-DFUjA",
+            "checkin_times": [ 
+                {"date": "2015-12-03", "time": "18:44:00"},
+                {"date": "2016-03-17", "time": "18:19:21"},
+                {"date": "2016-11-02", "time": "15:58:38"}
+             ]
+        },
+        {
+            "checkin_id": 25,
+            "business_id": "-1E2CQu_38mkghvmZgCCRw",
+            "checkin_times": [ 
+                {"date": "2019-04-04", "time": "22:02:37"}
+             ]
+        },
+        {
+            "checkin_id": 26,
+            "business_id": "-1wzk43IZ5D9Ysu6kzb5xA",
+            "checkin_times": [ 
+                {"date": "2019-02-27", "time": "14:03:08"}
+             ]
+        },
+        {
+            "checkin_id": 27,
+            "business_id": "-23R9P2eG7VTc6DVLjFKzA",
+            "checkin_times": [ 
+                {"date": "2011-12-21", "time": "19:02:51"},
+                {"date": "2012-04-15", "time": "04:21:39"},
+                {"date": "2012-04-15", "time": "14:23:56"},
+                {"date": "2013-06-30", "time": "22:39:51"},
+                {"date": "2013-10-04", "time": "20:34:13"},
+                {"date": "2014-07-16", "time": "02:28:40"}
+             ]
+        },
+        {
+            "checkin_id": 28,
+            "business_id": "-26MGfikhJiTfCI-GqmzhQ",
+            "checkin_times": [ 
+                {"date": "2018-06-13", "time": "20:16:07"}
+             ]
+        },
+        {
+            "checkin_id": 29,
+            "business_id": "-2bLuJsMZ0WhI9daurVQNQ",
+            "checkin_times": [ 
+                {"date": "2015-05-29", "time": "16:46:17"},
+                {"date": "2015-06-01", "time": "15:03:53"}
+             ]
+        },
+        {
+            "checkin_id": 30,
+            "business_id": "-2hDBMaza_ldqnZdiU06LQ",
+            "checkin_times": [ 
+                {"date": "2011-10-08", "time": "12:02:23"},
+                {"date": "2014-08-18", "time": "02:11:11"},
+                {"date": "2016-01-07", "time": "05:27:51"},
+                {"date": "2016-10-21", "time": "20:15:55"},
+                {"date": "2016-12-01", "time": "03:57:10"},
+                {"date": "2016-12-29", "time": "01:54:42"},
+                {"date": "2018-07-22", "time": "19:55:31"},
+                {"date": "2018-09-07", "time": "01:42:54"},
+                {"date": "2019-03-08", "time": "03:41:06"}
+             ]
+        }
+    ]
+);
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/insert-upsert-delete/closed/with-composite-sk/with-composite-sk.7.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/insert-upsert-delete/closed/with-composite-sk/with-composite-sk.7.query.sqlpp
new file mode 100644
index 0000000..baf548c
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/insert-upsert-delete/closed/with-composite-sk/with-composite-sk.7.query.sqlpp
@@ -0,0 +1,27 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+SET `compiler.arrayindex` "true";
+
+USE TestYelp;
+
+SELECT COUNT(*)
+FROM YelpCheckin C, C.checkin_times D
+WHERE D.date > " " AND 
+      D.time > " " AND 
+      C.business_id > " ";
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/insert-upsert-delete/closed/with-filter-fields/with-filter-fields.1.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/insert-upsert-delete/closed/with-filter-fields/with-filter-fields.1.ddl.sqlpp
new file mode 100644
index 0000000..c6739ca
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/insert-upsert-delete/closed/with-filter-fields/with-filter-fields.1.ddl.sqlpp
@@ -0,0 +1,37 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+/*
+ * Description: Verify the following DML actions for an array->atomic index having a dataset filter field:
+ *     1) Insert into an empty index (non bulk-load operation).
+ *     2) Delete all-but-one entry from the index.
+ *     3) Upsert all entries into the index.
+ */
+
+DROP DATAVERSE TestYelp IF EXISTS;
+CREATE DATAVERSE TestYelp;
+USE TestYelp;
+
+CREATE TYPE CheckinType AS {
+    business_id: string,
+    dates: [string]
+};
+
+CREATE DATASET YelpCheckin(CheckinType) PRIMARY KEY business_id WITH FILTER ON business_id;
+CREATE INDEX IdxYelpCheckinDates ON YelpCheckin (UNNEST dates);
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/insert-upsert-delete/closed/with-filter-fields/with-filter-fields.2.update.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/insert-upsert-delete/closed/with-filter-fields/with-filter-fields.2.update.sqlpp
new file mode 100644
index 0000000..b75bf1b
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/insert-upsert-delete/closed/with-filter-fields/with-filter-fields.2.update.sqlpp
@@ -0,0 +1,276 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+USE TestYelp;
+
+INSERT INTO YelpCheckin (
+    [
+        {
+            "business_id": "--1UhMGODdWsrMastO9DZw",
+            "dates": [
+                "2016-04-26 19:49:16",
+                "2016-08-30 18:36:57",
+                "2016-10-15 02:45:18",
+                "2016-11-18 01:54:50",
+                "2017-04-20 18:39:06",
+                "2017-05-03 17:58:02",
+                "2019-03-19 22:04:48"
+            ]
+        },
+        {
+            "business_id": "--EF5N7P70J_UYBTPypYlA",
+            "dates": [
+                "2018-05-25 19:52:07",
+                "2018-09-18 16:09:44",
+                "2019-10-18 21:29:09"
+            ]
+        },
+        {
+            "business_id": "--Ni3oJ4VOqfOEu7Sj2Vzg",
+            "dates": [
+                "2019-06-07 17:54:58"
+            ]
+        },
+        {
+            "business_id": "--Y1Adl1YUWfYIRSd8vkmA",
+            "dates": [
+                "2011-05-03 20:54:05",
+                "2011-08-23 20:49:45",
+                "2014-12-04 06:13:01",
+                "2016-11-16 19:25:55"
+            ]
+        },
+        {
+            "business_id": "--YPwqIlRJrhHkJcjY3eiA",
+            "dates": [
+                "2016-06-18 21:35:45",
+                "2016-10-15 18:17:51"
+            ]
+        },
+        {
+            "business_id": "--e8PjCNhEz32pprnPhCwQ",
+            "dates": [
+                "2015-04-02 21:45:17"
+            ]
+        },
+        {
+            "business_id": "--kinfHwmtdjz03g8B8z8Q",
+            "dates": [
+                "2014-08-27 17:49:18",
+                "2015-12-19 21:30:31",
+                "2018-11-27 15:53:50"
+            ]
+        },
+        {
+            "business_id": "--q6datkI-f0EoVheXNEeQ",
+            "dates": [
+                "2014-01-28 20:56:04",
+                "2014-11-16 16:11:58",
+                "2015-11-15 19:21:53",
+                "2015-11-15 19:33:39"
+            ]
+        },
+        {
+            "business_id": "--qvQS4MigHPykD2GV0-zw",
+            "dates": [
+                "2019-04-11 18:30:12"
+            ]
+        },
+        {
+            "business_id": "--wIGbLEhlpl_UeAIyDmZQ",
+            "dates": [
+                "2015-06-06 20:01:06",
+                "2019-03-14 22:01:52"
+            ]
+        },
+        {
+            "business_id": "-0FA-Qdi3SPYIoJz9UQw-A",
+            "dates": [
+                "2018-09-29 18:55:17",
+                "2018-10-20 16:48:05",
+                "2018-10-20 22:20:24"
+            ]
+        },
+        {
+            "business_id": "-0Hj1hb_XW6ybWq2M7QhGA",
+            "dates": [
+                "2011-04-23 21:11:22",
+                "2014-05-04 19:42:48",
+                "2014-05-11 19:16:08",
+                "2014-06-04 19:14:18",
+                "2015-12-05 19:22:42",
+                "2017-05-15 23:19:00"
+            ]
+        },
+        {
+            "business_id": "-0KMvRFwDWdVBeTpT11iHw",
+            "dates": [
+                "2012-07-13 21:43:57",
+                "2016-12-24 02:27:31",
+                "2017-08-31 00:35:26"
+            ]
+        },
+        {
+            "business_id": "-0LPtgJC31FWMrMv317p0Q",
+            "dates": [
+                "2013-04-13 12:35:33",
+                "2013-08-19 23:35:49",
+                "2013-10-04 19:14:56"
+            ]
+        },
+        {
+            "business_id": "-0M3o2uWBnQZwd3hmfEwuw",
+            "dates": [
+                "2016-09-10 19:26:19",
+                "2018-09-08 14:15:37",
+                "2019-09-13 22:47:25"
+            ]
+        },
+        {
+            "business_id": "-0RRiWDtfnS16AKCtfvBZg",
+            "dates": [
+                "2017-05-19 14:30:16",
+                "2017-05-19 14:30:25",
+                "2017-08-28 15:49:37",
+                "2017-09-20 20:19:51",
+                "2017-10-01 16:31:05",
+                "2017-10-01 16:56:27",
+                "2017-12-27 23:33:20"
+            ]
+        },
+        {
+            "business_id": "-0Soj75v-XoRcf2ERr8Bmg",
+            "dates": [
+                "2019-06-05 18:22:49"
+            ]
+        },
+        {
+            "business_id": "-0ZumLlFjMh4ZW1z2nXGug",
+            "dates": [
+                "2011-09-24 21:37:32",
+                "2014-03-10 20:20:07",
+                "2015-05-27 00:40:24",
+                "2015-08-29 17:58:15",
+                "2018-03-16 15:03:26"
+            ]
+        },
+        {
+            "business_id": "-0aOudcaAyac0VJbMX-L1g",
+            "dates": [
+                "2015-03-16 23:51:16",
+                "2015-12-21 04:48:01",
+                "2016-10-28 20:22:42",
+                "2016-10-28 20:23:00"
+            ]
+        },
+        {
+            "business_id": "-0b86isaXMY0v4g-V8GZ9Q",
+            "dates": [
+                "2013-10-22 16:49:21",
+                "2014-11-21 17:39:24"
+            ]
+        },
+        {
+            "business_id": "-0d-BfFSU0bwLcnMaGRxYw",
+            "dates": [
+                "2014-08-07 18:30:48",
+                "2014-09-16 20:41:45",
+                "2014-10-12 23:22:27",
+                "2015-07-21 20:43:56",
+                "2015-07-21 20:45:07"
+            ]
+        },
+        {
+            "business_id": "-0jz6c3C6i7RG7Ag22K-Pg",
+            "dates": [
+                "2015-05-02 19:49:05",
+                "2015-05-06 03:52:18",
+                "2015-09-26 01:13:19"
+            ]
+        },
+        {
+            "business_id": "-0y3MZU2oYP8r1ruDP1bfQ",
+            "dates": [
+                "2015-04-11 13:14:14",
+                "2015-11-21 16:05:56",
+                "2016-05-06 14:10:04",
+                "2017-08-09 15:15:10",
+                "2017-10-21 15:12:56"
+            ]
+        },
+        {
+            "business_id": "-1BPe8UjF2_l3nVk-DFUjA",
+            "dates": [
+                "2015-12-03 18:44:00",
+                "2016-03-17 18:19:21",
+                "2016-11-02 15:58:38"
+            ]
+        },
+        {
+            "business_id": "-1E2CQu_38mkghvmZgCCRw",
+            "dates": [
+                "2019-04-04 22:02:37"
+            ]
+        },
+        {
+            "business_id": "-1wzk43IZ5D9Ysu6kzb5xA",
+            "dates": [
+                "2019-02-27 14:03:08"
+            ]
+        },
+        {
+            "business_id": "-23R9P2eG7VTc6DVLjFKzA",
+            "dates": [
+                "2011-12-21 19:02:51",
+                "2012-04-15 04:21:39",
+                "2012-04-15 14:23:56",
+                "2013-06-30 22:39:51",
+                "2013-10-04 20:34:13",
+                "2014-07-16 02:28:40"
+            ]
+        },
+        {
+            "business_id": "-26MGfikhJiTfCI-GqmzhQ",
+            "dates": [
+                "2018-06-13 20:16:07"
+            ]
+        },
+        {
+            "business_id": "-2bLuJsMZ0WhI9daurVQNQ",
+            "dates": [
+                "2015-05-29 16:46:17",
+                "2015-06-01 15:03:53"
+            ]
+        },
+        {
+            "business_id": "-2hDBMaza_ldqnZdiU06LQ",
+            "dates": [
+                "2011-10-08 12:02:23",
+                "2014-08-18 02:11:11",
+                "2016-01-07 05:27:51",
+                "2016-10-21 20:15:55",
+                "2016-12-01 03:57:10",
+                "2016-12-29 01:54:42",
+                "2018-07-22 19:55:31",
+                "2018-09-07 01:42:54",
+                "2019-03-08 03:41:06"
+            ]
+        }
+    ]
+);
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/insert-upsert-delete/closed/with-filter-fields/with-filter-fields.3.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/insert-upsert-delete/closed/with-filter-fields/with-filter-fields.3.query.sqlpp
new file mode 100644
index 0000000..69fe7cc
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/insert-upsert-delete/closed/with-filter-fields/with-filter-fields.3.query.sqlpp
@@ -0,0 +1,25 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+SET `compiler.arrayindex` "true";
+
+USE TestYelp;
+
+SELECT COUNT(*)
+FROM YelpCheckin C, C.dates D
+WHERE D > " ";
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/insert-upsert-delete/closed/with-filter-fields/with-filter-fields.4.update.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/insert-upsert-delete/closed/with-filter-fields/with-filter-fields.4.update.sqlpp
new file mode 100644
index 0000000..ea1dba3
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/insert-upsert-delete/closed/with-filter-fields/with-filter-fields.4.update.sqlpp
@@ -0,0 +1,23 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+USE TestYelp;
+
+DELETE FROM YelpCheckin C
+WHERE C.business_id != "--1UhMGODdWsrMastO9DZw";
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/insert-upsert-delete/closed/with-filter-fields/with-filter-fields.5.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/insert-upsert-delete/closed/with-filter-fields/with-filter-fields.5.query.sqlpp
new file mode 100644
index 0000000..69fe7cc
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/insert-upsert-delete/closed/with-filter-fields/with-filter-fields.5.query.sqlpp
@@ -0,0 +1,25 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+SET `compiler.arrayindex` "true";
+
+USE TestYelp;
+
+SELECT COUNT(*)
+FROM YelpCheckin C, C.dates D
+WHERE D > " ";
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/insert-upsert-delete/closed/with-filter-fields/with-filter-fields.6.update.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/insert-upsert-delete/closed/with-filter-fields/with-filter-fields.6.update.sqlpp
new file mode 100644
index 0000000..055fe8c
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/insert-upsert-delete/closed/with-filter-fields/with-filter-fields.6.update.sqlpp
@@ -0,0 +1,276 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+USE TestYelp;
+
+UPSERT INTO YelpCheckin (
+    [
+        {
+            "business_id": "--1UhMGODdWsrMastO9DZw",
+            "dates": [
+                "2016-04-26 19:49:16",
+                "2016-08-30 18:36:57",
+                "2016-10-15 02:45:18",
+                "2016-11-18 01:54:50",
+                "2017-04-20 18:39:06",
+                "2017-05-03 17:58:02",
+                "2019-03-19 22:04:48"
+            ]
+        },
+        {
+            "business_id": "--EF5N7P70J_UYBTPypYlA",
+            "dates": [
+                "2018-05-25 19:52:07",
+                "2018-09-18 16:09:44",
+                "2019-10-18 21:29:09"
+            ]
+        },
+        {
+            "business_id": "--Ni3oJ4VOqfOEu7Sj2Vzg",
+            "dates": [
+                "2019-06-07 17:54:58"
+            ]
+        },
+        {
+            "business_id": "--Y1Adl1YUWfYIRSd8vkmA",
+            "dates": [
+                "2011-05-03 20:54:05",
+                "2011-08-23 20:49:45",
+                "2014-12-04 06:13:01",
+                "2016-11-16 19:25:55"
+            ]
+        },
+        {
+            "business_id": "--YPwqIlRJrhHkJcjY3eiA",
+            "dates": [
+                "2016-06-18 21:35:45",
+                "2016-10-15 18:17:51"
+            ]
+        },
+        {
+            "business_id": "--e8PjCNhEz32pprnPhCwQ",
+            "dates": [
+                "2015-04-02 21:45:17"
+            ]
+        },
+        {
+            "business_id": "--kinfHwmtdjz03g8B8z8Q",
+            "dates": [
+                "2014-08-27 17:49:18",
+                "2015-12-19 21:30:31",
+                "2018-11-27 15:53:50"
+            ]
+        },
+        {
+            "business_id": "--q6datkI-f0EoVheXNEeQ",
+            "dates": [
+                "2014-01-28 20:56:04",
+                "2014-11-16 16:11:58",
+                "2015-11-15 19:21:53",
+                "2015-11-15 19:33:39"
+            ]
+        },
+        {
+            "business_id": "--qvQS4MigHPykD2GV0-zw",
+            "dates": [
+                "2019-04-11 18:30:12"
+            ]
+        },
+        {
+            "business_id": "--wIGbLEhlpl_UeAIyDmZQ",
+            "dates": [
+                "2015-06-06 20:01:06",
+                "2019-03-14 22:01:52"
+            ]
+        },
+        {
+            "business_id": "-0FA-Qdi3SPYIoJz9UQw-A",
+            "dates": [
+                "2018-09-29 18:55:17",
+                "2018-10-20 16:48:05",
+                "2018-10-20 22:20:24"
+            ]
+        },
+        {
+            "business_id": "-0Hj1hb_XW6ybWq2M7QhGA",
+            "dates": [
+                "2011-04-23 21:11:22",
+                "2014-05-04 19:42:48",
+                "2014-05-11 19:16:08",
+                "2014-06-04 19:14:18",
+                "2015-12-05 19:22:42",
+                "2017-05-15 23:19:00"
+            ]
+        },
+        {
+            "business_id": "-0KMvRFwDWdVBeTpT11iHw",
+            "dates": [
+                "2012-07-13 21:43:57",
+                "2016-12-24 02:27:31",
+                "2017-08-31 00:35:26"
+            ]
+        },
+        {
+            "business_id": "-0LPtgJC31FWMrMv317p0Q",
+            "dates": [
+                "2013-04-13 12:35:33",
+                "2013-08-19 23:35:49",
+                "2013-10-04 19:14:56"
+            ]
+        },
+        {
+            "business_id": "-0M3o2uWBnQZwd3hmfEwuw",
+            "dates": [
+                "2016-09-10 19:26:19",
+                "2018-09-08 14:15:37",
+                "2019-09-13 22:47:25"
+            ]
+        },
+        {
+            "business_id": "-0RRiWDtfnS16AKCtfvBZg",
+            "dates": [
+                "2017-05-19 14:30:16",
+                "2017-05-19 14:30:25",
+                "2017-08-28 15:49:37",
+                "2017-09-20 20:19:51",
+                "2017-10-01 16:31:05",
+                "2017-10-01 16:56:27",
+                "2017-12-27 23:33:20"
+            ]
+        },
+        {
+            "business_id": "-0Soj75v-XoRcf2ERr8Bmg",
+            "dates": [
+                "2019-06-05 18:22:49"
+            ]
+        },
+        {
+            "business_id": "-0ZumLlFjMh4ZW1z2nXGug",
+            "dates": [
+                "2011-09-24 21:37:32",
+                "2014-03-10 20:20:07",
+                "2015-05-27 00:40:24",
+                "2015-08-29 17:58:15",
+                "2018-03-16 15:03:26"
+            ]
+        },
+        {
+            "business_id": "-0aOudcaAyac0VJbMX-L1g",
+            "dates": [
+                "2015-03-16 23:51:16",
+                "2015-12-21 04:48:01",
+                "2016-10-28 20:22:42",
+                "2016-10-28 20:23:00"
+            ]
+        },
+        {
+            "business_id": "-0b86isaXMY0v4g-V8GZ9Q",
+            "dates": [
+                "2013-10-22 16:49:21",
+                "2014-11-21 17:39:24"
+            ]
+        },
+        {
+            "business_id": "-0d-BfFSU0bwLcnMaGRxYw",
+            "dates": [
+                "2014-08-07 18:30:48",
+                "2014-09-16 20:41:45",
+                "2014-10-12 23:22:27",
+                "2015-07-21 20:43:56",
+                "2015-07-21 20:45:07"
+            ]
+        },
+        {
+            "business_id": "-0jz6c3C6i7RG7Ag22K-Pg",
+            "dates": [
+                "2015-05-02 19:49:05",
+                "2015-05-06 03:52:18",
+                "2015-09-26 01:13:19"
+            ]
+        },
+        {
+            "business_id": "-0y3MZU2oYP8r1ruDP1bfQ",
+            "dates": [
+                "2015-04-11 13:14:14",
+                "2015-11-21 16:05:56",
+                "2016-05-06 14:10:04",
+                "2017-08-09 15:15:10",
+                "2017-10-21 15:12:56"
+            ]
+        },
+        {
+            "business_id": "-1BPe8UjF2_l3nVk-DFUjA",
+            "dates": [
+                "2015-12-03 18:44:00",
+                "2016-03-17 18:19:21",
+                "2016-11-02 15:58:38"
+            ]
+        },
+        {
+            "business_id": "-1E2CQu_38mkghvmZgCCRw",
+            "dates": [
+                "2019-04-04 22:02:37"
+            ]
+        },
+        {
+            "business_id": "-1wzk43IZ5D9Ysu6kzb5xA",
+            "dates": [
+                "2019-02-27 14:03:08"
+            ]
+        },
+        {
+            "business_id": "-23R9P2eG7VTc6DVLjFKzA",
+            "dates": [
+                "2011-12-21 19:02:51",
+                "2012-04-15 04:21:39",
+                "2012-04-15 14:23:56",
+                "2013-06-30 22:39:51",
+                "2013-10-04 20:34:13",
+                "2014-07-16 02:28:40"
+            ]
+        },
+        {
+            "business_id": "-26MGfikhJiTfCI-GqmzhQ",
+            "dates": [
+                "2018-06-13 20:16:07"
+            ]
+        },
+        {
+            "business_id": "-2bLuJsMZ0WhI9daurVQNQ",
+            "dates": [
+                "2015-05-29 16:46:17",
+                "2015-06-01 15:03:53"
+            ]
+        },
+        {
+            "business_id": "-2hDBMaza_ldqnZdiU06LQ",
+            "dates": [
+                "2011-10-08 12:02:23",
+                "2014-08-18 02:11:11",
+                "2016-01-07 05:27:51",
+                "2016-10-21 20:15:55",
+                "2016-12-01 03:57:10",
+                "2016-12-29 01:54:42",
+                "2018-07-22 19:55:31",
+                "2018-09-07 01:42:54",
+                "2019-03-08 03:41:06"
+            ]
+        }
+    ]
+);
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/insert-upsert-delete/closed/with-filter-fields/with-filter-fields.7.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/insert-upsert-delete/closed/with-filter-fields/with-filter-fields.7.query.sqlpp
new file mode 100644
index 0000000..69fe7cc
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/insert-upsert-delete/closed/with-filter-fields/with-filter-fields.7.query.sqlpp
@@ -0,0 +1,25 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+SET `compiler.arrayindex` "true";
+
+USE TestYelp;
+
+SELECT COUNT(*)
+FROM YelpCheckin C, C.dates D
+WHERE D > " ";
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/insert-upsert-delete/open/use-case-1/use-case-1.1.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/insert-upsert-delete/open/use-case-1/use-case-1.1.ddl.sqlpp
new file mode 100644
index 0000000..0e54946
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/insert-upsert-delete/open/use-case-1/use-case-1.1.ddl.sqlpp
@@ -0,0 +1,36 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+/*
+ * Description: Verify the following DML actions for an array->atomic index:
+ *     1) Insert into an empty index (non bulk-load operation), additionally with two records that have no array-index qualifying entries.
+ *     2) Delete all-but-one entry from the index.
+ *     3) Upsert all *original* (all records have qualifying array-index entries now) entries into the index.
+ */
+
+DROP DATAVERSE TestYelp IF EXISTS;
+CREATE DATAVERSE TestYelp;
+USE TestYelp;
+
+CREATE TYPE CheckinType AS {
+    business_id: string
+};
+
+CREATE DATASET YelpCheckin(CheckinType) PRIMARY KEY business_id;
+CREATE INDEX IdxYelpCheckinDates ON YelpCheckin (UNNEST dates : string ?) ;
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/insert-upsert-delete/open/use-case-1/use-case-1.2.update.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/insert-upsert-delete/open/use-case-1/use-case-1.2.update.sqlpp
new file mode 100644
index 0000000..ba9a31b
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/insert-upsert-delete/open/use-case-1/use-case-1.2.update.sqlpp
@@ -0,0 +1,272 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+USE TestYelp;
+
+INSERT INTO YelpCheckin (
+    [
+        {
+            "business_id": "--1UhMGODdWsrMastO9DZw",
+            "dates": [
+                "2016-04-26 19:49:16",
+                "2016-08-30 18:36:57",
+                "2016-10-15 02:45:18",
+                "2016-11-18 01:54:50",
+                "2017-04-20 18:39:06",
+                "2017-05-03 17:58:02",
+                "2019-03-19 22:04:48"
+            ]
+        },
+        {
+            "business_id": "--EF5N7P70J_UYBTPypYlA",
+            "dates": [
+                "2018-05-25 19:52:07",
+                "2018-09-18 16:09:44",
+                "2019-10-18 21:29:09"
+            ]
+        },
+        {
+            "business_id": "--Ni3oJ4VOqfOEu7Sj2Vzg",
+            "dates": [
+                "2019-06-07 17:54:58"
+            ]
+        },
+        {
+            "business_id": "--Y1Adl1YUWfYIRSd8vkmA",
+            "dates": [
+                "2011-05-03 20:54:05",
+                "2011-08-23 20:49:45",
+                "2014-12-04 06:13:01",
+                "2016-11-16 19:25:55"
+            ]
+        },
+        {
+            "business_id": "--YPwqIlRJrhHkJcjY3eiA",
+            "dates": [
+                "2016-06-18 21:35:45",
+                "2016-10-15 18:17:51"
+            ]
+        },
+        {
+            "business_id": "--e8PjCNhEz32pprnPhCwQ",
+            "dates": [
+                "2015-04-02 21:45:17"
+            ]
+        },
+        {
+            "business_id": "--kinfHwmtdjz03g8B8z8Q",
+            "dates": [
+                "2014-08-27 17:49:18",
+                "2015-12-19 21:30:31",
+                "2018-11-27 15:53:50"
+            ]
+        },
+        {
+            "business_id": "--q6datkI-f0EoVheXNEeQ",
+            "dates": [
+                "2014-01-28 20:56:04",
+                "2014-11-16 16:11:58",
+                "2015-11-15 19:21:53",
+                "2015-11-15 19:33:39"
+            ]
+        },
+        {
+            "business_id": "--qvQS4MigHPykD2GV0-zw",
+            "dates": [
+                "2019-04-11 18:30:12"
+            ]
+        },
+        {
+            "business_id": "--wIGbLEhlpl_UeAIyDmZQ",
+            "dates": [
+                "2015-06-06 20:01:06",
+                "2019-03-14 22:01:52"
+            ]
+        },
+        {
+            "business_id": "-0FA-Qdi3SPYIoJz9UQw-A",
+            "dates": [
+                "2018-09-29 18:55:17",
+                "2018-10-20 16:48:05",
+                "2018-10-20 22:20:24"
+            ]
+        },
+        {
+            "business_id": "-0Hj1hb_XW6ybWq2M7QhGA",
+            "dates": [
+                "2011-04-23 21:11:22",
+                "2014-05-04 19:42:48",
+                "2014-05-11 19:16:08",
+                "2014-06-04 19:14:18",
+                "2015-12-05 19:22:42",
+                "2017-05-15 23:19:00"
+            ]
+        },
+        {
+            "business_id": "-0KMvRFwDWdVBeTpT11iHw",
+            "dates": [
+                "2012-07-13 21:43:57",
+                "2016-12-24 02:27:31",
+                "2017-08-31 00:35:26"
+            ]
+        },
+        {
+            "business_id": "-0LPtgJC31FWMrMv317p0Q",
+            "dates": [
+                "2013-04-13 12:35:33",
+                "2013-08-19 23:35:49",
+                "2013-10-04 19:14:56"
+            ]
+        },
+        {
+            "business_id": "-0M3o2uWBnQZwd3hmfEwuw",
+            "dates": [
+                "2016-09-10 19:26:19",
+                "2018-09-08 14:15:37",
+                "2019-09-13 22:47:25"
+            ]
+        },
+        {
+            "business_id": "-0RRiWDtfnS16AKCtfvBZg",
+            "dates": [
+                "2017-05-19 14:30:16",
+                "2017-05-19 14:30:25",
+                "2017-08-28 15:49:37",
+                "2017-09-20 20:19:51",
+                "2017-10-01 16:31:05",
+                "2017-10-01 16:56:27",
+                "2017-12-27 23:33:20"
+            ]
+        },
+        {
+            "business_id": "-0Soj75v-XoRcf2ERr8Bmg",
+            "dates": [
+                "2019-06-05 18:22:49"
+            ]
+        },
+        {
+            "business_id": "-0ZumLlFjMh4ZW1z2nXGug",
+            "dates": [
+                "2011-09-24 21:37:32",
+                "2014-03-10 20:20:07",
+                "2015-05-27 00:40:24",
+                "2015-08-29 17:58:15",
+                "2018-03-16 15:03:26"
+            ]
+        },
+        {
+            "business_id": "-0aOudcaAyac0VJbMX-L1g",
+            "dates": [
+                "2015-03-16 23:51:16",
+                "2015-12-21 04:48:01",
+                "2016-10-28 20:22:42",
+                "2016-10-28 20:23:00"
+            ]
+        },
+        {
+            "business_id": "-0b86isaXMY0v4g-V8GZ9Q",
+            "dates": [
+                "2013-10-22 16:49:21",
+                "2014-11-21 17:39:24"
+            ]
+        },
+        {
+            "business_id": "-0d-BfFSU0bwLcnMaGRxYw",
+            "dates": [
+                "2014-08-07 18:30:48",
+                "2014-09-16 20:41:45",
+                "2014-10-12 23:22:27",
+                "2015-07-21 20:43:56",
+                "2015-07-21 20:45:07"
+            ]
+        },
+        {
+            "business_id": "-0jz6c3C6i7RG7Ag22K-Pg",
+            "dates": [
+                "2015-05-02 19:49:05",
+                "2015-05-06 03:52:18",
+                "2015-09-26 01:13:19"
+            ]
+        },
+        {
+            "business_id": "-0y3MZU2oYP8r1ruDP1bfQ",
+            "dates": [
+                "2015-04-11 13:14:14",
+                "2015-11-21 16:05:56",
+                "2016-05-06 14:10:04",
+                "2017-08-09 15:15:10",
+                "2017-10-21 15:12:56"
+            ]
+        },
+        {
+            "business_id": "-1BPe8UjF2_l3nVk-DFUjA",
+            "dates": [
+                "2015-12-03 18:44:00",
+                "2016-03-17 18:19:21",
+                "2016-11-02 15:58:38"
+            ]
+        },
+        {
+            "business_id": "-1E2CQu_38mkghvmZgCCRw",
+            "dates": []
+        },
+        {
+            "business_id": "-1wzk43IZ5D9Ysu6kzb5xA",
+            "dates": []
+        },
+        {
+            "business_id": "-23R9P2eG7VTc6DVLjFKzA",
+            "dates": [
+                "2011-12-21 19:02:51",
+                "2012-04-15 04:21:39",
+                "2012-04-15 14:23:56",
+                "2013-06-30 22:39:51",
+                "2013-10-04 20:34:13",
+                "2014-07-16 02:28:40"
+            ]
+        },
+        {
+            "business_id": "-26MGfikhJiTfCI-GqmzhQ",
+            "dates": [
+                "2018-06-13 20:16:07"
+            ]
+        },
+        {
+            "business_id": "-2bLuJsMZ0WhI9daurVQNQ",
+            "dates": [
+                "2015-05-29 16:46:17",
+                "2015-06-01 15:03:53"
+            ]
+        },
+        {
+            "business_id": "-2hDBMaza_ldqnZdiU06LQ",
+            "dates": [
+                "2011-10-08 12:02:23",
+                "2014-08-18 02:11:11",
+                "2016-01-07 05:27:51",
+                "2016-10-21 20:15:55",
+                "2016-12-01 03:57:10",
+                "2016-12-29 01:54:42",
+                "2018-07-22 19:55:31",
+                "2018-09-07 01:42:54",
+                "2019-03-08 03:41:06"
+            ]
+        }
+    ]
+);
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/insert-upsert-delete/open/use-case-1/use-case-1.3.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/insert-upsert-delete/open/use-case-1/use-case-1.3.query.sqlpp
new file mode 100644
index 0000000..69fe7cc
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/insert-upsert-delete/open/use-case-1/use-case-1.3.query.sqlpp
@@ -0,0 +1,25 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+SET `compiler.arrayindex` "true";
+
+USE TestYelp;
+
+SELECT COUNT(*)
+FROM YelpCheckin C, C.dates D
+WHERE D > " ";
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/insert-upsert-delete/open/use-case-1/use-case-1.4.update.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/insert-upsert-delete/open/use-case-1/use-case-1.4.update.sqlpp
new file mode 100644
index 0000000..ea1dba3
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/insert-upsert-delete/open/use-case-1/use-case-1.4.update.sqlpp
@@ -0,0 +1,23 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+USE TestYelp;
+
+DELETE FROM YelpCheckin C
+WHERE C.business_id != "--1UhMGODdWsrMastO9DZw";
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/insert-upsert-delete/open/use-case-1/use-case-1.5.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/insert-upsert-delete/open/use-case-1/use-case-1.5.query.sqlpp
new file mode 100644
index 0000000..69fe7cc
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/insert-upsert-delete/open/use-case-1/use-case-1.5.query.sqlpp
@@ -0,0 +1,25 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+SET `compiler.arrayindex` "true";
+
+USE TestYelp;
+
+SELECT COUNT(*)
+FROM YelpCheckin C, C.dates D
+WHERE D > " ";
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/insert-upsert-delete/open/use-case-1/use-case-1.6.update.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/insert-upsert-delete/open/use-case-1/use-case-1.6.update.sqlpp
new file mode 100644
index 0000000..055fe8c
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/insert-upsert-delete/open/use-case-1/use-case-1.6.update.sqlpp
@@ -0,0 +1,276 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+USE TestYelp;
+
+UPSERT INTO YelpCheckin (
+    [
+        {
+            "business_id": "--1UhMGODdWsrMastO9DZw",
+            "dates": [
+                "2016-04-26 19:49:16",
+                "2016-08-30 18:36:57",
+                "2016-10-15 02:45:18",
+                "2016-11-18 01:54:50",
+                "2017-04-20 18:39:06",
+                "2017-05-03 17:58:02",
+                "2019-03-19 22:04:48"
+            ]
+        },
+        {
+            "business_id": "--EF5N7P70J_UYBTPypYlA",
+            "dates": [
+                "2018-05-25 19:52:07",
+                "2018-09-18 16:09:44",
+                "2019-10-18 21:29:09"
+            ]
+        },
+        {
+            "business_id": "--Ni3oJ4VOqfOEu7Sj2Vzg",
+            "dates": [
+                "2019-06-07 17:54:58"
+            ]
+        },
+        {
+            "business_id": "--Y1Adl1YUWfYIRSd8vkmA",
+            "dates": [
+                "2011-05-03 20:54:05",
+                "2011-08-23 20:49:45",
+                "2014-12-04 06:13:01",
+                "2016-11-16 19:25:55"
+            ]
+        },
+        {
+            "business_id": "--YPwqIlRJrhHkJcjY3eiA",
+            "dates": [
+                "2016-06-18 21:35:45",
+                "2016-10-15 18:17:51"
+            ]
+        },
+        {
+            "business_id": "--e8PjCNhEz32pprnPhCwQ",
+            "dates": [
+                "2015-04-02 21:45:17"
+            ]
+        },
+        {
+            "business_id": "--kinfHwmtdjz03g8B8z8Q",
+            "dates": [
+                "2014-08-27 17:49:18",
+                "2015-12-19 21:30:31",
+                "2018-11-27 15:53:50"
+            ]
+        },
+        {
+            "business_id": "--q6datkI-f0EoVheXNEeQ",
+            "dates": [
+                "2014-01-28 20:56:04",
+                "2014-11-16 16:11:58",
+                "2015-11-15 19:21:53",
+                "2015-11-15 19:33:39"
+            ]
+        },
+        {
+            "business_id": "--qvQS4MigHPykD2GV0-zw",
+            "dates": [
+                "2019-04-11 18:30:12"
+            ]
+        },
+        {
+            "business_id": "--wIGbLEhlpl_UeAIyDmZQ",
+            "dates": [
+                "2015-06-06 20:01:06",
+                "2019-03-14 22:01:52"
+            ]
+        },
+        {
+            "business_id": "-0FA-Qdi3SPYIoJz9UQw-A",
+            "dates": [
+                "2018-09-29 18:55:17",
+                "2018-10-20 16:48:05",
+                "2018-10-20 22:20:24"
+            ]
+        },
+        {
+            "business_id": "-0Hj1hb_XW6ybWq2M7QhGA",
+            "dates": [
+                "2011-04-23 21:11:22",
+                "2014-05-04 19:42:48",
+                "2014-05-11 19:16:08",
+                "2014-06-04 19:14:18",
+                "2015-12-05 19:22:42",
+                "2017-05-15 23:19:00"
+            ]
+        },
+        {
+            "business_id": "-0KMvRFwDWdVBeTpT11iHw",
+            "dates": [
+                "2012-07-13 21:43:57",
+                "2016-12-24 02:27:31",
+                "2017-08-31 00:35:26"
+            ]
+        },
+        {
+            "business_id": "-0LPtgJC31FWMrMv317p0Q",
+            "dates": [
+                "2013-04-13 12:35:33",
+                "2013-08-19 23:35:49",
+                "2013-10-04 19:14:56"
+            ]
+        },
+        {
+            "business_id": "-0M3o2uWBnQZwd3hmfEwuw",
+            "dates": [
+                "2016-09-10 19:26:19",
+                "2018-09-08 14:15:37",
+                "2019-09-13 22:47:25"
+            ]
+        },
+        {
+            "business_id": "-0RRiWDtfnS16AKCtfvBZg",
+            "dates": [
+                "2017-05-19 14:30:16",
+                "2017-05-19 14:30:25",
+                "2017-08-28 15:49:37",
+                "2017-09-20 20:19:51",
+                "2017-10-01 16:31:05",
+                "2017-10-01 16:56:27",
+                "2017-12-27 23:33:20"
+            ]
+        },
+        {
+            "business_id": "-0Soj75v-XoRcf2ERr8Bmg",
+            "dates": [
+                "2019-06-05 18:22:49"
+            ]
+        },
+        {
+            "business_id": "-0ZumLlFjMh4ZW1z2nXGug",
+            "dates": [
+                "2011-09-24 21:37:32",
+                "2014-03-10 20:20:07",
+                "2015-05-27 00:40:24",
+                "2015-08-29 17:58:15",
+                "2018-03-16 15:03:26"
+            ]
+        },
+        {
+            "business_id": "-0aOudcaAyac0VJbMX-L1g",
+            "dates": [
+                "2015-03-16 23:51:16",
+                "2015-12-21 04:48:01",
+                "2016-10-28 20:22:42",
+                "2016-10-28 20:23:00"
+            ]
+        },
+        {
+            "business_id": "-0b86isaXMY0v4g-V8GZ9Q",
+            "dates": [
+                "2013-10-22 16:49:21",
+                "2014-11-21 17:39:24"
+            ]
+        },
+        {
+            "business_id": "-0d-BfFSU0bwLcnMaGRxYw",
+            "dates": [
+                "2014-08-07 18:30:48",
+                "2014-09-16 20:41:45",
+                "2014-10-12 23:22:27",
+                "2015-07-21 20:43:56",
+                "2015-07-21 20:45:07"
+            ]
+        },
+        {
+            "business_id": "-0jz6c3C6i7RG7Ag22K-Pg",
+            "dates": [
+                "2015-05-02 19:49:05",
+                "2015-05-06 03:52:18",
+                "2015-09-26 01:13:19"
+            ]
+        },
+        {
+            "business_id": "-0y3MZU2oYP8r1ruDP1bfQ",
+            "dates": [
+                "2015-04-11 13:14:14",
+                "2015-11-21 16:05:56",
+                "2016-05-06 14:10:04",
+                "2017-08-09 15:15:10",
+                "2017-10-21 15:12:56"
+            ]
+        },
+        {
+            "business_id": "-1BPe8UjF2_l3nVk-DFUjA",
+            "dates": [
+                "2015-12-03 18:44:00",
+                "2016-03-17 18:19:21",
+                "2016-11-02 15:58:38"
+            ]
+        },
+        {
+            "business_id": "-1E2CQu_38mkghvmZgCCRw",
+            "dates": [
+                "2019-04-04 22:02:37"
+            ]
+        },
+        {
+            "business_id": "-1wzk43IZ5D9Ysu6kzb5xA",
+            "dates": [
+                "2019-02-27 14:03:08"
+            ]
+        },
+        {
+            "business_id": "-23R9P2eG7VTc6DVLjFKzA",
+            "dates": [
+                "2011-12-21 19:02:51",
+                "2012-04-15 04:21:39",
+                "2012-04-15 14:23:56",
+                "2013-06-30 22:39:51",
+                "2013-10-04 20:34:13",
+                "2014-07-16 02:28:40"
+            ]
+        },
+        {
+            "business_id": "-26MGfikhJiTfCI-GqmzhQ",
+            "dates": [
+                "2018-06-13 20:16:07"
+            ]
+        },
+        {
+            "business_id": "-2bLuJsMZ0WhI9daurVQNQ",
+            "dates": [
+                "2015-05-29 16:46:17",
+                "2015-06-01 15:03:53"
+            ]
+        },
+        {
+            "business_id": "-2hDBMaza_ldqnZdiU06LQ",
+            "dates": [
+                "2011-10-08 12:02:23",
+                "2014-08-18 02:11:11",
+                "2016-01-07 05:27:51",
+                "2016-10-21 20:15:55",
+                "2016-12-01 03:57:10",
+                "2016-12-29 01:54:42",
+                "2018-07-22 19:55:31",
+                "2018-09-07 01:42:54",
+                "2019-03-08 03:41:06"
+            ]
+        }
+    ]
+);
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/insert-upsert-delete/open/use-case-1/use-case-1.7.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/insert-upsert-delete/open/use-case-1/use-case-1.7.query.sqlpp
new file mode 100644
index 0000000..69fe7cc
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/insert-upsert-delete/open/use-case-1/use-case-1.7.query.sqlpp
@@ -0,0 +1,25 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+SET `compiler.arrayindex` "true";
+
+USE TestYelp;
+
+SELECT COUNT(*)
+FROM YelpCheckin C, C.dates D
+WHERE D > " ";
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/insert-upsert-delete/open/use-case-2/use-case-2.1.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/insert-upsert-delete/open/use-case-2/use-case-2.1.ddl.sqlpp
new file mode 100644
index 0000000..9b0d495
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/insert-upsert-delete/open/use-case-2/use-case-2.1.ddl.sqlpp
@@ -0,0 +1,37 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+/*
+ * Description: Verify the following DML actions for an array->atomic index:
+ *     1) Insert into an empty index (non bulk-load operation).
+ *     2) Delete all-but-one entry from the index.
+ *     3) Upsert all entries into the index.
+ */
+
+DROP DATAVERSE TestYelp IF EXISTS;
+CREATE DATAVERSE TestYelp;
+USE TestYelp;
+
+CREATE TYPE CheckinType AS {
+    business_id: string
+};
+
+CREATE DATASET YelpCheckin(CheckinType) PRIMARY KEY business_id;
+CREATE INDEX IdxYelpCheckinDates ON YelpCheckin (UNNEST checkin_times.dates : string ?) ;
+CREATE INDEX IdxYelpCheckinTimes ON YelpCheckin (UNNEST checkin_times.times : string ?) ;
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/insert-upsert-delete/open/use-case-2/use-case-2.2.update.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/insert-upsert-delete/open/use-case-2/use-case-2.2.update.sqlpp
new file mode 100644
index 0000000..2549e8f
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/insert-upsert-delete/open/use-case-2/use-case-2.2.update.sqlpp
@@ -0,0 +1,497 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+USE TestYelp;
+
+INSERT INTO YelpCheckin (
+    [
+        {
+            "business_id": "--1UhMGODdWsrMastO9DZw",
+            "checkin_times": {
+                "dates": [
+                    "2016-04-26",
+                    "2016-08-30",
+                    "2016-10-15",
+                    "2016-11-18",
+                    "2017-04-20",
+                    "2017-05-03",
+                    "2019-03-19"
+                ],
+                "times": [
+                    "19:49:16",
+                    "18:36:57",
+                    "02:45:18",
+                    "01:54:50",
+                    "18:39:06",
+                    "17:58:02",
+                    "22:04:48"
+                ]
+            }
+        },
+        {
+            "business_id": "--EF5N7P70J_UYBTPypYlA",
+            "checkin_times": {
+                "dates": [
+                    "2018-05-25",
+                    "2018-09-18",
+                    "2019-10-18"
+                ],
+                "times": [
+                    "19:52:07",
+                    "16:09:44",
+                    "21:29:09"
+                ]
+            }
+        },
+        {
+            "business_id": "--Ni3oJ4VOqfOEu7Sj2Vzg",
+            "checkin_times": {
+                "dates": [
+                    "2019-06-07"
+                ],
+                "times": [
+                    "17:54:58"
+                ]
+            }
+        },
+        {
+            "business_id": "--Y1Adl1YUWfYIRSd8vkmA",
+            "checkin_times": {
+                "dates": [
+                    "2011-05-03",
+                    "2011-08-23",
+                    "2014-12-04",
+                    "2016-11-16"
+                ],
+                "times": [
+                    "20:54:05",
+                    "20:49:45",
+                    "06:13:01",
+                    "19:25:55"
+                ]
+            }
+        },
+        {
+            "business_id": "--YPwqIlRJrhHkJcjY3eiA",
+            "checkin_times": {
+                "dates": [
+                    "2016-06-18",
+                    "2016-10-15"
+                ],
+                "times": [
+                    "21:35:45",
+                    "18:17:51"
+                ]
+            }
+        },
+        {
+            "business_id": "--e8PjCNhEz32pprnPhCwQ",
+            "checkin_times": {
+                "dates": [
+                    "2015-04-02"
+                ],
+                "times": [
+                    "21:45:17"
+                ]
+            }
+        },
+        {
+            "business_id": "--kinfHwmtdjz03g8B8z8Q",
+            "checkin_times": {
+                "dates": [
+                    "2014-08-27",
+                    "2015-12-19",
+                    "2018-11-27"
+                ],
+                "times": [
+                    "17:49:18",
+                    "21:30:31",
+                    "15:53:50"
+                ]
+            }
+        },
+        {
+            "business_id": "--q6datkI-f0EoVheXNEeQ",
+            "checkin_times": {
+                "dates": [
+                    "2014-01-28",
+                    "2014-11-16",
+                    "2015-11-15",
+                    "2015-11-15"
+                ],
+                "times": [
+                    "20:56:04",
+                    "16:11:58",
+                    "19:21:53",
+                    "19:33:39"
+                ]
+            }
+        },
+        {
+            "business_id": "--qvQS4MigHPykD2GV0-zw",
+            "checkin_times": {
+                "dates": [
+                    "2019-04-11"
+                ],
+                "times": [
+                    "18:30:12"
+                ]
+            }
+        },
+        {
+            "business_id": "--wIGbLEhlpl_UeAIyDmZQ",
+            "checkin_times": {
+                "dates": [
+                    "2015-06-06",
+                    "2019-03-14"
+                ],
+                "times": [
+                    "20:01:06",
+                    "22:01:52"
+                ]
+            }
+        },
+        {
+            "business_id": "-0FA-Qdi3SPYIoJz9UQw-A",
+            "checkin_times": {
+                "dates": [
+                    "2018-09-29",
+                    "2018-10-20",
+                    "2018-10-20"
+                ],
+                "times": [
+                    "18:55:17",
+                    "16:48:05",
+                    "22:20:24"
+                ]
+            }
+        },
+        {
+            "business_id": "-0Hj1hb_XW6ybWq2M7QhGA",
+            "checkin_times": {
+                "dates": [
+                    "2011-04-23",
+                    "2014-05-04",
+                    "2014-05-11",
+                    "2014-06-04",
+                    "2015-12-05",
+                    "2017-05-15"
+                ],
+                "times": [
+                    "21:11:22",
+                    "19:42:48",
+                    "19:16:08",
+                    "19:14:18",
+                    "19:22:42",
+                    "23:19:00"
+                ]
+            }
+        },
+        {
+            "business_id": "-0KMvRFwDWdVBeTpT11iHw",
+            "checkin_times": {
+                "dates": [
+                    "2012-07-13",
+                    "2016-12-24",
+                    "2017-08-31"
+                ],
+                "times": [
+                    "21:43:57",
+                    "02:27:31",
+                    "00:35:26"
+                ]
+            }
+        },
+        {
+            "business_id": "-0LPtgJC31FWMrMv317p0Q",
+            "checkin_times": {
+                "dates": [
+                    "2013-04-13",
+                    "2013-08-19",
+                    "2013-10-04"
+                ],
+                "times": [
+                    "12:35:33",
+                    "23:35:49",
+                    "19:14:56"
+                ]
+            }
+        },
+        {
+            "business_id": "-0M3o2uWBnQZwd3hmfEwuw",
+            "checkin_times": {
+                "dates": [
+                    "2016-09-10",
+                    "2018-09-08",
+                    "2019-09-13"
+                ],
+                "times": [
+                    "19:26:19",
+                    "14:15:37",
+                    "22:47:25"
+                ]
+            }
+        },
+        {
+            "business_id": "-0RRiWDtfnS16AKCtfvBZg",
+            "checkin_times": {
+                "dates": [
+                    "2017-05-19",
+                    "2017-05-19",
+                    "2017-08-28",
+                    "2017-09-20",
+                    "2017-10-01",
+                    "2017-10-01",
+                    "2017-12-27"
+                ],
+                "times": [
+                    "14:30:16",
+                    "14:30:25",
+                    "15:49:37",
+                    "20:19:51",
+                    "16:31:05",
+                    "16:56:27",
+                    "23:33:20"
+                ]
+            }
+        },
+        {
+            "business_id": "-0Soj75v-XoRcf2ERr8Bmg",
+            "checkin_times": {
+                "dates": [
+                    "2019-06-05"
+                ],
+                "times": [
+                    "18:22:49"
+                ]
+            }
+        },
+        {
+            "business_id": "-0ZumLlFjMh4ZW1z2nXGug",
+            "checkin_times": {
+                "dates": [
+                    "2011-09-24",
+                    "2014-03-10",
+                    "2015-05-27",
+                    "2015-08-29",
+                    "2018-03-16"
+                ],
+                "times": [
+                    "21:37:32",
+                    "20:20:07",
+                    "00:40:24",
+                    "17:58:15",
+                    "15:03:26"
+                ]
+            }
+        },
+        {
+            "business_id": "-0aOudcaAyac0VJbMX-L1g",
+            "checkin_times": {
+                "dates": [
+                    "2015-03-16",
+                    "2015-12-21",
+                    "2016-10-28",
+                    "2016-10-28"
+                ],
+                "times": [
+                    "23:51:16",
+                    "04:48:01",
+                    "20:22:42",
+                    "20:23:00"
+                ]
+            }
+        },
+        {
+            "business_id": "-0b86isaXMY0v4g-V8GZ9Q",
+            "checkin_times": {
+                "dates": [
+                    "2013-10-22",
+                    "2014-11-21"
+                ],
+                "times": [
+                    "16:49:21",
+                    "17:39:24"
+                ]
+            }
+        },
+        {
+            "business_id": "-0d-BfFSU0bwLcnMaGRxYw",
+            "checkin_times": {
+                "dates": [
+                    "2014-08-07",
+                    "2014-09-16",
+                    "2014-10-12",
+                    "2015-07-21",
+                    "2015-07-21"
+                ],
+                "times": [
+                    "18:30:48",
+                    "20:41:45",
+                    "23:22:27",
+                    "20:43:56",
+                    "20:45:07"
+                ]
+            }
+        },
+        {
+            "business_id": "-0jz6c3C6i7RG7Ag22K-Pg",
+            "checkin_times": {
+                "dates": [
+                    "2015-05-02",
+                    "2015-05-06",
+                    "2015-09-26"
+                ],
+                "times": [
+                    "19:49:05",
+                    "03:52:18",
+                    "01:13:19"
+                ]
+            }
+        },
+        {
+            "business_id": "-0y3MZU2oYP8r1ruDP1bfQ",
+            "checkin_times": {
+                "dates": [
+                    "2015-04-11",
+                    "2015-11-21",
+                    "2016-05-06",
+                    "2017-08-09",
+                    "2017-10-21"
+                ],
+                "times": [
+                    "13:14:14",
+                    "16:05:56",
+                    "14:10:04",
+                    "15:15:10",
+                    "15:12:56"
+                ]
+            }
+        },
+        {
+            "business_id": "-1BPe8UjF2_l3nVk-DFUjA",
+            "checkin_times": {
+                "dates": [
+                    "2015-12-03",
+                    "2016-03-17",
+                    "2016-11-02"
+                ],
+                "times": [
+                    "18:44:00",
+                    "18:19:21",
+                    "15:58:38"
+                ]
+            }
+        },
+        {
+            "business_id": "-1E2CQu_38mkghvmZgCCRw",
+            "checkin_times": {
+                "dates": [
+                    "2019-04-04"
+                ],
+                "times": [
+                    "22:02:37"
+                ]
+            }
+        },
+        {
+            "business_id": "-1wzk43IZ5D9Ysu6kzb5xA",
+            "checkin_times": {
+                "dates": [
+                    "2019-02-27"
+                ],
+                "times": [
+                    "14:03:08"
+                ]
+            }
+        },
+        {
+            "business_id": "-23R9P2eG7VTc6DVLjFKzA",
+            "checkin_times": {
+                "dates": [
+                    "2011-12-21",
+                    "2012-04-15",
+                    "2012-04-15",
+                    "2013-06-30",
+                    "2013-10-04",
+                    "2014-07-16"
+                ],
+                "times": [
+                    "19:02:51",
+                    "04:21:39",
+                    "14:23:56",
+                    "22:39:51",
+                    "20:34:13",
+                    "02:28:40"
+                ]
+            }
+        },
+        {
+            "business_id": "-26MGfikhJiTfCI-GqmzhQ",
+            "checkin_times": {
+                "dates": [
+                    "2018-06-13"
+                ],
+                "times": [
+                    "20:16:07"
+                ]
+            }
+        },
+        {
+            "business_id": "-2bLuJsMZ0WhI9daurVQNQ",
+            "checkin_times": {
+                "dates": [
+                    "2015-05-29",
+                    "2015-06-01"
+                ],
+                "times": [
+                    "16:46:17",
+                    "15:03:53"
+                ]
+            }
+        },
+        {
+            "business_id": "-2hDBMaza_ldqnZdiU06LQ",
+            "checkin_times": {
+                "dates": [
+                    "2011-10-08",
+                    "2014-08-18",
+                    "2016-01-07",
+                    "2016-10-21",
+                    "2016-12-01",
+                    "2016-12-29",
+                    "2018-07-22",
+                    "2018-09-07",
+                    "2019-03-08"
+                ],
+                "times": [
+                    "12:02:23",
+                    "02:11:11",
+                    "05:27:51",
+                    "20:15:55",
+                    "03:57:10",
+                    "01:54:42",
+                    "19:55:31",
+                    "01:42:54",
+                    "03:41:06"
+                ]
+            }
+        }
+    ]
+);
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/insert-upsert-delete/open/use-case-2/use-case-2.3.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/insert-upsert-delete/open/use-case-2/use-case-2.3.query.sqlpp
new file mode 100644
index 0000000..a440550
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/insert-upsert-delete/open/use-case-2/use-case-2.3.query.sqlpp
@@ -0,0 +1,25 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+SET `compiler.arrayindex` "true";
+
+USE TestYelp;
+
+SELECT COUNT(*)
+FROM YelpCheckin C, C.checkin_times.dates D
+WHERE D > " ";
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/insert-upsert-delete/open/use-case-2/use-case-2.4.update.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/insert-upsert-delete/open/use-case-2/use-case-2.4.update.sqlpp
new file mode 100644
index 0000000..ea1dba3
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/insert-upsert-delete/open/use-case-2/use-case-2.4.update.sqlpp
@@ -0,0 +1,23 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+USE TestYelp;
+
+DELETE FROM YelpCheckin C
+WHERE C.business_id != "--1UhMGODdWsrMastO9DZw";
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/insert-upsert-delete/open/use-case-2/use-case-2.5.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/insert-upsert-delete/open/use-case-2/use-case-2.5.query.sqlpp
new file mode 100644
index 0000000..a440550
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/insert-upsert-delete/open/use-case-2/use-case-2.5.query.sqlpp
@@ -0,0 +1,25 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+SET `compiler.arrayindex` "true";
+
+USE TestYelp;
+
+SELECT COUNT(*)
+FROM YelpCheckin C, C.checkin_times.dates D
+WHERE D > " ";
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/insert-upsert-delete/open/use-case-2/use-case-2.6.update.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/insert-upsert-delete/open/use-case-2/use-case-2.6.update.sqlpp
new file mode 100644
index 0000000..9467e0d
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/insert-upsert-delete/open/use-case-2/use-case-2.6.update.sqlpp
@@ -0,0 +1,497 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+USE TestYelp;
+
+UPSERT INTO YelpCheckin (
+    [
+        {
+            "business_id": "--1UhMGODdWsrMastO9DZw",
+            "checkin_times": {
+                "dates": [
+                    "2016-04-26",
+                    "2016-08-30",
+                    "2016-10-15",
+                    "2016-11-18",
+                    "2017-04-20",
+                    "2017-05-03",
+                    "2019-03-19"
+                ],
+                "times": [
+                    "19:49:16",
+                    "18:36:57",
+                    "02:45:18",
+                    "01:54:50",
+                    "18:39:06",
+                    "17:58:02",
+                    "22:04:48"
+                ]
+            }
+        },
+        {
+            "business_id": "--EF5N7P70J_UYBTPypYlA",
+            "checkin_times": {
+                "dates": [
+                    "2018-05-25",
+                    "2018-09-18",
+                    "2019-10-18"
+                ],
+                "times": [
+                    "19:52:07",
+                    "16:09:44",
+                    "21:29:09"
+                ]
+            }
+        },
+        {
+            "business_id": "--Ni3oJ4VOqfOEu7Sj2Vzg",
+            "checkin_times": {
+                "dates": [
+                    "2019-06-07"
+                ],
+                "times": [
+                    "17:54:58"
+                ]
+            }
+        },
+        {
+            "business_id": "--Y1Adl1YUWfYIRSd8vkmA",
+            "checkin_times": {
+                "dates": [
+                    "2011-05-03",
+                    "2011-08-23",
+                    "2014-12-04",
+                    "2016-11-16"
+                ],
+                "times": [
+                    "20:54:05",
+                    "20:49:45",
+                    "06:13:01",
+                    "19:25:55"
+                ]
+            }
+        },
+        {
+            "business_id": "--YPwqIlRJrhHkJcjY3eiA",
+            "checkin_times": {
+                "dates": [
+                    "2016-06-18",
+                    "2016-10-15"
+                ],
+                "times": [
+                    "21:35:45",
+                    "18:17:51"
+                ]
+            }
+        },
+        {
+            "business_id": "--e8PjCNhEz32pprnPhCwQ",
+            "checkin_times": {
+                "dates": [
+                    "2015-04-02"
+                ],
+                "times": [
+                    "21:45:17"
+                ]
+            }
+        },
+        {
+            "business_id": "--kinfHwmtdjz03g8B8z8Q",
+            "checkin_times": {
+                "dates": [
+                    "2014-08-27",
+                    "2015-12-19",
+                    "2018-11-27"
+                ],
+                "times": [
+                    "17:49:18",
+                    "21:30:31",
+                    "15:53:50"
+                ]
+            }
+        },
+        {
+            "business_id": "--q6datkI-f0EoVheXNEeQ",
+            "checkin_times": {
+                "dates": [
+                    "2014-01-28",
+                    "2014-11-16",
+                    "2015-11-15",
+                    "2015-11-15"
+                ],
+                "times": [
+                    "20:56:04",
+                    "16:11:58",
+                    "19:21:53",
+                    "19:33:39"
+                ]
+            }
+        },
+        {
+            "business_id": "--qvQS4MigHPykD2GV0-zw",
+            "checkin_times": {
+                "dates": [
+                    "2019-04-11"
+                ],
+                "times": [
+                    "18:30:12"
+                ]
+            }
+        },
+        {
+            "business_id": "--wIGbLEhlpl_UeAIyDmZQ",
+            "checkin_times": {
+                "dates": [
+                    "2015-06-06",
+                    "2019-03-14"
+                ],
+                "times": [
+                    "20:01:06",
+                    "22:01:52"
+                ]
+            }
+        },
+        {
+            "business_id": "-0FA-Qdi3SPYIoJz9UQw-A",
+            "checkin_times": {
+                "dates": [
+                    "2018-09-29",
+                    "2018-10-20",
+                    "2018-10-20"
+                ],
+                "times": [
+                    "18:55:17",
+                    "16:48:05",
+                    "22:20:24"
+                ]
+            }
+        },
+        {
+            "business_id": "-0Hj1hb_XW6ybWq2M7QhGA",
+            "checkin_times": {
+                "dates": [
+                    "2011-04-23",
+                    "2014-05-04",
+                    "2014-05-11",
+                    "2014-06-04",
+                    "2015-12-05",
+                    "2017-05-15"
+                ],
+                "times": [
+                    "21:11:22",
+                    "19:42:48",
+                    "19:16:08",
+                    "19:14:18",
+                    "19:22:42",
+                    "23:19:00"
+                ]
+            }
+        },
+        {
+            "business_id": "-0KMvRFwDWdVBeTpT11iHw",
+            "checkin_times": {
+                "dates": [
+                    "2012-07-13",
+                    "2016-12-24",
+                    "2017-08-31"
+                ],
+                "times": [
+                    "21:43:57",
+                    "02:27:31",
+                    "00:35:26"
+                ]
+            }
+        },
+        {
+            "business_id": "-0LPtgJC31FWMrMv317p0Q",
+            "checkin_times": {
+                "dates": [
+                    "2013-04-13",
+                    "2013-08-19",
+                    "2013-10-04"
+                ],
+                "times": [
+                    "12:35:33",
+                    "23:35:49",
+                    "19:14:56"
+                ]
+            }
+        },
+        {
+            "business_id": "-0M3o2uWBnQZwd3hmfEwuw",
+            "checkin_times": {
+                "dates": [
+                    "2016-09-10",
+                    "2018-09-08",
+                    "2019-09-13"
+                ],
+                "times": [
+                    "19:26:19",
+                    "14:15:37",
+                    "22:47:25"
+                ]
+            }
+        },
+        {
+            "business_id": "-0RRiWDtfnS16AKCtfvBZg",
+            "checkin_times": {
+                "dates": [
+                    "2017-05-19",
+                    "2017-05-19",
+                    "2017-08-28",
+                    "2017-09-20",
+                    "2017-10-01",
+                    "2017-10-01",
+                    "2017-12-27"
+                ],
+                "times": [
+                    "14:30:16",
+                    "14:30:25",
+                    "15:49:37",
+                    "20:19:51",
+                    "16:31:05",
+                    "16:56:27",
+                    "23:33:20"
+                ]
+            }
+        },
+        {
+            "business_id": "-0Soj75v-XoRcf2ERr8Bmg",
+            "checkin_times": {
+                "dates": [
+                    "2019-06-05"
+                ],
+                "times": [
+                    "18:22:49"
+                ]
+            }
+        },
+        {
+            "business_id": "-0ZumLlFjMh4ZW1z2nXGug",
+            "checkin_times": {
+                "dates": [
+                    "2011-09-24",
+                    "2014-03-10",
+                    "2015-05-27",
+                    "2015-08-29",
+                    "2018-03-16"
+                ],
+                "times": [
+                    "21:37:32",
+                    "20:20:07",
+                    "00:40:24",
+                    "17:58:15",
+                    "15:03:26"
+                ]
+            }
+        },
+        {
+            "business_id": "-0aOudcaAyac0VJbMX-L1g",
+            "checkin_times": {
+                "dates": [
+                    "2015-03-16",
+                    "2015-12-21",
+                    "2016-10-28",
+                    "2016-10-28"
+                ],
+                "times": [
+                    "23:51:16",
+                    "04:48:01",
+                    "20:22:42",
+                    "20:23:00"
+                ]
+            }
+        },
+        {
+            "business_id": "-0b86isaXMY0v4g-V8GZ9Q",
+            "checkin_times": {
+                "dates": [
+                    "2013-10-22",
+                    "2014-11-21"
+                ],
+                "times": [
+                    "16:49:21",
+                    "17:39:24"
+                ]
+            }
+        },
+        {
+            "business_id": "-0d-BfFSU0bwLcnMaGRxYw",
+            "checkin_times": {
+                "dates": [
+                    "2014-08-07",
+                    "2014-09-16",
+                    "2014-10-12",
+                    "2015-07-21",
+                    "2015-07-21"
+                ],
+                "times": [
+                    "18:30:48",
+                    "20:41:45",
+                    "23:22:27",
+                    "20:43:56",
+                    "20:45:07"
+                ]
+            }
+        },
+        {
+            "business_id": "-0jz6c3C6i7RG7Ag22K-Pg",
+            "checkin_times": {
+                "dates": [
+                    "2015-05-02",
+                    "2015-05-06",
+                    "2015-09-26"
+                ],
+                "times": [
+                    "19:49:05",
+                    "03:52:18",
+                    "01:13:19"
+                ]
+            }
+        },
+        {
+            "business_id": "-0y3MZU2oYP8r1ruDP1bfQ",
+            "checkin_times": {
+                "dates": [
+                    "2015-04-11",
+                    "2015-11-21",
+                    "2016-05-06",
+                    "2017-08-09",
+                    "2017-10-21"
+                ],
+                "times": [
+                    "13:14:14",
+                    "16:05:56",
+                    "14:10:04",
+                    "15:15:10",
+                    "15:12:56"
+                ]
+            }
+        },
+        {
+            "business_id": "-1BPe8UjF2_l3nVk-DFUjA",
+            "checkin_times": {
+                "dates": [
+                    "2015-12-03",
+                    "2016-03-17",
+                    "2016-11-02"
+                ],
+                "times": [
+                    "18:44:00",
+                    "18:19:21",
+                    "15:58:38"
+                ]
+            }
+        },
+        {
+            "business_id": "-1E2CQu_38mkghvmZgCCRw",
+            "checkin_times": {
+                "dates": [
+                    "2019-04-04"
+                ],
+                "times": [
+                    "22:02:37"
+                ]
+            }
+        },
+        {
+            "business_id": "-1wzk43IZ5D9Ysu6kzb5xA",
+            "checkin_times": {
+                "dates": [
+                    "2019-02-27"
+                ],
+                "times": [
+                    "14:03:08"
+                ]
+            }
+        },
+        {
+            "business_id": "-23R9P2eG7VTc6DVLjFKzA",
+            "checkin_times": {
+                "dates": [
+                    "2011-12-21",
+                    "2012-04-15",
+                    "2012-04-15",
+                    "2013-06-30",
+                    "2013-10-04",
+                    "2014-07-16"
+                ],
+                "times": [
+                    "19:02:51",
+                    "04:21:39",
+                    "14:23:56",
+                    "22:39:51",
+                    "20:34:13",
+                    "02:28:40"
+                ]
+            }
+        },
+        {
+            "business_id": "-26MGfikhJiTfCI-GqmzhQ",
+            "checkin_times": {
+                "dates": [
+                    "2018-06-13"
+                ],
+                "times": [
+                    "20:16:07"
+                ]
+            }
+        },
+        {
+            "business_id": "-2bLuJsMZ0WhI9daurVQNQ",
+            "checkin_times": {
+                "dates": [
+                    "2015-05-29",
+                    "2015-06-01"
+                ],
+                "times": [
+                    "16:46:17",
+                    "15:03:53"
+                ]
+            }
+        },
+        {
+            "business_id": "-2hDBMaza_ldqnZdiU06LQ",
+            "checkin_times": {
+                "dates": [
+                    "2011-10-08",
+                    "2014-08-18",
+                    "2016-01-07",
+                    "2016-10-21",
+                    "2016-12-01",
+                    "2016-12-29",
+                    "2018-07-22",
+                    "2018-09-07",
+                    "2019-03-08"
+                ],
+                "times": [
+                    "12:02:23",
+                    "02:11:11",
+                    "05:27:51",
+                    "20:15:55",
+                    "03:57:10",
+                    "01:54:42",
+                    "19:55:31",
+                    "01:42:54",
+                    "03:41:06"
+                ]
+            }
+        }
+    ]
+);
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/insert-upsert-delete/open/use-case-2/use-case-2.7.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/insert-upsert-delete/open/use-case-2/use-case-2.7.query.sqlpp
new file mode 100644
index 0000000..a440550
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/insert-upsert-delete/open/use-case-2/use-case-2.7.query.sqlpp
@@ -0,0 +1,25 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+SET `compiler.arrayindex` "true";
+
+USE TestYelp;
+
+SELECT COUNT(*)
+FROM YelpCheckin C, C.checkin_times.dates D
+WHERE D > " ";
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/insert-upsert-delete/open/use-case-3/use-case-3.1.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/insert-upsert-delete/open/use-case-3/use-case-3.1.ddl.sqlpp
new file mode 100644
index 0000000..1d44ffb
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/insert-upsert-delete/open/use-case-3/use-case-3.1.ddl.sqlpp
@@ -0,0 +1,37 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+/*
+ * Description: Verify the following DML actions for an array->record->atomic index:
+ *     1) Insert into an empty index (non bulk-load operation).
+ *     2) Delete all-but-one entry from the index.
+ *     3) Upsert all entries into the index.
+ */
+
+DROP DATAVERSE TestYelp IF EXISTS;
+CREATE DATAVERSE TestYelp;
+USE TestYelp;
+
+CREATE TYPE CheckinType AS {
+    business_id: string
+};
+
+CREATE DATASET YelpCheckin(CheckinType) PRIMARY KEY business_id;
+CREATE INDEX IdxYelpCheckinDates ON YelpCheckin (UNNEST checkin_times SELECT date : string ?) ;
+CREATE INDEX IdxYelpCheckinTimes ON YelpCheckin (UNNEST checkin_times SELECT time : string ?) ;
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/insert-upsert-delete/open/use-case-3/use-case-3.2.update.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/insert-upsert-delete/open/use-case-3/use-case-3.2.update.sqlpp
new file mode 100644
index 0000000..4fd7c64
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/insert-upsert-delete/open/use-case-3/use-case-3.2.update.sqlpp
@@ -0,0 +1,276 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+USE TestYelp;
+
+INSERT INTO YelpCheckin (
+    [
+        {
+            "business_id": "--1UhMGODdWsrMastO9DZw",
+            "checkin_times": [
+                {"date": "2016-04-26", "time": "19:49:16"},
+                {"date": "2016-08-30", "time": "18:36:57"},
+                {"date": "2016-10-15", "time": "02:45:18"},
+                {"date": "2016-11-18", "time": "01:54:50"},
+                {"date": "2017-04-20", "time": "18:39:06"},
+                {"date": "2017-05-03", "time": "17:58:02"},
+                {"date": "2019-03-19", "time": "22:04:48"}
+             ]
+        },
+        {
+            "business_id": "--EF5N7P70J_UYBTPypYlA",
+            "checkin_times": [
+                {"date": "2018-05-25", "time": "19:52:07"},
+                {"date": "2018-09-18", "time": "16:09:44"},
+                {"date": "2019-10-18", "time": "21:29:09"}
+             ]
+        },
+        {
+            "business_id": "--Ni3oJ4VOqfOEu7Sj2Vzg",
+            "checkin_times": [
+                {"date": "2019-06-07", "time": "17:54:58"}
+             ]
+        },
+        {
+            "business_id": "--Y1Adl1YUWfYIRSd8vkmA",
+            "checkin_times": [
+                {"date": "2011-05-03", "time": "20:54:05"},
+                {"date": "2011-08-23", "time": "20:49:45"},
+                {"date": "2014-12-04", "time": "06:13:01"},
+                {"date": "2016-11-16", "time": "19:25:55"}
+             ]
+        },
+        {
+            "business_id": "--YPwqIlRJrhHkJcjY3eiA",
+            "checkin_times": [
+                {"date": "2016-06-18", "time": "21:35:45"},
+                {"date": "2016-10-15", "time": "18:17:51"}
+             ]
+        },
+        {
+            "business_id": "--e8PjCNhEz32pprnPhCwQ",
+            "checkin_times": [
+                {"date": "2015-04-02", "time": "21:45:17"}
+             ]
+        },
+        {
+            "business_id": "--kinfHwmtdjz03g8B8z8Q",
+            "checkin_times": [
+                {"date": "2014-08-27", "time": "17:49:18"},
+                {"date": "2015-12-19", "time": "21:30:31"},
+                {"date": "2018-11-27", "time": "15:53:50"}
+             ]
+        },
+        {
+            "business_id": "--q6datkI-f0EoVheXNEeQ",
+            "checkin_times": [
+                {"date": "2014-01-28", "time": "20:56:04"},
+                {"date": "2014-11-16", "time": "16:11:58"},
+                {"date": "2015-11-15", "time": "19:21:53"},
+                {"date": "2015-11-15", "time": "19:33:39"}
+             ]
+        },
+        {
+            "business_id": "--qvQS4MigHPykD2GV0-zw",
+            "checkin_times": [
+                {"date": "2019-04-11", "time": "18:30:12"}
+             ]
+        },
+        {
+            "business_id": "--wIGbLEhlpl_UeAIyDmZQ",
+            "checkin_times": [
+                {"date": "2015-06-06", "time": "20:01:06"},
+                {"date": "2019-03-14", "time": "22:01:52"}
+             ]
+        },
+        {
+            "business_id": "-0FA-Qdi3SPYIoJz9UQw-A",
+            "checkin_times": [
+                {"date": "2018-09-29", "time": "18:55:17"},
+                {"date": "2018-10-20", "time": "16:48:05"},
+                {"date": "2018-10-20", "time": "22:20:24"}
+             ]
+        },
+        {
+            "business_id": "-0Hj1hb_XW6ybWq2M7QhGA",
+            "checkin_times": [
+                {"date": "2011-04-23", "time": "21:11:22"},
+                {"date": "2014-05-04", "time": "19:42:48"},
+                {"date": "2014-05-11", "time": "19:16:08"},
+                {"date": "2014-06-04", "time": "19:14:18"},
+                {"date": "2015-12-05", "time": "19:22:42"},
+                {"date": "2017-05-15", "time": "23:19:00"}
+             ]
+        },
+        {
+            "business_id": "-0KMvRFwDWdVBeTpT11iHw",
+            "checkin_times": [
+                {"date": "2012-07-13", "time": "21:43:57"},
+                {"date": "2016-12-24", "time": "02:27:31"},
+                {"date": "2017-08-31", "time": "00:35:26"}
+             ]
+        },
+        {
+            "business_id": "-0LPtgJC31FWMrMv317p0Q",
+            "checkin_times": [
+                {"date": "2013-04-13", "time": "12:35:33"},
+                {"date": "2013-08-19", "time": "23:35:49"},
+                {"date": "2013-10-04", "time": "19:14:56"}
+             ]
+        },
+        {
+            "business_id": "-0M3o2uWBnQZwd3hmfEwuw",
+            "checkin_times": [
+                {"date": "2016-09-10", "time": "19:26:19"},
+                {"date": "2018-09-08", "time": "14:15:37"},
+                {"date": "2019-09-13", "time": "22:47:25"}
+             ]
+        },
+        {
+            "business_id": "-0RRiWDtfnS16AKCtfvBZg",
+            "checkin_times": [
+                {"date": "2017-05-19", "time": "14:30:16"},
+                {"date": "2017-05-19", "time": "14:30:25"},
+                {"date": "2017-08-28", "time": "15:49:37"},
+                {"date": "2017-09-20", "time": "20:19:51"},
+                {"date": "2017-10-01", "time": "16:31:05"},
+                {"date": "2017-10-01", "time": "16:56:27"},
+                {"date": "2017-12-27", "time": "23:33:20"}
+             ]
+        },
+        {
+            "business_id": "-0Soj75v-XoRcf2ERr8Bmg",
+            "checkin_times": [
+                {"date": "2019-06-05", "time": "18:22:49"}
+             ]
+        },
+        {
+            "business_id": "-0ZumLlFjMh4ZW1z2nXGug",
+            "checkin_times": [
+                {"date": "2011-09-24", "time": "21:37:32"},
+                {"date": "2014-03-10", "time": "20:20:07"},
+                {"date": "2015-05-27", "time": "00:40:24"},
+                {"date": "2015-08-29", "time": "17:58:15"},
+                {"date": "2018-03-16", "time": "15:03:26"}
+             ]
+        },
+        {
+            "business_id": "-0aOudcaAyac0VJbMX-L1g",
+            "checkin_times": [
+                {"date": "2015-03-16", "time": "23:51:16"},
+                {"date": "2015-12-21", "time": "04:48:01"},
+                {"date": "2016-10-28", "time": "20:22:42"},
+                {"date": "2016-10-28", "time": "20:23:00"}
+             ]
+        },
+        {
+            "business_id": "-0b86isaXMY0v4g-V8GZ9Q",
+            "checkin_times": [
+                {"date": "2013-10-22", "time": "16:49:21"},
+                {"date": "2014-11-21", "time": "17:39:24"}
+             ]
+        },
+        {
+            "business_id": "-0d-BfFSU0bwLcnMaGRxYw",
+            "checkin_times": [
+                {"date": "2014-08-07", "time": "18:30:48"},
+                {"date": "2014-09-16", "time": "20:41:45"},
+                {"date": "2014-10-12", "time": "23:22:27"},
+                {"date": "2015-07-21", "time": "20:43:56"},
+                {"date": "2015-07-21", "time": "20:45:07"}
+             ]
+        },
+        {
+            "business_id": "-0jz6c3C6i7RG7Ag22K-Pg",
+            "checkin_times": [
+                {"date": "2015-05-02", "time": "19:49:05"},
+                {"date": "2015-05-06", "time": "03:52:18"},
+                {"date": "2015-09-26", "time": "01:13:19"}
+             ]
+        },
+        {
+            "business_id": "-0y3MZU2oYP8r1ruDP1bfQ",
+            "checkin_times": [
+                {"date": "2015-04-11", "time": "13:14:14"},
+                {"date": "2015-11-21", "time": "16:05:56"},
+                {"date": "2016-05-06", "time": "14:10:04"},
+                {"date": "2017-08-09", "time": "15:15:10"},
+                {"date": "2017-10-21", "time": "15:12:56"}
+             ]
+        },
+        {
+            "business_id": "-1BPe8UjF2_l3nVk-DFUjA",
+            "checkin_times": [
+                {"date": "2015-12-03", "time": "18:44:00"},
+                {"date": "2016-03-17", "time": "18:19:21"},
+                {"date": "2016-11-02", "time": "15:58:38"}
+             ]
+        },
+        {
+            "business_id": "-1E2CQu_38mkghvmZgCCRw",
+            "checkin_times": [
+                {"date": "2019-04-04", "time": "22:02:37"}
+             ]
+        },
+        {
+            "business_id": "-1wzk43IZ5D9Ysu6kzb5xA",
+            "checkin_times": [
+                {"date": "2019-02-27", "time": "14:03:08"}
+             ]
+        },
+        {
+            "business_id": "-23R9P2eG7VTc6DVLjFKzA",
+            "checkin_times": [
+                {"date": "2011-12-21", "time": "19:02:51"},
+                {"date": "2012-04-15", "time": "04:21:39"},
+                {"date": "2012-04-15", "time": "14:23:56"},
+                {"date": "2013-06-30", "time": "22:39:51"},
+                {"date": "2013-10-04", "time": "20:34:13"},
+                {"date": "2014-07-16", "time": "02:28:40"}
+             ]
+        },
+        {
+            "business_id": "-26MGfikhJiTfCI-GqmzhQ",
+            "checkin_times": [
+                {"date": "2018-06-13", "time": "20:16:07"}
+             ]
+        },
+        {
+            "business_id": "-2bLuJsMZ0WhI9daurVQNQ",
+            "checkin_times": [
+                {"date": "2015-05-29", "time": "16:46:17"},
+                {"date": "2015-06-01", "time": "15:03:53"}
+             ]
+        },
+        {
+            "business_id": "-2hDBMaza_ldqnZdiU06LQ",
+            "checkin_times": [
+                {"date": "2011-10-08", "time": "12:02:23"},
+                {"date": "2014-08-18", "time": "02:11:11"},
+                {"date": "2016-01-07", "time": "05:27:51"},
+                {"date": "2016-10-21", "time": "20:15:55"},
+                {"date": "2016-12-01", "time": "03:57:10"},
+                {"date": "2016-12-29", "time": "01:54:42"},
+                {"date": "2018-07-22", "time": "19:55:31"},
+                {"date": "2018-09-07", "time": "01:42:54"},
+                {"date": "2019-03-08", "time": "03:41:06"}
+             ]
+        }
+    ]
+);
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/insert-upsert-delete/open/use-case-3/use-case-3.3.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/insert-upsert-delete/open/use-case-3/use-case-3.3.query.sqlpp
new file mode 100644
index 0000000..2757b02
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/insert-upsert-delete/open/use-case-3/use-case-3.3.query.sqlpp
@@ -0,0 +1,25 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+SET `compiler.arrayindex` "true";
+
+USE TestYelp;
+
+SELECT COUNT(*)
+FROM YelpCheckin C, C.checkin_times D
+WHERE D.date > " ";
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/insert-upsert-delete/open/use-case-3/use-case-3.4.update.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/insert-upsert-delete/open/use-case-3/use-case-3.4.update.sqlpp
new file mode 100644
index 0000000..ea1dba3
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/insert-upsert-delete/open/use-case-3/use-case-3.4.update.sqlpp
@@ -0,0 +1,23 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+USE TestYelp;
+
+DELETE FROM YelpCheckin C
+WHERE C.business_id != "--1UhMGODdWsrMastO9DZw";
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/insert-upsert-delete/open/use-case-3/use-case-3.5.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/insert-upsert-delete/open/use-case-3/use-case-3.5.query.sqlpp
new file mode 100644
index 0000000..2757b02
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/insert-upsert-delete/open/use-case-3/use-case-3.5.query.sqlpp
@@ -0,0 +1,25 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+SET `compiler.arrayindex` "true";
+
+USE TestYelp;
+
+SELECT COUNT(*)
+FROM YelpCheckin C, C.checkin_times D
+WHERE D.date > " ";
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/insert-upsert-delete/open/use-case-3/use-case-3.6.update.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/insert-upsert-delete/open/use-case-3/use-case-3.6.update.sqlpp
new file mode 100644
index 0000000..b3f8466
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/insert-upsert-delete/open/use-case-3/use-case-3.6.update.sqlpp
@@ -0,0 +1,276 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+USE TestYelp;
+
+UPSERT INTO YelpCheckin (
+    [
+        {
+            "business_id": "--1UhMGODdWsrMastO9DZw",
+            "checkin_times": [
+                {"date": "2016-04-26", "time": "19:49:16"},
+                {"date": "2016-08-30", "time": "18:36:57"},
+                {"date": "2016-10-15", "time": "02:45:18"},
+                {"date": "2016-11-18", "time": "01:54:50"},
+                {"date": "2017-04-20", "time": "18:39:06"},
+                {"date": "2017-05-03", "time": "17:58:02"},
+                {"date": "2019-03-19", "time": "22:04:48"}
+             ]
+        },
+        {
+            "business_id": "--EF5N7P70J_UYBTPypYlA",
+            "checkin_times": [
+                {"date": "2018-05-25", "time": "19:52:07"},
+                {"date": "2018-09-18", "time": "16:09:44"},
+                {"date": "2019-10-18", "time": "21:29:09"}
+             ]
+        },
+        {
+            "business_id": "--Ni3oJ4VOqfOEu7Sj2Vzg",
+            "checkin_times": [
+                {"date": "2019-06-07", "time": "17:54:58"}
+             ]
+        },
+        {
+            "business_id": "--Y1Adl1YUWfYIRSd8vkmA",
+            "checkin_times": [
+                {"date": "2011-05-03", "time": "20:54:05"},
+                {"date": "2011-08-23", "time": "20:49:45"},
+                {"date": "2014-12-04", "time": "06:13:01"},
+                {"date": "2016-11-16", "time": "19:25:55"}
+             ]
+        },
+        {
+            "business_id": "--YPwqIlRJrhHkJcjY3eiA",
+            "checkin_times": [
+                {"date": "2016-06-18", "time": "21:35:45"},
+                {"date": "2016-10-15", "time": "18:17:51"}
+             ]
+        },
+        {
+            "business_id": "--e8PjCNhEz32pprnPhCwQ",
+            "checkin_times": [
+                {"date": "2015-04-02", "time": "21:45:17"}
+             ]
+        },
+        {
+            "business_id": "--kinfHwmtdjz03g8B8z8Q",
+            "checkin_times": [
+                {"date": "2014-08-27", "time": "17:49:18"},
+                {"date": "2015-12-19", "time": "21:30:31"},
+                {"date": "2018-11-27", "time": "15:53:50"}
+             ]
+        },
+        {
+            "business_id": "--q6datkI-f0EoVheXNEeQ",
+            "checkin_times": [
+                {"date": "2014-01-28", "time": "20:56:04"},
+                {"date": "2014-11-16", "time": "16:11:58"},
+                {"date": "2015-11-15", "time": "19:21:53"},
+                {"date": "2015-11-15", "time": "19:33:39"}
+             ]
+        },
+        {
+            "business_id": "--qvQS4MigHPykD2GV0-zw",
+            "checkin_times": [
+                {"date": "2019-04-11", "time": "18:30:12"}
+             ]
+        },
+        {
+            "business_id": "--wIGbLEhlpl_UeAIyDmZQ",
+            "checkin_times": [
+                {"date": "2015-06-06", "time": "20:01:06"},
+                {"date": "2019-03-14", "time": "22:01:52"}
+             ]
+        },
+        {
+            "business_id": "-0FA-Qdi3SPYIoJz9UQw-A",
+            "checkin_times": [
+                {"date": "2018-09-29", "time": "18:55:17"},
+                {"date": "2018-10-20", "time": "16:48:05"},
+                {"date": "2018-10-20", "time": "22:20:24"}
+             ]
+        },
+        {
+            "business_id": "-0Hj1hb_XW6ybWq2M7QhGA",
+            "checkin_times": [
+                {"date": "2011-04-23", "time": "21:11:22"},
+                {"date": "2014-05-04", "time": "19:42:48"},
+                {"date": "2014-05-11", "time": "19:16:08"},
+                {"date": "2014-06-04", "time": "19:14:18"},
+                {"date": "2015-12-05", "time": "19:22:42"},
+                {"date": "2017-05-15", "time": "23:19:00"}
+             ]
+        },
+        {
+            "business_id": "-0KMvRFwDWdVBeTpT11iHw",
+            "checkin_times": [
+                {"date": "2012-07-13", "time": "21:43:57"},
+                {"date": "2016-12-24", "time": "02:27:31"},
+                {"date": "2017-08-31", "time": "00:35:26"}
+             ]
+        },
+        {
+            "business_id": "-0LPtgJC31FWMrMv317p0Q",
+            "checkin_times": [
+                {"date": "2013-04-13", "time": "12:35:33"},
+                {"date": "2013-08-19", "time": "23:35:49"},
+                {"date": "2013-10-04", "time": "19:14:56"}
+             ]
+        },
+        {
+            "business_id": "-0M3o2uWBnQZwd3hmfEwuw",
+            "checkin_times": [
+                {"date": "2016-09-10", "time": "19:26:19"},
+                {"date": "2018-09-08", "time": "14:15:37"},
+                {"date": "2019-09-13", "time": "22:47:25"}
+             ]
+        },
+        {
+            "business_id": "-0RRiWDtfnS16AKCtfvBZg",
+            "checkin_times": [
+                {"date": "2017-05-19", "time": "14:30:16"},
+                {"date": "2017-05-19", "time": "14:30:25"},
+                {"date": "2017-08-28", "time": "15:49:37"},
+                {"date": "2017-09-20", "time": "20:19:51"},
+                {"date": "2017-10-01", "time": "16:31:05"},
+                {"date": "2017-10-01", "time": "16:56:27"},
+                {"date": "2017-12-27", "time": "23:33:20"}
+             ]
+        },
+        {
+            "business_id": "-0Soj75v-XoRcf2ERr8Bmg",
+            "checkin_times": [
+                {"date": "2019-06-05", "time": "18:22:49"}
+             ]
+        },
+        {
+            "business_id": "-0ZumLlFjMh4ZW1z2nXGug",
+            "checkin_times": [
+                {"date": "2011-09-24", "time": "21:37:32"},
+                {"date": "2014-03-10", "time": "20:20:07"},
+                {"date": "2015-05-27", "time": "00:40:24"},
+                {"date": "2015-08-29", "time": "17:58:15"},
+                {"date": "2018-03-16", "time": "15:03:26"}
+             ]
+        },
+        {
+            "business_id": "-0aOudcaAyac0VJbMX-L1g",
+            "checkin_times": [
+                {"date": "2015-03-16", "time": "23:51:16"},
+                {"date": "2015-12-21", "time": "04:48:01"},
+                {"date": "2016-10-28", "time": "20:22:42"},
+                {"date": "2016-10-28", "time": "20:23:00"}
+             ]
+        },
+        {
+            "business_id": "-0b86isaXMY0v4g-V8GZ9Q",
+            "checkin_times": [
+                {"date": "2013-10-22", "time": "16:49:21"},
+                {"date": "2014-11-21", "time": "17:39:24"}
+             ]
+        },
+        {
+            "business_id": "-0d-BfFSU0bwLcnMaGRxYw",
+            "checkin_times": [
+                {"date": "2014-08-07", "time": "18:30:48"},
+                {"date": "2014-09-16", "time": "20:41:45"},
+                {"date": "2014-10-12", "time": "23:22:27"},
+                {"date": "2015-07-21", "time": "20:43:56"},
+                {"date": "2015-07-21", "time": "20:45:07"}
+             ]
+        },
+        {
+            "business_id": "-0jz6c3C6i7RG7Ag22K-Pg",
+            "checkin_times": [
+                {"date": "2015-05-02", "time": "19:49:05"},
+                {"date": "2015-05-06", "time": "03:52:18"},
+                {"date": "2015-09-26", "time": "01:13:19"}
+             ]
+        },
+        {
+            "business_id": "-0y3MZU2oYP8r1ruDP1bfQ",
+            "checkin_times": [
+                {"date": "2015-04-11", "time": "13:14:14"},
+                {"date": "2015-11-21", "time": "16:05:56"},
+                {"date": "2016-05-06", "time": "14:10:04"},
+                {"date": "2017-08-09", "time": "15:15:10"},
+                {"date": "2017-10-21", "time": "15:12:56"}
+             ]
+        },
+        {
+            "business_id": "-1BPe8UjF2_l3nVk-DFUjA",
+            "checkin_times": [
+                {"date": "2015-12-03", "time": "18:44:00"},
+                {"date": "2016-03-17", "time": "18:19:21"},
+                {"date": "2016-11-02", "time": "15:58:38"}
+             ]
+        },
+        {
+            "business_id": "-1E2CQu_38mkghvmZgCCRw",
+            "checkin_times": [
+                {"date": "2019-04-04", "time": "22:02:37"}
+             ]
+        },
+        {
+            "business_id": "-1wzk43IZ5D9Ysu6kzb5xA",
+            "checkin_times": [
+                {"date": "2019-02-27", "time": "14:03:08"}
+             ]
+        },
+        {
+            "business_id": "-23R9P2eG7VTc6DVLjFKzA",
+            "checkin_times": [
+                {"date": "2011-12-21", "time": "19:02:51"},
+                {"date": "2012-04-15", "time": "04:21:39"},
+                {"date": "2012-04-15", "time": "14:23:56"},
+                {"date": "2013-06-30", "time": "22:39:51"},
+                {"date": "2013-10-04", "time": "20:34:13"},
+                {"date": "2014-07-16", "time": "02:28:40"}
+             ]
+        },
+        {
+            "business_id": "-26MGfikhJiTfCI-GqmzhQ",
+            "checkin_times": [
+                {"date": "2018-06-13", "time": "20:16:07"}
+             ]
+        },
+        {
+            "business_id": "-2bLuJsMZ0WhI9daurVQNQ",
+            "checkin_times": [
+                {"date": "2015-05-29", "time": "16:46:17"},
+                {"date": "2015-06-01", "time": "15:03:53"}
+             ]
+        },
+        {
+            "business_id": "-2hDBMaza_ldqnZdiU06LQ",
+            "checkin_times": [
+                {"date": "2011-10-08", "time": "12:02:23"},
+                {"date": "2014-08-18", "time": "02:11:11"},
+                {"date": "2016-01-07", "time": "05:27:51"},
+                {"date": "2016-10-21", "time": "20:15:55"},
+                {"date": "2016-12-01", "time": "03:57:10"},
+                {"date": "2016-12-29", "time": "01:54:42"},
+                {"date": "2018-07-22", "time": "19:55:31"},
+                {"date": "2018-09-07", "time": "01:42:54"},
+                {"date": "2019-03-08", "time": "03:41:06"}
+             ]
+        }
+    ]
+);
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/insert-upsert-delete/open/use-case-3/use-case-3.7.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/insert-upsert-delete/open/use-case-3/use-case-3.7.query.sqlpp
new file mode 100644
index 0000000..2757b02
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/insert-upsert-delete/open/use-case-3/use-case-3.7.query.sqlpp
@@ -0,0 +1,25 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+SET `compiler.arrayindex` "true";
+
+USE TestYelp;
+
+SELECT COUNT(*)
+FROM YelpCheckin C, C.checkin_times D
+WHERE D.date > " ";
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/insert-upsert-delete/open/use-case-4/use-case-4.1.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/insert-upsert-delete/open/use-case-4/use-case-4.1.ddl.sqlpp
new file mode 100644
index 0000000..4b8453f
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/insert-upsert-delete/open/use-case-4/use-case-4.1.ddl.sqlpp
@@ -0,0 +1,37 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+/*
+ * Description: Verify the following DML actions for an array->record->array->atomic index:
+ *     1) Insert into an empty index (non bulk-load operation).
+ *     2) Delete all-but-one entry from the index.
+ *     3) Upsert all entries into the index.
+ */
+
+DROP DATAVERSE TestYelp IF EXISTS;
+CREATE DATAVERSE TestYelp;
+USE TestYelp;
+
+CREATE TYPE CheckinType AS {
+    business_id: string
+};
+
+CREATE DATASET YelpCheckin(CheckinType) PRIMARY KEY business_id;
+CREATE INDEX IdxYelpCheckinDates ON YelpCheckin (UNNEST checkin_times UNNEST dates : string ?) ;
+CREATE INDEX IdxYelpCheckinTimes ON YelpCheckin (UNNEST checkin_times UNNEST times : string ?) ;
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/insert-upsert-delete/open/use-case-4/use-case-4.2.update.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/insert-upsert-delete/open/use-case-4/use-case-4.2.update.sqlpp
new file mode 100644
index 0000000..0360912
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/insert-upsert-delete/open/use-case-4/use-case-4.2.update.sqlpp
@@ -0,0 +1,425 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+USE TestYelp;
+
+INSERT INTO YelpCheckin (
+      [
+        {
+            "business_id": "--1UhMGODdWsrMastO9DZw",
+            "checkin_times": [
+                {
+                    "dates": ["2016-04-26", "2016-08-30", "2016-10-15", "2016-11-18"],
+                    "times": ["19:49:16", "18:36:57", "02:45:18", "01:54:50"]
+                },
+                {
+                    "dates": ["2017-04-20", "2017-05-03"],
+                    "times": ["18:39:06", "17:58:02"]
+                },
+                {
+                    "dates": ["2019-03-19"],
+                    "times": ["22:04:48"]
+                }
+             ]
+        },
+        {
+            "business_id": "--EF5N7P70J_UYBTPypYlA",
+            "checkin_times": [
+                {
+                    "dates": ["2018-05-25", "2018-09-18"],
+                    "times": ["19:52:07", "16:09:44"]
+                },
+                {
+                    "dates": ["2019-10-18"],
+                    "times": ["21:29:09"]
+                }
+             ]
+        },
+        {
+            "business_id": "--Ni3oJ4VOqfOEu7Sj2Vzg",
+            "checkin_times": [
+                {
+                    "dates": ["2019-06-07"],
+                    "times": ["17:54:58"]
+                }
+             ]
+        },
+        {
+            "business_id": "--Y1Adl1YUWfYIRSd8vkmA",
+            "checkin_times": [
+                {
+                    "dates": ["2011-05-03", "2011-08-23"],
+                    "times": ["20:54:05", "20:49:45"]
+                },
+                {
+                    "dates": ["2014-12-04"],
+                    "times": ["06:13:01"]
+                },
+                {
+                    "dates": ["2016-11-16"],
+                    "times": ["19:25:55"]
+                }
+             ]
+        },
+        {
+            "business_id": "--YPwqIlRJrhHkJcjY3eiA",
+            "checkin_times": [
+               {
+                    "dates": ["2016-06-18", "2016-10-15"],
+                    "times": ["21:35:45", "18:17:51"]
+                }
+             ]
+        },
+        {
+            "business_id": "--e8PjCNhEz32pprnPhCwQ",
+            "checkin_times": [
+                {
+                    "dates": ["2015-04-02"],
+                    "times": ["21:45:17"]
+                }
+             ]
+        },
+        {
+            "business_id": "--kinfHwmtdjz03g8B8z8Q",
+            "checkin_times": [
+                {
+                    "dates": ["2014-08-27"],
+                    "times": ["17:49:18"]
+                },
+                {
+                    "dates": ["2015-12-19"],
+                    "times": ["21:30:31"]
+                },
+                {
+                    "dates": ["2018-11-27"],
+                    "times": ["15:53:50"]
+                }
+             ]
+        },
+        {
+            "business_id": "--q6datkI-f0EoVheXNEeQ",
+            "checkin_times": [
+                {
+                    "dates": ["2014-01-28", "2014-11-16"],
+                    "times": ["20:56:04", "16:11:58"]
+                },
+                {
+                    "dates": ["2015-11-15", "2015-11-15"],
+                    "times": ["19:21:53", "19:33:39"]
+                }
+             ]
+        },
+        {
+            "business_id": "--qvQS4MigHPykD2GV0-zw",
+            "checkin_times": [
+                {
+                    "dates": ["2019-04-11"],
+                    "times": ["18:30:12"]
+                }
+             ]
+        },
+        {
+            "business_id": "--wIGbLEhlpl_UeAIyDmZQ",
+            "checkin_times": [
+                {
+                    "dates": ["2015-06-06"],
+                    "times": ["20:01:06"]
+                },
+                {
+                    "dates": ["2019-03-14"],
+                    "times": ["22:01:52"]
+                }
+             ]
+        },
+        {
+            "business_id": "-0FA-Qdi3SPYIoJz9UQw-A",
+            "checkin_times": [
+                {
+                    "dates": ["2018-09-29", "2018-10-20", "2018-10-20"],
+                    "times": ["18:55:17", "16:48:05", "22:20:24"]
+                }
+             ]
+        },
+        {
+            "business_id": "-0Hj1hb_XW6ybWq2M7QhGA",
+            "checkin_times": [
+                {
+                    "dates": ["2011-04-23"],
+                    "times": ["21:11:22"]
+                },
+                {
+                    "dates": ["2014-05-04", "2014-05-11", "2014-06-04"],
+                    "times": ["19:42:48", "19:16:08", "19:14:18"]
+                },
+                {
+                    "dates": ["2015-12-05"],
+                    "times": ["19:22:42"]
+                },
+                {
+                    "dates": ["2017-05-15"],
+                    "times": ["23:19:00"]
+                }
+             ]
+        },
+        {
+            "business_id": "-0KMvRFwDWdVBeTpT11iHw",
+            "checkin_times": [
+                {
+                    "dates": ["2012-07-13"],
+                    "times": ["21:43:57"]
+                },
+                {
+                    "dates": ["2016-12-24"],
+                    "times": ["02:27:31"]
+                },
+                {
+                    "dates": ["2017-08-31"],
+                    "times": ["00:35:26"]
+                }
+             ]
+        },
+        {
+            "business_id": "-0LPtgJC31FWMrMv317p0Q",
+            "checkin_times": [
+                {
+                    "dates": ["2013-04-13", "2013-08-19", "2013-10-04"],
+                    "times": ["12:35:33", "23:35:49", "19:14:56"]
+                }
+
+             ]
+        },
+        {
+            "business_id": "-0M3o2uWBnQZwd3hmfEwuw",
+            "checkin_times": [
+                {
+                    "dates": ["2016-09-10"],
+                    "times": ["19:26:19"]
+                },
+                {
+                    "dates": ["2018-09-08"],
+                    "times": ["14:15:37"]
+                },
+                {
+                    "dates": ["2019-09-13"],
+                    "times": ["22:47:25"]
+                }
+             ]
+        },
+        {
+            "business_id": "-0RRiWDtfnS16AKCtfvBZg",
+            "checkin_times": [
+                 {
+                    "dates": ["2017-05-19", "2017-05-19", "2017-08-28", "2017-09-20", "2017-10-01", "2017-10-01", "2017-12-27"],
+                    "times": ["14:30:16", "14:30:25", "15:49:37", "20:19:51", "16:31:05", "16:56:27", "23:33:20"]
+                }
+             ]
+        },
+        {
+            "business_id": "-0Soj75v-XoRcf2ERr8Bmg",
+            "checkin_times": [
+                {
+                    "dates": ["2019-06-05"],
+                    "times": ["18:22:49"]
+                }
+             ]
+        },
+        {
+            "business_id": "-0ZumLlFjMh4ZW1z2nXGug",
+            "checkin_times": [
+                {
+                    "dates": ["2011-09-24"],
+                    "times": ["21:37:32"]
+                },
+                {
+                    "dates": ["2014-03-10"],
+                    "times": ["20:20:07"]
+                },
+                {
+                    "dates": ["2015-05-27", "2015-08-29"],
+                    "times": ["00:40:24", "17:58:15"]
+                },
+                {
+                    "dates": ["2018-03-16"],
+                    "times": ["15:03:26"]
+                }
+             ]
+        },
+        {
+            "business_id": "-0aOudcaAyac0VJbMX-L1g",
+            "checkin_times": [
+                {
+                    "dates": ["2015-03-16", "2015-12-21"],
+                    "times": ["23:51:16", "04:48:01"]
+                },
+                {
+                    "dates": ["2016-10-28", "2016-10-28"],
+                    "times": ["20:22:42", "20:23:00"]
+                }
+             ]
+        },
+        {
+            "business_id": "-0b86isaXMY0v4g-V8GZ9Q",
+            "checkin_times": [
+                {
+                    "dates": ["2013-10-22"],
+                    "times": ["16:49:21"]
+                },
+                {
+                    "dates": ["2014-11-21"],
+                    "times": ["17:39:24"]
+                }
+             ]
+        },
+        {
+            "business_id": "-0d-BfFSU0bwLcnMaGRxYw",
+            "checkin_times": [
+                {
+                    "dates": ["2014-08-07", "2014-09-16", "2014-10-12"],
+                    "times": ["18:30:48", "20:41:45", "23:22:27"]
+                },
+                {
+                    "dates": ["2015-07-21", "2015-07-21"],
+                    "times": ["20:43:56", "20:45:07"]
+                }
+             ]
+        },
+        {
+            "business_id": "-0jz6c3C6i7RG7Ag22K-Pg",
+            "checkin_times": [
+                {
+                    "dates": ["2015-05-02", "2015-05-06", "2015-09-26"],
+                    "times": ["19:49:05", "03:52:18", "01:13:19"]
+                }
+
+             ]
+        },
+        {
+            "business_id": "-0y3MZU2oYP8r1ruDP1bfQ",
+            "checkin_times": [
+                {
+                    "dates": ["2015-04-11", "2015-11-21"],
+                    "times": ["13:14:14", "16:05:56"]
+                },
+                {
+                    "dates": ["2016-05-06"],
+                    "times": ["14:10:04"]
+                },
+                {
+                    "dates": ["2017-08-09", "2017-10-21"],
+                    "times": ["15:15:10", "15:12:56"]
+                }
+             ]
+        },
+        {
+            "business_id": "-1BPe8UjF2_l3nVk-DFUjA",
+            "checkin_times": [
+                {
+                    "dates": ["2015-12-03"],
+                    "times": ["18:44:00"]
+                },
+                {
+                    "dates": ["2016-03-17", "2016-11-02"],
+                    "times": ["18:19:21", "15:58:38"]
+                }
+             ]
+        },
+        {
+            "business_id": "-1E2CQu_38mkghvmZgCCRw",
+            "checkin_times": [
+                {
+                    "dates": ["2019-04-04"],
+                    "times": ["22:02:37"]
+                }
+             ]
+        },
+        {
+            "business_id": "-1wzk43IZ5D9Ysu6kzb5xA",
+            "checkin_times": [
+                {
+                    "dates": ["2019-02-27"],
+                    "times": ["14:03:08"]
+                }
+             ]
+        },
+        {
+            "business_id": "-23R9P2eG7VTc6DVLjFKzA",
+            "checkin_times": [
+                {
+                    "dates": ["2011-12-21"],
+                    "times": ["19:02:51"]
+                },
+                {
+                    "dates": ["2012-04-15", "2012-04-15"],
+                    "times": ["04:21:39", "14:23:56"]
+                },
+                {
+                    "dates": ["2013-06-30", "2013-10-04"],
+                    "times": ["22:39:51", "20:34:13"]
+                },
+                {
+                    "dates": ["2014-07-16"],
+                    "times": ["02:28:40"]
+                }
+             ]
+        },
+        {
+            "business_id": "-26MGfikhJiTfCI-GqmzhQ",
+            "checkin_times": [
+                {
+                    "dates": ["2018-06-13"],
+                    "times": ["20:16:07"]
+                }
+             ]
+        },
+        {
+            "business_id": "-2bLuJsMZ0WhI9daurVQNQ",
+            "checkin_times": [
+                {
+                    "dates": ["2015-05-29", "2015-06-01"],
+                    "times": ["16:46:17", "15:03:53"]
+                }
+             ]
+        },
+        {
+            "business_id": "-2hDBMaza_ldqnZdiU06LQ",
+            "checkin_times": [
+                {
+                    "dates": ["2011-10-08"],
+                    "times": ["12:02:23"]
+                },
+                {
+                    "dates": ["2014-08-18"],
+                    "times": ["02:11:11"]
+                },
+                {
+                    "dates": ["2016-01-07", "2016-10-21", "2016-12-01", "2016-12-29"],
+                    "times": ["05:27:51", "20:15:55", "03:57:10", "01:54:42"]
+                },
+                {
+                    "dates": ["2018-07-22", "2018-09-07"],
+                    "times": ["19:55:31", "01:42:54"]
+                },
+                {
+                    "dates": ["2019-03-08"],
+                    "times": ["03:41:06"]
+                }
+             ]
+        }
+    ]
+);
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/insert-upsert-delete/open/use-case-4/use-case-4.3.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/insert-upsert-delete/open/use-case-4/use-case-4.3.query.sqlpp
new file mode 100644
index 0000000..3406a35
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/insert-upsert-delete/open/use-case-4/use-case-4.3.query.sqlpp
@@ -0,0 +1,25 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+SET `compiler.arrayindex` "true";
+
+USE TestYelp;
+
+SELECT COUNT(*)
+FROM YelpCheckin C, C.checkin_times CT, CT.dates D
+WHERE D > " ";
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/insert-upsert-delete/open/use-case-4/use-case-4.4.update.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/insert-upsert-delete/open/use-case-4/use-case-4.4.update.sqlpp
new file mode 100644
index 0000000..ea1dba3
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/insert-upsert-delete/open/use-case-4/use-case-4.4.update.sqlpp
@@ -0,0 +1,23 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+USE TestYelp;
+
+DELETE FROM YelpCheckin C
+WHERE C.business_id != "--1UhMGODdWsrMastO9DZw";
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/insert-upsert-delete/open/use-case-4/use-case-4.5.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/insert-upsert-delete/open/use-case-4/use-case-4.5.query.sqlpp
new file mode 100644
index 0000000..3406a35
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/insert-upsert-delete/open/use-case-4/use-case-4.5.query.sqlpp
@@ -0,0 +1,25 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+SET `compiler.arrayindex` "true";
+
+USE TestYelp;
+
+SELECT COUNT(*)
+FROM YelpCheckin C, C.checkin_times CT, CT.dates D
+WHERE D > " ";
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/insert-upsert-delete/open/use-case-4/use-case-4.6.update.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/insert-upsert-delete/open/use-case-4/use-case-4.6.update.sqlpp
new file mode 100644
index 0000000..b5b6c72
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/insert-upsert-delete/open/use-case-4/use-case-4.6.update.sqlpp
@@ -0,0 +1,425 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+USE TestYelp;
+
+UPSERT INTO YelpCheckin (
+      [
+        {
+            "business_id": "--1UhMGODdWsrMastO9DZw",
+            "checkin_times": [
+                {
+                    "dates": ["2016-04-26", "2016-08-30", "2016-10-15", "2016-11-18"],
+                    "times": ["19:49:16", "18:36:57", "02:45:18", "01:54:50"]
+                },
+                {
+                    "dates": ["2017-04-20", "2017-05-03"],
+                    "times": ["18:39:06", "17:58:02"]
+                },
+                {
+                    "dates": ["2019-03-19"],
+                    "times": ["22:04:48"]
+                }
+             ]
+        },
+        {
+            "business_id": "--EF5N7P70J_UYBTPypYlA",
+            "checkin_times": [
+                {
+                    "dates": ["2018-05-25", "2018-09-18"],
+                    "times": ["19:52:07", "16:09:44"]
+                },
+                {
+                    "dates": ["2019-10-18"],
+                    "times": ["21:29:09"]
+                }
+             ]
+        },
+        {
+            "business_id": "--Ni3oJ4VOqfOEu7Sj2Vzg",
+            "checkin_times": [
+                {
+                    "dates": ["2019-06-07"],
+                    "times": ["17:54:58"]
+                }
+             ]
+        },
+        {
+            "business_id": "--Y1Adl1YUWfYIRSd8vkmA",
+            "checkin_times": [
+                {
+                    "dates": ["2011-05-03", "2011-08-23"],
+                    "times": ["20:54:05", "20:49:45"]
+                },
+                {
+                    "dates": ["2014-12-04"],
+                    "times": ["06:13:01"]
+                },
+                {
+                    "dates": ["2016-11-16"],
+                    "times": ["19:25:55"]
+                }
+             ]
+        },
+        {
+            "business_id": "--YPwqIlRJrhHkJcjY3eiA",
+            "checkin_times": [
+               {
+                    "dates": ["2016-06-18", "2016-10-15"],
+                    "times": ["21:35:45", "18:17:51"]
+                }
+             ]
+        },
+        {
+            "business_id": "--e8PjCNhEz32pprnPhCwQ",
+            "checkin_times": [
+                {
+                    "dates": ["2015-04-02"],
+                    "times": ["21:45:17"]
+                }
+             ]
+        },
+        {
+            "business_id": "--kinfHwmtdjz03g8B8z8Q",
+            "checkin_times": [
+                {
+                    "dates": ["2014-08-27"],
+                    "times": ["17:49:18"]
+                },
+                {
+                    "dates": ["2015-12-19"],
+                    "times": ["21:30:31"]
+                },
+                {
+                    "dates": ["2018-11-27"],
+                    "times": ["15:53:50"]
+                }
+             ]
+        },
+        {
+            "business_id": "--q6datkI-f0EoVheXNEeQ",
+            "checkin_times": [
+                {
+                    "dates": ["2014-01-28", "2014-11-16"],
+                    "times": ["20:56:04", "16:11:58"]
+                },
+                {
+                    "dates": ["2015-11-15", "2015-11-15"],
+                    "times": ["19:21:53", "19:33:39"]
+                }
+             ]
+        },
+        {
+            "business_id": "--qvQS4MigHPykD2GV0-zw",
+            "checkin_times": [
+                {
+                    "dates": ["2019-04-11"],
+                    "times": ["18:30:12"]
+                }
+             ]
+        },
+        {
+            "business_id": "--wIGbLEhlpl_UeAIyDmZQ",
+            "checkin_times": [
+                {
+                    "dates": ["2015-06-06"],
+                    "times": ["20:01:06"]
+                },
+                {
+                    "dates": ["2019-03-14"],
+                    "times": ["22:01:52"]
+                }
+             ]
+        },
+        {
+            "business_id": "-0FA-Qdi3SPYIoJz9UQw-A",
+            "checkin_times": [
+                {
+                    "dates": ["2018-09-29", "2018-10-20", "2018-10-20"],
+                    "times": ["18:55:17", "16:48:05", "22:20:24"]
+                }
+             ]
+        },
+        {
+            "business_id": "-0Hj1hb_XW6ybWq2M7QhGA",
+            "checkin_times": [
+                {
+                    "dates": ["2011-04-23"],
+                    "times": ["21:11:22"]
+                },
+                {
+                    "dates": ["2014-05-04", "2014-05-11", "2014-06-04"],
+                    "times": ["19:42:48", "19:16:08", "19:14:18"]
+                },
+                {
+                    "dates": ["2015-12-05"],
+                    "times": ["19:22:42"]
+                },
+                {
+                    "dates": ["2017-05-15"],
+                    "times": ["23:19:00"]
+                }
+             ]
+        },
+        {
+            "business_id": "-0KMvRFwDWdVBeTpT11iHw",
+            "checkin_times": [
+                {
+                    "dates": ["2012-07-13"],
+                    "times": ["21:43:57"]
+                },
+                {
+                    "dates": ["2016-12-24"],
+                    "times": ["02:27:31"]
+                },
+                {
+                    "dates": ["2017-08-31"],
+                    "times": ["00:35:26"]
+                }
+             ]
+        },
+        {
+            "business_id": "-0LPtgJC31FWMrMv317p0Q",
+            "checkin_times": [
+                {
+                    "dates": ["2013-04-13", "2013-08-19", "2013-10-04"],
+                    "times": ["12:35:33", "23:35:49", "19:14:56"]
+                }
+
+             ]
+        },
+        {
+            "business_id": "-0M3o2uWBnQZwd3hmfEwuw",
+            "checkin_times": [
+                {
+                    "dates": ["2016-09-10"],
+                    "times": ["19:26:19"]
+                },
+                {
+                    "dates": ["2018-09-08"],
+                    "times": ["14:15:37"]
+                },
+                {
+                    "dates": ["2019-09-13"],
+                    "times": ["22:47:25"]
+                }
+             ]
+        },
+        {
+            "business_id": "-0RRiWDtfnS16AKCtfvBZg",
+            "checkin_times": [
+                 {
+                    "dates": ["2017-05-19", "2017-05-19", "2017-08-28", "2017-09-20", "2017-10-01", "2017-10-01", "2017-12-27"],
+                    "times": ["14:30:16", "14:30:25", "15:49:37", "20:19:51", "16:31:05", "16:56:27", "23:33:20"]
+                }
+             ]
+        },
+        {
+            "business_id": "-0Soj75v-XoRcf2ERr8Bmg",
+            "checkin_times": [
+                {
+                    "dates": ["2019-06-05"],
+                    "times": ["18:22:49"]
+                }
+             ]
+        },
+        {
+            "business_id": "-0ZumLlFjMh4ZW1z2nXGug",
+            "checkin_times": [
+                {
+                    "dates": ["2011-09-24"],
+                    "times": ["21:37:32"]
+                },
+                {
+                    "dates": ["2014-03-10"],
+                    "times": ["20:20:07"]
+                },
+                {
+                    "dates": ["2015-05-27", "2015-08-29"],
+                    "times": ["00:40:24", "17:58:15"]
+                },
+                {
+                    "dates": ["2018-03-16"],
+                    "times": ["15:03:26"]
+                }
+             ]
+        },
+        {
+            "business_id": "-0aOudcaAyac0VJbMX-L1g",
+            "checkin_times": [
+                {
+                    "dates": ["2015-03-16", "2015-12-21"],
+                    "times": ["23:51:16", "04:48:01"]
+                },
+                {
+                    "dates": ["2016-10-28", "2016-10-28"],
+                    "times": ["20:22:42", "20:23:00"]
+                }
+             ]
+        },
+        {
+            "business_id": "-0b86isaXMY0v4g-V8GZ9Q",
+            "checkin_times": [
+                {
+                    "dates": ["2013-10-22"],
+                    "times": ["16:49:21"]
+                },
+                {
+                    "dates": ["2014-11-21"],
+                    "times": ["17:39:24"]
+                }
+             ]
+        },
+        {
+            "business_id": "-0d-BfFSU0bwLcnMaGRxYw",
+            "checkin_times": [
+                {
+                    "dates": ["2014-08-07", "2014-09-16", "2014-10-12"],
+                    "times": ["18:30:48", "20:41:45", "23:22:27"]
+                },
+                {
+                    "dates": ["2015-07-21", "2015-07-21"],
+                    "times": ["20:43:56", "20:45:07"]
+                }
+             ]
+        },
+        {
+            "business_id": "-0jz6c3C6i7RG7Ag22K-Pg",
+            "checkin_times": [
+                {
+                    "dates": ["2015-05-02", "2015-05-06", "2015-09-26"],
+                    "times": ["19:49:05", "03:52:18", "01:13:19"]
+                }
+
+             ]
+        },
+        {
+            "business_id": "-0y3MZU2oYP8r1ruDP1bfQ",
+            "checkin_times": [
+                {
+                    "dates": ["2015-04-11", "2015-11-21"],
+                    "times": ["13:14:14", "16:05:56"]
+                },
+                {
+                    "dates": ["2016-05-06"],
+                    "times": ["14:10:04"]
+                },
+                {
+                    "dates": ["2017-08-09", "2017-10-21"],
+                    "times": ["15:15:10", "15:12:56"]
+                }
+             ]
+        },
+        {
+            "business_id": "-1BPe8UjF2_l3nVk-DFUjA",
+            "checkin_times": [
+                {
+                    "dates": ["2015-12-03"],
+                    "times": ["18:44:00"]
+                },
+                {
+                    "dates": ["2016-03-17", "2016-11-02"],
+                    "times": ["18:19:21", "15:58:38"]
+                }
+             ]
+        },
+        {
+            "business_id": "-1E2CQu_38mkghvmZgCCRw",
+            "checkin_times": [
+                {
+                    "dates": ["2019-04-04"],
+                    "times": ["22:02:37"]
+                }
+             ]
+        },
+        {
+            "business_id": "-1wzk43IZ5D9Ysu6kzb5xA",
+            "checkin_times": [
+                {
+                    "dates": ["2019-02-27"],
+                    "times": ["14:03:08"]
+                }
+             ]
+        },
+        {
+            "business_id": "-23R9P2eG7VTc6DVLjFKzA",
+            "checkin_times": [
+                {
+                    "dates": ["2011-12-21"],
+                    "times": ["19:02:51"]
+                },
+                {
+                    "dates": ["2012-04-15", "2012-04-15"],
+                    "times": ["04:21:39", "14:23:56"]
+                },
+                {
+                    "dates": ["2013-06-30", "2013-10-04"],
+                    "times": ["22:39:51", "20:34:13"]
+                },
+                {
+                    "dates": ["2014-07-16"],
+                    "times": ["02:28:40"]
+                }
+             ]
+        },
+        {
+            "business_id": "-26MGfikhJiTfCI-GqmzhQ",
+            "checkin_times": [
+                {
+                    "dates": ["2018-06-13"],
+                    "times": ["20:16:07"]
+                }
+             ]
+        },
+        {
+            "business_id": "-2bLuJsMZ0WhI9daurVQNQ",
+            "checkin_times": [
+                {
+                    "dates": ["2015-05-29", "2015-06-01"],
+                    "times": ["16:46:17", "15:03:53"]
+                }
+             ]
+        },
+        {
+            "business_id": "-2hDBMaza_ldqnZdiU06LQ",
+            "checkin_times": [
+                {
+                    "dates": ["2011-10-08"],
+                    "times": ["12:02:23"]
+                },
+                {
+                    "dates": ["2014-08-18"],
+                    "times": ["02:11:11"]
+                },
+                {
+                    "dates": ["2016-01-07", "2016-10-21", "2016-12-01", "2016-12-29"],
+                    "times": ["05:27:51", "20:15:55", "03:57:10", "01:54:42"]
+                },
+                {
+                    "dates": ["2018-07-22", "2018-09-07"],
+                    "times": ["19:55:31", "01:42:54"]
+                },
+                {
+                    "dates": ["2019-03-08"],
+                    "times": ["03:41:06"]
+                }
+             ]
+        }
+    ]
+);
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/insert-upsert-delete/open/use-case-4/use-case-4.7.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/insert-upsert-delete/open/use-case-4/use-case-4.7.query.sqlpp
new file mode 100644
index 0000000..3406a35
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/insert-upsert-delete/open/use-case-4/use-case-4.7.query.sqlpp
@@ -0,0 +1,25 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+SET `compiler.arrayindex` "true";
+
+USE TestYelp;
+
+SELECT COUNT(*)
+FROM YelpCheckin C, C.checkin_times CT, CT.dates D
+WHERE D > " ";
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/insert-upsert-delete/open/with-additional-atomic-index/with-additional-atomic-index.1.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/insert-upsert-delete/open/with-additional-atomic-index/with-additional-atomic-index.1.ddl.sqlpp
new file mode 100644
index 0000000..5edd2a0
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/insert-upsert-delete/open/with-additional-atomic-index/with-additional-atomic-index.1.ddl.sqlpp
@@ -0,0 +1,37 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+/*
+ * Description: Verify the following DML actions for an array->atomic index, with an additional atomic index:
+ *     1) Insert into an empty index (non bulk-load operation), additionally with two records that have no array-index qualifying entries.
+ *     2) Delete all-but-one entry from the index.
+ *     3) Upsert all *original* (all records have qualifying array-index entries now) entries into the index.
+ */
+
+DROP DATAVERSE TestYelp IF EXISTS;
+CREATE DATAVERSE TestYelp;
+USE TestYelp;
+
+CREATE TYPE CheckinType AS {
+    checkin_id: int
+};
+
+CREATE DATASET YelpCheckin(CheckinType) PRIMARY KEY checkin_id;
+CREATE INDEX IdxYelpCheckinDates ON YelpCheckin (UNNEST dates : string ?) ;
+CREATE INDEX IdxYelpCheckinBusinessID ON YelpCheckin (business_id : string ?) ;
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/insert-upsert-delete/open/with-additional-atomic-index/with-additional-atomic-index.2.update.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/insert-upsert-delete/open/with-additional-atomic-index/with-additional-atomic-index.2.update.sqlpp
new file mode 100644
index 0000000..8ee1570
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/insert-upsert-delete/open/with-additional-atomic-index/with-additional-atomic-index.2.update.sqlpp
@@ -0,0 +1,302 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+USE TestYelp;
+
+INSERT INTO YelpCheckin (
+    [
+        {
+            "checkin_id": 1,
+            "business_id": "--1UhMGODdWsrMastO9DZw",
+            "dates": [
+                "2016-04-26 19:49:16",
+                "2016-08-30 18:36:57",
+                "2016-10-15 02:45:18",
+                "2016-11-18 01:54:50",
+                "2017-04-20 18:39:06",
+                "2017-05-03 17:58:02",
+                "2019-03-19 22:04:48"
+            ]
+        },
+        {
+            "checkin_id": 2,
+            "business_id": "--EF5N7P70J_UYBTPypYlA",
+            "dates": [
+                "2018-05-25 19:52:07",
+                "2018-09-18 16:09:44",
+                "2019-10-18 21:29:09"
+            ]
+        },
+        {
+            "checkin_id": 3,
+            "business_id": "--Ni3oJ4VOqfOEu7Sj2Vzg",
+            "dates": [
+                "2019-06-07 17:54:58"
+            ]
+        },
+        {
+            "checkin_id": 4,
+            "business_id": "--Y1Adl1YUWfYIRSd8vkmA",
+            "dates": [
+                "2011-05-03 20:54:05",
+                "2011-08-23 20:49:45",
+                "2014-12-04 06:13:01",
+                "2016-11-16 19:25:55"
+            ]
+        },
+        {
+            "checkin_id": 5,
+            "business_id": "--YPwqIlRJrhHkJcjY3eiA",
+            "dates": [
+                "2016-06-18 21:35:45",
+                "2016-10-15 18:17:51"
+            ]
+        },
+        {
+            "checkin_id": 6,
+            "business_id": "--e8PjCNhEz32pprnPhCwQ",
+            "dates": [
+                "2015-04-02 21:45:17"
+            ]
+        },
+        {
+            "checkin_id": 7,
+            "business_id": "--kinfHwmtdjz03g8B8z8Q",
+            "dates": [
+                "2014-08-27 17:49:18",
+                "2015-12-19 21:30:31",
+                "2018-11-27 15:53:50"
+            ]
+        },
+        {
+            "checkin_id": 8,
+            "business_id": "--q6datkI-f0EoVheXNEeQ",
+            "dates": [
+                "2014-01-28 20:56:04",
+                "2014-11-16 16:11:58",
+                "2015-11-15 19:21:53",
+                "2015-11-15 19:33:39"
+            ]
+        },
+        {
+            "checkin_id": 9,
+            "business_id": "--qvQS4MigHPykD2GV0-zw",
+            "dates": [
+                "2019-04-11 18:30:12"
+            ]
+        },
+        {
+            "checkin_id": 10,
+            "business_id": "--wIGbLEhlpl_UeAIyDmZQ",
+            "dates": [
+                "2015-06-06 20:01:06",
+                "2019-03-14 22:01:52"
+            ]
+        },
+        {
+            "checkin_id": 11,
+            "business_id": "-0FA-Qdi3SPYIoJz9UQw-A",
+            "dates": [
+                "2018-09-29 18:55:17",
+                "2018-10-20 16:48:05",
+                "2018-10-20 22:20:24"
+            ]
+        },
+        {
+            "checkin_id": 12,
+            "business_id": "-0Hj1hb_XW6ybWq2M7QhGA",
+            "dates": [
+                "2011-04-23 21:11:22",
+                "2014-05-04 19:42:48",
+                "2014-05-11 19:16:08",
+                "2014-06-04 19:14:18",
+                "2015-12-05 19:22:42",
+                "2017-05-15 23:19:00"
+            ]
+        },
+        {
+            "checkin_id": 13,
+            "business_id": "-0KMvRFwDWdVBeTpT11iHw",
+            "dates": [
+                "2012-07-13 21:43:57",
+                "2016-12-24 02:27:31",
+                "2017-08-31 00:35:26"
+            ]
+        },
+        {
+            "checkin_id": 14,
+            "business_id": "-0LPtgJC31FWMrMv317p0Q",
+            "dates": [
+                "2013-04-13 12:35:33",
+                "2013-08-19 23:35:49",
+                "2013-10-04 19:14:56"
+            ]
+        },
+        {
+            "checkin_id": 15,
+            "business_id": "-0M3o2uWBnQZwd3hmfEwuw",
+            "dates": [
+                "2016-09-10 19:26:19",
+                "2018-09-08 14:15:37",
+                "2019-09-13 22:47:25"
+            ]
+        },
+        {
+            "checkin_id": 16,
+            "business_id": "-0RRiWDtfnS16AKCtfvBZg",
+            "dates": [
+                "2017-05-19 14:30:16",
+                "2017-05-19 14:30:25",
+                "2017-08-28 15:49:37",
+                "2017-09-20 20:19:51",
+                "2017-10-01 16:31:05",
+                "2017-10-01 16:56:27",
+                "2017-12-27 23:33:20"
+            ]
+        },
+        {
+            "checkin_id": 17,
+            "business_id": "-0Soj75v-XoRcf2ERr8Bmg",
+            "dates": [
+                "2019-06-05 18:22:49"
+            ]
+        },
+        {
+            "checkin_id": 18,
+            "business_id": "-0ZumLlFjMh4ZW1z2nXGug",
+            "dates": [
+                "2011-09-24 21:37:32",
+                "2014-03-10 20:20:07",
+                "2015-05-27 00:40:24",
+                "2015-08-29 17:58:15",
+                "2018-03-16 15:03:26"
+            ]
+        },
+        {
+            "checkin_id": 19,
+            "business_id": "-0aOudcaAyac0VJbMX-L1g",
+            "dates": [
+                "2015-03-16 23:51:16",
+                "2015-12-21 04:48:01",
+                "2016-10-28 20:22:42",
+                "2016-10-28 20:23:00"
+            ]
+        },
+        {
+            "checkin_id": 20,
+            "business_id": "-0b86isaXMY0v4g-V8GZ9Q",
+            "dates": [
+                "2013-10-22 16:49:21",
+                "2014-11-21 17:39:24"
+            ]
+        },
+        {
+            "checkin_id": 21,
+            "business_id": "-0d-BfFSU0bwLcnMaGRxYw",
+            "dates": [
+                "2014-08-07 18:30:48",
+                "2014-09-16 20:41:45",
+                "2014-10-12 23:22:27",
+                "2015-07-21 20:43:56",
+                "2015-07-21 20:45:07"
+            ]
+        },
+        {
+            "checkin_id": 22,
+            "business_id": "-0jz6c3C6i7RG7Ag22K-Pg",
+            "dates": [
+                "2015-05-02 19:49:05",
+                "2015-05-06 03:52:18",
+                "2015-09-26 01:13:19"
+            ]
+        },
+        {
+            "checkin_id": 23,
+            "business_id": "-0y3MZU2oYP8r1ruDP1bfQ",
+            "dates": [
+                "2015-04-11 13:14:14",
+                "2015-11-21 16:05:56",
+                "2016-05-06 14:10:04",
+                "2017-08-09 15:15:10",
+                "2017-10-21 15:12:56"
+            ]
+        },
+        {
+            "checkin_id": 24,
+            "business_id": "-1BPe8UjF2_l3nVk-DFUjA",
+            "dates": [
+                "2015-12-03 18:44:00",
+                "2016-03-17 18:19:21",
+                "2016-11-02 15:58:38"
+            ]
+        },
+        {
+            "checkin_id": 25,
+            "business_id": "-1E2CQu_38mkghvmZgCCRw",
+            "dates": []
+        },
+        {
+            "checkin_id": 26,
+            "business_id": "-1wzk43IZ5D9Ysu6kzb5xA",
+            "dates": []
+        },
+        {
+            "checkin_id": 27,
+            "business_id": "-23R9P2eG7VTc6DVLjFKzA",
+            "dates": [
+                "2011-12-21 19:02:51",
+                "2012-04-15 04:21:39",
+                "2012-04-15 14:23:56",
+                "2013-06-30 22:39:51",
+                "2013-10-04 20:34:13",
+                "2014-07-16 02:28:40"
+            ]
+        },
+        {
+            "checkin_id": 28,
+            "business_id": "-26MGfikhJiTfCI-GqmzhQ",
+            "dates": [
+                "2018-06-13 20:16:07"
+            ]
+        },
+        {
+            "checkin_id": 29,
+            "business_id": "-2bLuJsMZ0WhI9daurVQNQ",
+            "dates": [
+                "2015-05-29 16:46:17",
+                "2015-06-01 15:03:53"
+            ]
+        },
+        {
+            "checkin_id": 30,
+            "business_id": "-2hDBMaza_ldqnZdiU06LQ",
+            "dates": [
+                "2011-10-08 12:02:23",
+                "2014-08-18 02:11:11",
+                "2016-01-07 05:27:51",
+                "2016-10-21 20:15:55",
+                "2016-12-01 03:57:10",
+                "2016-12-29 01:54:42",
+                "2018-07-22 19:55:31",
+                "2018-09-07 01:42:54",
+                "2019-03-08 03:41:06"
+            ]
+        }
+    ]
+);
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/insert-upsert-delete/open/with-additional-atomic-index/with-additional-atomic-index.3.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/insert-upsert-delete/open/with-additional-atomic-index/with-additional-atomic-index.3.query.sqlpp
new file mode 100644
index 0000000..69fe7cc
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/insert-upsert-delete/open/with-additional-atomic-index/with-additional-atomic-index.3.query.sqlpp
@@ -0,0 +1,25 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+SET `compiler.arrayindex` "true";
+
+USE TestYelp;
+
+SELECT COUNT(*)
+FROM YelpCheckin C, C.dates D
+WHERE D > " ";
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/insert-upsert-delete/open/with-additional-atomic-index/with-additional-atomic-index.4.update.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/insert-upsert-delete/open/with-additional-atomic-index/with-additional-atomic-index.4.update.sqlpp
new file mode 100644
index 0000000..ea1dba3
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/insert-upsert-delete/open/with-additional-atomic-index/with-additional-atomic-index.4.update.sqlpp
@@ -0,0 +1,23 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+USE TestYelp;
+
+DELETE FROM YelpCheckin C
+WHERE C.business_id != "--1UhMGODdWsrMastO9DZw";
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/insert-upsert-delete/open/with-additional-atomic-index/with-additional-atomic-index.5.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/insert-upsert-delete/open/with-additional-atomic-index/with-additional-atomic-index.5.query.sqlpp
new file mode 100644
index 0000000..69fe7cc
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/insert-upsert-delete/open/with-additional-atomic-index/with-additional-atomic-index.5.query.sqlpp
@@ -0,0 +1,25 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+SET `compiler.arrayindex` "true";
+
+USE TestYelp;
+
+SELECT COUNT(*)
+FROM YelpCheckin C, C.dates D
+WHERE D > " ";
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/insert-upsert-delete/open/with-additional-atomic-index/with-additional-atomic-index.6.update.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/insert-upsert-delete/open/with-additional-atomic-index/with-additional-atomic-index.6.update.sqlpp
new file mode 100644
index 0000000..965ccc8
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/insert-upsert-delete/open/with-additional-atomic-index/with-additional-atomic-index.6.update.sqlpp
@@ -0,0 +1,306 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+USE TestYelp;
+
+UPSERT INTO YelpCheckin (
+    [
+        {
+            "checkin_id": 1,
+            "business_id": "--1UhMGODdWsrMastO9DZw",
+            "dates": [
+                "2016-04-26 19:49:16",
+                "2016-08-30 18:36:57",
+                "2016-10-15 02:45:18",
+                "2016-11-18 01:54:50",
+                "2017-04-20 18:39:06",
+                "2017-05-03 17:58:02",
+                "2019-03-19 22:04:48"
+            ]
+        },
+        {
+            "checkin_id": 2,
+            "business_id": "--EF5N7P70J_UYBTPypYlA",
+            "dates": [
+                "2018-05-25 19:52:07",
+                "2018-09-18 16:09:44",
+                "2019-10-18 21:29:09"
+            ]
+        },
+        {
+            "checkin_id": 3,
+            "business_id": "--Ni3oJ4VOqfOEu7Sj2Vzg",
+            "dates": [
+                "2019-06-07 17:54:58"
+            ]
+        },
+        {
+            "checkin_id": 4,
+            "business_id": "--Y1Adl1YUWfYIRSd8vkmA",
+            "dates": [
+                "2011-05-03 20:54:05",
+                "2011-08-23 20:49:45",
+                "2014-12-04 06:13:01",
+                "2016-11-16 19:25:55"
+            ]
+        },
+        {
+            "checkin_id": 5,
+            "business_id": "--YPwqIlRJrhHkJcjY3eiA",
+            "dates": [
+                "2016-06-18 21:35:45",
+                "2016-10-15 18:17:51"
+            ]
+        },
+        {
+            "checkin_id": 6,
+            "business_id": "--e8PjCNhEz32pprnPhCwQ",
+            "dates": [
+                "2015-04-02 21:45:17"
+            ]
+        },
+        {
+            "checkin_id": 7,
+            "business_id": "--kinfHwmtdjz03g8B8z8Q",
+            "dates": [
+                "2014-08-27 17:49:18",
+                "2015-12-19 21:30:31",
+                "2018-11-27 15:53:50"
+            ]
+        },
+        {
+            "checkin_id": 8,
+            "business_id": "--q6datkI-f0EoVheXNEeQ",
+            "dates": [
+                "2014-01-28 20:56:04",
+                "2014-11-16 16:11:58",
+                "2015-11-15 19:21:53",
+                "2015-11-15 19:33:39"
+            ]
+        },
+        {
+            "checkin_id": 9,
+            "business_id": "--qvQS4MigHPykD2GV0-zw",
+            "dates": [
+                "2019-04-11 18:30:12"
+            ]
+        },
+        {
+            "checkin_id": 10,
+            "business_id": "--wIGbLEhlpl_UeAIyDmZQ",
+            "dates": [
+                "2015-06-06 20:01:06",
+                "2019-03-14 22:01:52"
+            ]
+        },
+        {
+            "checkin_id": 11,
+            "business_id": "-0FA-Qdi3SPYIoJz9UQw-A",
+            "dates": [
+                "2018-09-29 18:55:17",
+                "2018-10-20 16:48:05",
+                "2018-10-20 22:20:24"
+            ]
+        },
+        {
+            "checkin_id": 12,
+            "business_id": "-0Hj1hb_XW6ybWq2M7QhGA",
+            "dates": [
+                "2011-04-23 21:11:22",
+                "2014-05-04 19:42:48",
+                "2014-05-11 19:16:08",
+                "2014-06-04 19:14:18",
+                "2015-12-05 19:22:42",
+                "2017-05-15 23:19:00"
+            ]
+        },
+        {
+            "checkin_id": 13,
+            "business_id": "-0KMvRFwDWdVBeTpT11iHw",
+            "dates": [
+                "2012-07-13 21:43:57",
+                "2016-12-24 02:27:31",
+                "2017-08-31 00:35:26"
+            ]
+        },
+        {
+            "checkin_id": 14,
+            "business_id": "-0LPtgJC31FWMrMv317p0Q",
+            "dates": [
+                "2013-04-13 12:35:33",
+                "2013-08-19 23:35:49",
+                "2013-10-04 19:14:56"
+            ]
+        },
+        {
+            "checkin_id": 15,
+            "business_id": "-0M3o2uWBnQZwd3hmfEwuw",
+            "dates": [
+                "2016-09-10 19:26:19",
+                "2018-09-08 14:15:37",
+                "2019-09-13 22:47:25"
+            ]
+        },
+        {
+            "checkin_id": 16,
+            "business_id": "-0RRiWDtfnS16AKCtfvBZg",
+            "dates": [
+                "2017-05-19 14:30:16",
+                "2017-05-19 14:30:25",
+                "2017-08-28 15:49:37",
+                "2017-09-20 20:19:51",
+                "2017-10-01 16:31:05",
+                "2017-10-01 16:56:27",
+                "2017-12-27 23:33:20"
+            ]
+        },
+        {
+            "checkin_id": 17,
+            "business_id": "-0Soj75v-XoRcf2ERr8Bmg",
+            "dates": [
+                "2019-06-05 18:22:49"
+            ]
+        },
+        {
+            "checkin_id": 18,
+            "business_id": "-0ZumLlFjMh4ZW1z2nXGug",
+            "dates": [
+                "2011-09-24 21:37:32",
+                "2014-03-10 20:20:07",
+                "2015-05-27 00:40:24",
+                "2015-08-29 17:58:15",
+                "2018-03-16 15:03:26"
+            ]
+        },
+        {
+            "checkin_id": 19,
+            "business_id": "-0aOudcaAyac0VJbMX-L1g",
+            "dates": [
+                "2015-03-16 23:51:16",
+                "2015-12-21 04:48:01",
+                "2016-10-28 20:22:42",
+                "2016-10-28 20:23:00"
+            ]
+        },
+        {
+            "checkin_id": 20,
+            "business_id": "-0b86isaXMY0v4g-V8GZ9Q",
+            "dates": [
+                "2013-10-22 16:49:21",
+                "2014-11-21 17:39:24"
+            ]
+        },
+        {
+            "checkin_id": 21,
+            "business_id": "-0d-BfFSU0bwLcnMaGRxYw",
+            "dates": [
+                "2014-08-07 18:30:48",
+                "2014-09-16 20:41:45",
+                "2014-10-12 23:22:27",
+                "2015-07-21 20:43:56",
+                "2015-07-21 20:45:07"
+            ]
+        },
+        {
+            "checkin_id": 22,
+            "business_id": "-0jz6c3C6i7RG7Ag22K-Pg",
+            "dates": [
+                "2015-05-02 19:49:05",
+                "2015-05-06 03:52:18",
+                "2015-09-26 01:13:19"
+            ]
+        },
+        {
+            "checkin_id": 23,
+            "business_id": "-0y3MZU2oYP8r1ruDP1bfQ",
+            "dates": [
+                "2015-04-11 13:14:14",
+                "2015-11-21 16:05:56",
+                "2016-05-06 14:10:04",
+                "2017-08-09 15:15:10",
+                "2017-10-21 15:12:56"
+            ]
+        },
+        {
+            "checkin_id": 24,
+            "business_id": "-1BPe8UjF2_l3nVk-DFUjA",
+            "dates": [
+                "2015-12-03 18:44:00",
+                "2016-03-17 18:19:21",
+                "2016-11-02 15:58:38"
+            ]
+        },
+        {
+            "checkin_id": 25,
+            "business_id": "-1E2CQu_38mkghvmZgCCRw",
+            "dates": [
+                "2019-04-04 22:02:37"
+            ]
+        },
+        {
+            "checkin_id": 26,
+            "business_id": "-1wzk43IZ5D9Ysu6kzb5xA",
+            "dates": [
+                "2019-02-27 14:03:08"
+            ]
+        },
+        {
+            "checkin_id": 27,
+            "business_id": "-23R9P2eG7VTc6DVLjFKzA",
+            "dates": [
+                "2011-12-21 19:02:51",
+                "2012-04-15 04:21:39",
+                "2012-04-15 14:23:56",
+                "2013-06-30 22:39:51",
+                "2013-10-04 20:34:13",
+                "2014-07-16 02:28:40"
+            ]
+        },
+        {
+            "checkin_id": 28,
+            "business_id": "-26MGfikhJiTfCI-GqmzhQ",
+            "dates": [
+                "2018-06-13 20:16:07"
+            ]
+        },
+        {
+            "checkin_id": 29,
+            "business_id": "-2bLuJsMZ0WhI9daurVQNQ",
+            "dates": [
+                "2015-05-29 16:46:17",
+                "2015-06-01 15:03:53"
+            ]
+        },
+        {
+            "checkin_id": 30,
+            "business_id": "-2hDBMaza_ldqnZdiU06LQ",
+            "dates": [
+                "2011-10-08 12:02:23",
+                "2014-08-18 02:11:11",
+                "2016-01-07 05:27:51",
+                "2016-10-21 20:15:55",
+                "2016-12-01 03:57:10",
+                "2016-12-29 01:54:42",
+                "2018-07-22 19:55:31",
+                "2018-09-07 01:42:54",
+                "2019-03-08 03:41:06"
+            ]
+        }
+    ]
+);
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/insert-upsert-delete/open/with-additional-atomic-index/with-additional-atomic-index.7.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/insert-upsert-delete/open/with-additional-atomic-index/with-additional-atomic-index.7.query.sqlpp
new file mode 100644
index 0000000..69fe7cc
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/insert-upsert-delete/open/with-additional-atomic-index/with-additional-atomic-index.7.query.sqlpp
@@ -0,0 +1,25 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+SET `compiler.arrayindex` "true";
+
+USE TestYelp;
+
+SELECT COUNT(*)
+FROM YelpCheckin C, C.dates D
+WHERE D > " ";
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/insert-upsert-delete/open/with-composite-sk/with-composite-sk.1.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/insert-upsert-delete/open/with-composite-sk/with-composite-sk.1.ddl.sqlpp
new file mode 100644
index 0000000..9046654
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/insert-upsert-delete/open/with-composite-sk/with-composite-sk.1.ddl.sqlpp
@@ -0,0 +1,36 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+/*
+ * Description: Verify the following DML actions for a composite array->atomic index:
+ *     1) Insert into an empty index (non bulk-load operation).
+ *     2) Delete all-but-one entry from the index.
+ *     3) Upsert all entries into the index.
+ */
+
+DROP DATAVERSE TestYelp IF EXISTS;
+CREATE DATAVERSE TestYelp;
+USE TestYelp;
+
+CREATE TYPE CheckinType AS {
+    checkin_id: int
+};
+
+CREATE DATASET YelpCheckin(CheckinType) PRIMARY KEY checkin_id;
+CREATE INDEX IdxYelpCheckinDates ON YelpCheckin (UNNEST checkin_times SELECT date : string ?, time : string ?);
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/insert-upsert-delete/open/with-composite-sk/with-composite-sk.2.update.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/insert-upsert-delete/open/with-composite-sk/with-composite-sk.2.update.sqlpp
new file mode 100644
index 0000000..2f7a9fd
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/insert-upsert-delete/open/with-composite-sk/with-composite-sk.2.update.sqlpp
@@ -0,0 +1,306 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+USE TestYelp;
+
+INSERT INTO YelpCheckin (
+    [
+        {
+            "checkin_id": 1,
+            "business_id": "--1UhMGODdWsrMastO9DZw",
+            "checkin_times": [ 
+                {"date": "2016-04-26", "time": "19:49:16"},
+                {"date": "2016-08-30", "time": "18:36:57"},
+                {"date": "2016-10-15", "time": "02:45:18"},
+                {"date": "2016-11-18", "time": "01:54:50"},
+                {"date": "2017-04-20", "time": "18:39:06"},
+                {"date": "2017-05-03", "time": "17:58:02"},
+                {"date": "2019-03-19", "time": "22:04:48"}
+             ]
+        },
+        {
+            "checkin_id": 2,
+            "business_id": "--EF5N7P70J_UYBTPypYlA",
+            "checkin_times": [ 
+                {"date": "2018-05-25", "time": "19:52:07"},
+                {"date": "2018-09-18", "time": "16:09:44"},
+                {"date": "2019-10-18", "time": "21:29:09"}
+             ]
+        },
+        {
+            "checkin_id": 3,
+            "business_id": "--Ni3oJ4VOqfOEu7Sj2Vzg",
+            "checkin_times": [ 
+                {"date": "2019-06-07", "time": "17:54:58"}
+             ]
+        },
+        {
+            "checkin_id": 4,
+            "business_id": "--Y1Adl1YUWfYIRSd8vkmA",
+            "checkin_times": [ 
+                {"date": "2011-05-03", "time": "20:54:05"},
+                {"date": "2011-08-23", "time": "20:49:45"},
+                {"date": "2014-12-04", "time": "06:13:01"},
+                {"date": "2016-11-16", "time": "19:25:55"}
+             ]
+        },
+        {
+            "checkin_id": 5,
+            "business_id": "--YPwqIlRJrhHkJcjY3eiA",
+            "checkin_times": [ 
+                {"date": "2016-06-18", "time": "21:35:45"},
+                {"date": "2016-10-15", "time": "18:17:51"}
+             ]
+        },
+        {
+            "checkin_id": 6,
+            "business_id": "--e8PjCNhEz32pprnPhCwQ",
+            "checkin_times": [ 
+                {"date": "2015-04-02", "time": "21:45:17"}
+             ]
+        },
+        {
+            "checkin_id": 7,
+            "business_id": "--kinfHwmtdjz03g8B8z8Q",
+            "checkin_times": [ 
+                {"date": "2014-08-27", "time": "17:49:18"},
+                {"date": "2015-12-19", "time": "21:30:31"},
+                {"date": "2018-11-27", "time": "15:53:50"}
+             ]
+        },
+        {
+            "checkin_id": 8,
+            "business_id": "--q6datkI-f0EoVheXNEeQ",
+            "checkin_times": [ 
+                {"date": "2014-01-28", "time": "20:56:04"},
+                {"date": "2014-11-16", "time": "16:11:58"},
+                {"date": "2015-11-15", "time": "19:21:53"},
+                {"date": "2015-11-15", "time": "19:33:39"}
+             ]
+        },
+        {
+            "checkin_id": 9,
+            "business_id": "--qvQS4MigHPykD2GV0-zw",
+            "checkin_times": [ 
+                {"date": "2019-04-11", "time": "18:30:12"}
+             ]
+        },
+        {
+            "checkin_id": 10,
+            "business_id": "--wIGbLEhlpl_UeAIyDmZQ",
+            "checkin_times": [ 
+                {"date": "2015-06-06", "time": "20:01:06"},
+                {"date": "2019-03-14", "time": "22:01:52"}
+             ]
+        },
+        {
+            "checkin_id": 11,
+            "business_id": "-0FA-Qdi3SPYIoJz9UQw-A",
+            "checkin_times": [ 
+                {"date": "2018-09-29", "time": "18:55:17"},
+                {"date": "2018-10-20", "time": "16:48:05"},
+                {"date": "2018-10-20", "time": "22:20:24"}
+             ]
+        },
+        {
+            "checkin_id": 12,
+            "business_id": "-0Hj1hb_XW6ybWq2M7QhGA",
+            "checkin_times": [ 
+                {"date": "2011-04-23", "time": "21:11:22"},
+                {"date": "2014-05-04", "time": "19:42:48"},
+                {"date": "2014-05-11", "time": "19:16:08"},
+                {"date": "2014-06-04", "time": "19:14:18"},
+                {"date": "2015-12-05", "time": "19:22:42"},
+                {"date": "2017-05-15", "time": "23:19:00"}
+             ]
+        },
+        {
+            "checkin_id": 13,
+            "business_id": "-0KMvRFwDWdVBeTpT11iHw",
+            "checkin_times": [ 
+                {"date": "2012-07-13", "time": "21:43:57"},
+                {"date": "2016-12-24", "time": "02:27:31"},
+                {"date": "2017-08-31", "time": "00:35:26"}
+             ]
+        },
+        {
+            "checkin_id": 14,
+            "business_id": "-0LPtgJC31FWMrMv317p0Q",
+            "checkin_times": [ 
+                {"date": "2013-04-13", "time": "12:35:33"},
+                {"date": "2013-08-19", "time": "23:35:49"},
+                {"date": "2013-10-04", "time": "19:14:56"}
+             ]
+        },
+        {
+            "checkin_id": 15,
+            "business_id": "-0M3o2uWBnQZwd3hmfEwuw",
+            "checkin_times": [ 
+                {"date": "2016-09-10", "time": "19:26:19"},
+                {"date": "2018-09-08", "time": "14:15:37"},
+                {"date": "2019-09-13", "time": "22:47:25"}
+             ]
+        },
+        {
+            "checkin_id": 16,
+            "business_id": "-0RRiWDtfnS16AKCtfvBZg",
+            "checkin_times": [ 
+                {"date": "2017-05-19", "time": "14:30:16"},
+                {"date": "2017-05-19", "time": "14:30:25"},
+                {"date": "2017-08-28", "time": "15:49:37"},
+                {"date": "2017-09-20", "time": "20:19:51"},
+                {"date": "2017-10-01", "time": "16:31:05"},
+                {"date": "2017-10-01", "time": "16:56:27"},
+                {"date": "2017-12-27", "time": "23:33:20"}
+             ]
+        },
+        {
+            "checkin_id": 17,
+            "business_id": "-0Soj75v-XoRcf2ERr8Bmg",
+            "checkin_times": [ 
+                {"date": "2019-06-05", "time": "18:22:49"}
+             ]
+        },
+        {
+            "checkin_id": 18,
+            "business_id": "-0ZumLlFjMh4ZW1z2nXGug",
+            "checkin_times": [ 
+                {"date": "2011-09-24", "time": "21:37:32"},
+                {"date": "2014-03-10", "time": "20:20:07"},
+                {"date": "2015-05-27", "time": "00:40:24"},
+                {"date": "2015-08-29", "time": "17:58:15"},
+                {"date": "2018-03-16", "time": "15:03:26"}
+             ]
+        },
+        {
+            "checkin_id": 19,
+            "business_id": "-0aOudcaAyac0VJbMX-L1g",
+            "checkin_times": [ 
+                {"date": "2015-03-16", "time": "23:51:16"},
+                {"date": "2015-12-21", "time": "04:48:01"},
+                {"date": "2016-10-28", "time": "20:22:42"},
+                {"date": "2016-10-28", "time": "20:23:00"}
+             ]
+        },
+        {
+            "checkin_id": 20,
+            "business_id": "-0b86isaXMY0v4g-V8GZ9Q",
+            "checkin_times": [ 
+                {"date": "2013-10-22", "time": "16:49:21"},
+                {"date": "2014-11-21", "time": "17:39:24"}
+             ]
+        },
+        {
+            "checkin_id": 21,
+            "business_id": "-0d-BfFSU0bwLcnMaGRxYw",
+            "checkin_times": [ 
+                {"date": "2014-08-07", "time": "18:30:48"},
+                {"date": "2014-09-16", "time": "20:41:45"},
+                {"date": "2014-10-12", "time": "23:22:27"},
+                {"date": "2015-07-21", "time": "20:43:56"},
+                {"date": "2015-07-21", "time": "20:45:07"}
+             ]
+        },
+        {
+            "checkin_id": 22,
+            "business_id": "-0jz6c3C6i7RG7Ag22K-Pg",
+            "checkin_times": [ 
+                {"date": "2015-05-02", "time": "19:49:05"},
+                {"date": "2015-05-06", "time": "03:52:18"},
+                {"date": "2015-09-26", "time": "01:13:19"}
+             ]
+        },
+        {
+            "checkin_id": 23,
+            "business_id": "-0y3MZU2oYP8r1ruDP1bfQ",
+            "checkin_times": [ 
+                {"date": "2015-04-11", "time": "13:14:14"},
+                {"date": "2015-11-21", "time": "16:05:56"},
+                {"date": "2016-05-06", "time": "14:10:04"},
+                {"date": "2017-08-09", "time": "15:15:10"},
+                {"date": "2017-10-21", "time": "15:12:56"}
+             ]
+        },
+        {
+            "checkin_id": 24,
+            "business_id": "-1BPe8UjF2_l3nVk-DFUjA",
+            "checkin_times": [ 
+                {"date": "2015-12-03", "time": "18:44:00"},
+                {"date": "2016-03-17", "time": "18:19:21"},
+                {"date": "2016-11-02", "time": "15:58:38"}
+             ]
+        },
+        {
+            "checkin_id": 25,
+            "business_id": "-1E2CQu_38mkghvmZgCCRw",
+            "checkin_times": [ 
+                {"date": "2019-04-04", "time": "22:02:37"}
+             ]
+        },
+        {
+            "checkin_id": 26,
+            "business_id": "-1wzk43IZ5D9Ysu6kzb5xA",
+            "checkin_times": [ 
+                {"date": "2019-02-27", "time": "14:03:08"}
+             ]
+        },
+        {
+            "checkin_id": 27,
+            "business_id": "-23R9P2eG7VTc6DVLjFKzA",
+            "checkin_times": [ 
+                {"date": "2011-12-21", "time": "19:02:51"},
+                {"date": "2012-04-15", "time": "04:21:39"},
+                {"date": "2012-04-15", "time": "14:23:56"},
+                {"date": "2013-06-30", "time": "22:39:51"},
+                {"date": "2013-10-04", "time": "20:34:13"},
+                {"date": "2014-07-16", "time": "02:28:40"}
+             ]
+        },
+        {
+            "checkin_id": 28,
+            "business_id": "-26MGfikhJiTfCI-GqmzhQ",
+            "checkin_times": [ 
+                {"date": "2018-06-13", "time": "20:16:07"}
+             ]
+        },
+        {
+            "checkin_id": 29,
+            "business_id": "-2bLuJsMZ0WhI9daurVQNQ",
+            "checkin_times": [ 
+                {"date": "2015-05-29", "time": "16:46:17"},
+                {"date": "2015-06-01", "time": "15:03:53"}
+             ]
+        },
+        {
+            "checkin_id": 30,
+            "business_id": "-2hDBMaza_ldqnZdiU06LQ",
+            "checkin_times": [ 
+                {"date": "2011-10-08", "time": "12:02:23"},
+                {"date": "2014-08-18", "time": "02:11:11"},
+                {"date": "2016-01-07", "time": "05:27:51"},
+                {"date": "2016-10-21", "time": "20:15:55"},
+                {"date": "2016-12-01", "time": "03:57:10"},
+                {"date": "2016-12-29", "time": "01:54:42"},
+                {"date": "2018-07-22", "time": "19:55:31"},
+                {"date": "2018-09-07", "time": "01:42:54"},
+                {"date": "2019-03-08", "time": "03:41:06"}
+             ]
+        }
+    ]
+);
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/insert-upsert-delete/open/with-composite-sk/with-composite-sk.3.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/insert-upsert-delete/open/with-composite-sk/with-composite-sk.3.query.sqlpp
new file mode 100644
index 0000000..baf548c
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/insert-upsert-delete/open/with-composite-sk/with-composite-sk.3.query.sqlpp
@@ -0,0 +1,27 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+SET `compiler.arrayindex` "true";
+
+USE TestYelp;
+
+SELECT COUNT(*)
+FROM YelpCheckin C, C.checkin_times D
+WHERE D.date > " " AND 
+      D.time > " " AND 
+      C.business_id > " ";
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/insert-upsert-delete/open/with-composite-sk/with-composite-sk.4.update.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/insert-upsert-delete/open/with-composite-sk/with-composite-sk.4.update.sqlpp
new file mode 100644
index 0000000..ea1dba3
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/insert-upsert-delete/open/with-composite-sk/with-composite-sk.4.update.sqlpp
@@ -0,0 +1,23 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+USE TestYelp;
+
+DELETE FROM YelpCheckin C
+WHERE C.business_id != "--1UhMGODdWsrMastO9DZw";
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/insert-upsert-delete/open/with-composite-sk/with-composite-sk.5.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/insert-upsert-delete/open/with-composite-sk/with-composite-sk.5.query.sqlpp
new file mode 100644
index 0000000..baf548c
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/insert-upsert-delete/open/with-composite-sk/with-composite-sk.5.query.sqlpp
@@ -0,0 +1,27 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+SET `compiler.arrayindex` "true";
+
+USE TestYelp;
+
+SELECT COUNT(*)
+FROM YelpCheckin C, C.checkin_times D
+WHERE D.date > " " AND 
+      D.time > " " AND 
+      C.business_id > " ";
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/insert-upsert-delete/open/with-composite-sk/with-composite-sk.6.update.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/insert-upsert-delete/open/with-composite-sk/with-composite-sk.6.update.sqlpp
new file mode 100644
index 0000000..5ec7879
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/insert-upsert-delete/open/with-composite-sk/with-composite-sk.6.update.sqlpp
@@ -0,0 +1,306 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+USE TestYelp;
+
+UPSERT INTO YelpCheckin (
+    [
+        {
+            "checkin_id": 1,
+            "business_id": "--1UhMGODdWsrMastO9DZw",
+            "checkin_times": [ 
+                {"date": "2016-04-26", "time": "19:49:16"},
+                {"date": "2016-08-30", "time": "18:36:57"},
+                {"date": "2016-10-15", "time": "02:45:18"},
+                {"date": "2016-11-18", "time": "01:54:50"},
+                {"date": "2017-04-20", "time": "18:39:06"},
+                {"date": "2017-05-03", "time": "17:58:02"},
+                {"date": "2019-03-19", "time": "22:04:48"}
+             ]
+        },
+        {
+            "checkin_id": 2,
+            "business_id": "--EF5N7P70J_UYBTPypYlA",
+            "checkin_times": [ 
+                {"date": "2018-05-25", "time": "19:52:07"},
+                {"date": "2018-09-18", "time": "16:09:44"},
+                {"date": "2019-10-18", "time": "21:29:09"}
+             ]
+        },
+        {
+            "checkin_id": 3,
+            "business_id": "--Ni3oJ4VOqfOEu7Sj2Vzg",
+            "checkin_times": [ 
+                {"date": "2019-06-07", "time": "17:54:58"}
+             ]
+        },
+        {
+            "checkin_id": 4,
+            "business_id": "--Y1Adl1YUWfYIRSd8vkmA",
+            "checkin_times": [ 
+                {"date": "2011-05-03", "time": "20:54:05"},
+                {"date": "2011-08-23", "time": "20:49:45"},
+                {"date": "2014-12-04", "time": "06:13:01"},
+                {"date": "2016-11-16", "time": "19:25:55"}
+             ]
+        },
+        {
+            "checkin_id": 5,
+            "business_id": "--YPwqIlRJrhHkJcjY3eiA",
+            "checkin_times": [ 
+                {"date": "2016-06-18", "time": "21:35:45"},
+                {"date": "2016-10-15", "time": "18:17:51"}
+             ]
+        },
+        {
+            "checkin_id": 6,
+            "business_id": "--e8PjCNhEz32pprnPhCwQ",
+            "checkin_times": [ 
+                {"date": "2015-04-02", "time": "21:45:17"}
+             ]
+        },
+        {
+            "checkin_id": 7,
+            "business_id": "--kinfHwmtdjz03g8B8z8Q",
+            "checkin_times": [ 
+                {"date": "2014-08-27", "time": "17:49:18"},
+                {"date": "2015-12-19", "time": "21:30:31"},
+                {"date": "2018-11-27", "time": "15:53:50"}
+             ]
+        },
+        {
+            "checkin_id": 8,
+            "business_id": "--q6datkI-f0EoVheXNEeQ",
+            "checkin_times": [ 
+                {"date": "2014-01-28", "time": "20:56:04"},
+                {"date": "2014-11-16", "time": "16:11:58"},
+                {"date": "2015-11-15", "time": "19:21:53"},
+                {"date": "2015-11-15", "time": "19:33:39"}
+             ]
+        },
+        {
+            "checkin_id": 9,
+            "business_id": "--qvQS4MigHPykD2GV0-zw",
+            "checkin_times": [ 
+                {"date": "2019-04-11", "time": "18:30:12"}
+             ]
+        },
+        {
+            "checkin_id": 10,
+            "business_id": "--wIGbLEhlpl_UeAIyDmZQ",
+            "checkin_times": [ 
+                {"date": "2015-06-06", "time": "20:01:06"},
+                {"date": "2019-03-14", "time": "22:01:52"}
+             ]
+        },
+        {
+            "checkin_id": 11,
+            "business_id": "-0FA-Qdi3SPYIoJz9UQw-A",
+            "checkin_times": [ 
+                {"date": "2018-09-29", "time": "18:55:17"},
+                {"date": "2018-10-20", "time": "16:48:05"},
+                {"date": "2018-10-20", "time": "22:20:24"}
+             ]
+        },
+        {
+            "checkin_id": 12,
+            "business_id": "-0Hj1hb_XW6ybWq2M7QhGA",
+            "checkin_times": [ 
+                {"date": "2011-04-23", "time": "21:11:22"},
+                {"date": "2014-05-04", "time": "19:42:48"},
+                {"date": "2014-05-11", "time": "19:16:08"},
+                {"date": "2014-06-04", "time": "19:14:18"},
+                {"date": "2015-12-05", "time": "19:22:42"},
+                {"date": "2017-05-15", "time": "23:19:00"}
+             ]
+        },
+        {
+            "checkin_id": 13,
+            "business_id": "-0KMvRFwDWdVBeTpT11iHw",
+            "checkin_times": [ 
+                {"date": "2012-07-13", "time": "21:43:57"},
+                {"date": "2016-12-24", "time": "02:27:31"},
+                {"date": "2017-08-31", "time": "00:35:26"}
+             ]
+        },
+        {
+            "checkin_id": 14,
+            "business_id": "-0LPtgJC31FWMrMv317p0Q",
+            "checkin_times": [ 
+                {"date": "2013-04-13", "time": "12:35:33"},
+                {"date": "2013-08-19", "time": "23:35:49"},
+                {"date": "2013-10-04", "time": "19:14:56"}
+             ]
+        },
+        {
+            "checkin_id": 15,
+            "business_id": "-0M3o2uWBnQZwd3hmfEwuw",
+            "checkin_times": [ 
+                {"date": "2016-09-10", "time": "19:26:19"},
+                {"date": "2018-09-08", "time": "14:15:37"},
+                {"date": "2019-09-13", "time": "22:47:25"}
+             ]
+        },
+        {
+            "checkin_id": 16,
+            "business_id": "-0RRiWDtfnS16AKCtfvBZg",
+            "checkin_times": [ 
+                {"date": "2017-05-19", "time": "14:30:16"},
+                {"date": "2017-05-19", "time": "14:30:25"},
+                {"date": "2017-08-28", "time": "15:49:37"},
+                {"date": "2017-09-20", "time": "20:19:51"},
+                {"date": "2017-10-01", "time": "16:31:05"},
+                {"date": "2017-10-01", "time": "16:56:27"},
+                {"date": "2017-12-27", "time": "23:33:20"}
+             ]
+        },
+        {
+            "checkin_id": 17,
+            "business_id": "-0Soj75v-XoRcf2ERr8Bmg",
+            "checkin_times": [ 
+                {"date": "2019-06-05", "time": "18:22:49"}
+             ]
+        },
+        {
+            "checkin_id": 18,
+            "business_id": "-0ZumLlFjMh4ZW1z2nXGug",
+            "checkin_times": [ 
+                {"date": "2011-09-24", "time": "21:37:32"},
+                {"date": "2014-03-10", "time": "20:20:07"},
+                {"date": "2015-05-27", "time": "00:40:24"},
+                {"date": "2015-08-29", "time": "17:58:15"},
+                {"date": "2018-03-16", "time": "15:03:26"}
+             ]
+        },
+        {
+            "checkin_id": 19,
+            "business_id": "-0aOudcaAyac0VJbMX-L1g",
+            "checkin_times": [ 
+                {"date": "2015-03-16", "time": "23:51:16"},
+                {"date": "2015-12-21", "time": "04:48:01"},
+                {"date": "2016-10-28", "time": "20:22:42"},
+                {"date": "2016-10-28", "time": "20:23:00"}
+             ]
+        },
+        {
+            "checkin_id": 20,
+            "business_id": "-0b86isaXMY0v4g-V8GZ9Q",
+            "checkin_times": [ 
+                {"date": "2013-10-22", "time": "16:49:21"},
+                {"date": "2014-11-21", "time": "17:39:24"}
+             ]
+        },
+        {
+            "checkin_id": 21,
+            "business_id": "-0d-BfFSU0bwLcnMaGRxYw",
+            "checkin_times": [ 
+                {"date": "2014-08-07", "time": "18:30:48"},
+                {"date": "2014-09-16", "time": "20:41:45"},
+                {"date": "2014-10-12", "time": "23:22:27"},
+                {"date": "2015-07-21", "time": "20:43:56"},
+                {"date": "2015-07-21", "time": "20:45:07"}
+             ]
+        },
+        {
+            "checkin_id": 22,
+            "business_id": "-0jz6c3C6i7RG7Ag22K-Pg",
+            "checkin_times": [ 
+                {"date": "2015-05-02", "time": "19:49:05"},
+                {"date": "2015-05-06", "time": "03:52:18"},
+                {"date": "2015-09-26", "time": "01:13:19"}
+             ]
+        },
+        {
+            "checkin_id": 23,
+            "business_id": "-0y3MZU2oYP8r1ruDP1bfQ",
+            "checkin_times": [ 
+                {"date": "2015-04-11", "time": "13:14:14"},
+                {"date": "2015-11-21", "time": "16:05:56"},
+                {"date": "2016-05-06", "time": "14:10:04"},
+                {"date": "2017-08-09", "time": "15:15:10"},
+                {"date": "2017-10-21", "time": "15:12:56"}
+             ]
+        },
+        {
+            "checkin_id": 24,
+            "business_id": "-1BPe8UjF2_l3nVk-DFUjA",
+            "checkin_times": [ 
+                {"date": "2015-12-03", "time": "18:44:00"},
+                {"date": "2016-03-17", "time": "18:19:21"},
+                {"date": "2016-11-02", "time": "15:58:38"}
+             ]
+        },
+        {
+            "checkin_id": 25,
+            "business_id": "-1E2CQu_38mkghvmZgCCRw",
+            "checkin_times": [ 
+                {"date": "2019-04-04", "time": "22:02:37"}
+             ]
+        },
+        {
+            "checkin_id": 26,
+            "business_id": "-1wzk43IZ5D9Ysu6kzb5xA",
+            "checkin_times": [ 
+                {"date": "2019-02-27", "time": "14:03:08"}
+             ]
+        },
+        {
+            "checkin_id": 27,
+            "business_id": "-23R9P2eG7VTc6DVLjFKzA",
+            "checkin_times": [ 
+                {"date": "2011-12-21", "time": "19:02:51"},
+                {"date": "2012-04-15", "time": "04:21:39"},
+                {"date": "2012-04-15", "time": "14:23:56"},
+                {"date": "2013-06-30", "time": "22:39:51"},
+                {"date": "2013-10-04", "time": "20:34:13"},
+                {"date": "2014-07-16", "time": "02:28:40"}
+             ]
+        },
+        {
+            "checkin_id": 28,
+            "business_id": "-26MGfikhJiTfCI-GqmzhQ",
+            "checkin_times": [ 
+                {"date": "2018-06-13", "time": "20:16:07"}
+             ]
+        },
+        {
+            "checkin_id": 29,
+            "business_id": "-2bLuJsMZ0WhI9daurVQNQ",
+            "checkin_times": [ 
+                {"date": "2015-05-29", "time": "16:46:17"},
+                {"date": "2015-06-01", "time": "15:03:53"}
+             ]
+        },
+        {
+            "checkin_id": 30,
+            "business_id": "-2hDBMaza_ldqnZdiU06LQ",
+            "checkin_times": [ 
+                {"date": "2011-10-08", "time": "12:02:23"},
+                {"date": "2014-08-18", "time": "02:11:11"},
+                {"date": "2016-01-07", "time": "05:27:51"},
+                {"date": "2016-10-21", "time": "20:15:55"},
+                {"date": "2016-12-01", "time": "03:57:10"},
+                {"date": "2016-12-29", "time": "01:54:42"},
+                {"date": "2018-07-22", "time": "19:55:31"},
+                {"date": "2018-09-07", "time": "01:42:54"},
+                {"date": "2019-03-08", "time": "03:41:06"}
+             ]
+        }
+    ]
+);
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/insert-upsert-delete/open/with-composite-sk/with-composite-sk.7.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/insert-upsert-delete/open/with-composite-sk/with-composite-sk.7.query.sqlpp
new file mode 100644
index 0000000..baf548c
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/insert-upsert-delete/open/with-composite-sk/with-composite-sk.7.query.sqlpp
@@ -0,0 +1,27 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+SET `compiler.arrayindex` "true";
+
+USE TestYelp;
+
+SELECT COUNT(*)
+FROM YelpCheckin C, C.checkin_times D
+WHERE D.date > " " AND 
+      D.time > " " AND 
+      C.business_id > " ";
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/join-unnest-queries/use-case-1/use-case-1.1.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/join-unnest-queries/use-case-1/use-case-1.1.ddl.sqlpp
new file mode 100644
index 0000000..39f18a0
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/join-unnest-queries/use-case-1/use-case-1.1.ddl.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.
+ */
+
+/*
+ * Description: Verify that we are able to correctly perform the following types of queries using an array->atomic index.
+ *     1) Implicit INNER JOIN query, with additional predicate on inner collection before the unnest.
+ *     2) Implicit INNER JOIN query, with additional predicate on the outer collection.
+ *     3) Explicit INNER JOIN query using UNNEST subquery, no additional predicate.
+ *     TODO (GLENN) 4) Explicit LEFT OUTER JOIN query using UNNEST subquery, no additional predicate.
+ */
+
+DROP DATAVERSE TestYelp IF EXISTS;
+CREATE DATAVERSE TestYelp;
+USE TestYelp;
+
+CREATE TYPE CheckinType AS {
+    checkin_id: uuid,
+    business_id: string,
+    dates: [string]
+};
+CREATE TYPE CheckinDatesToMarkersType AS {
+    marker: string,
+    datetime: string,
+    date: string,
+    time: string
+};
+
+CREATE DATASET YelpCheckin(CheckinType) PRIMARY KEY checkin_id AUTOGENERATED;
+CREATE DATASET YelpCheckinDateMarkers (CheckinDatesToMarkersType) PRIMARY KEY marker;
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/join-unnest-queries/use-case-1/use-case-1.2.update.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/join-unnest-queries/use-case-1/use-case-1.2.update.sqlpp
new file mode 100644
index 0000000..b75bf1b
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/join-unnest-queries/use-case-1/use-case-1.2.update.sqlpp
@@ -0,0 +1,276 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+USE TestYelp;
+
+INSERT INTO YelpCheckin (
+    [
+        {
+            "business_id": "--1UhMGODdWsrMastO9DZw",
+            "dates": [
+                "2016-04-26 19:49:16",
+                "2016-08-30 18:36:57",
+                "2016-10-15 02:45:18",
+                "2016-11-18 01:54:50",
+                "2017-04-20 18:39:06",
+                "2017-05-03 17:58:02",
+                "2019-03-19 22:04:48"
+            ]
+        },
+        {
+            "business_id": "--EF5N7P70J_UYBTPypYlA",
+            "dates": [
+                "2018-05-25 19:52:07",
+                "2018-09-18 16:09:44",
+                "2019-10-18 21:29:09"
+            ]
+        },
+        {
+            "business_id": "--Ni3oJ4VOqfOEu7Sj2Vzg",
+            "dates": [
+                "2019-06-07 17:54:58"
+            ]
+        },
+        {
+            "business_id": "--Y1Adl1YUWfYIRSd8vkmA",
+            "dates": [
+                "2011-05-03 20:54:05",
+                "2011-08-23 20:49:45",
+                "2014-12-04 06:13:01",
+                "2016-11-16 19:25:55"
+            ]
+        },
+        {
+            "business_id": "--YPwqIlRJrhHkJcjY3eiA",
+            "dates": [
+                "2016-06-18 21:35:45",
+                "2016-10-15 18:17:51"
+            ]
+        },
+        {
+            "business_id": "--e8PjCNhEz32pprnPhCwQ",
+            "dates": [
+                "2015-04-02 21:45:17"
+            ]
+        },
+        {
+            "business_id": "--kinfHwmtdjz03g8B8z8Q",
+            "dates": [
+                "2014-08-27 17:49:18",
+                "2015-12-19 21:30:31",
+                "2018-11-27 15:53:50"
+            ]
+        },
+        {
+            "business_id": "--q6datkI-f0EoVheXNEeQ",
+            "dates": [
+                "2014-01-28 20:56:04",
+                "2014-11-16 16:11:58",
+                "2015-11-15 19:21:53",
+                "2015-11-15 19:33:39"
+            ]
+        },
+        {
+            "business_id": "--qvQS4MigHPykD2GV0-zw",
+            "dates": [
+                "2019-04-11 18:30:12"
+            ]
+        },
+        {
+            "business_id": "--wIGbLEhlpl_UeAIyDmZQ",
+            "dates": [
+                "2015-06-06 20:01:06",
+                "2019-03-14 22:01:52"
+            ]
+        },
+        {
+            "business_id": "-0FA-Qdi3SPYIoJz9UQw-A",
+            "dates": [
+                "2018-09-29 18:55:17",
+                "2018-10-20 16:48:05",
+                "2018-10-20 22:20:24"
+            ]
+        },
+        {
+            "business_id": "-0Hj1hb_XW6ybWq2M7QhGA",
+            "dates": [
+                "2011-04-23 21:11:22",
+                "2014-05-04 19:42:48",
+                "2014-05-11 19:16:08",
+                "2014-06-04 19:14:18",
+                "2015-12-05 19:22:42",
+                "2017-05-15 23:19:00"
+            ]
+        },
+        {
+            "business_id": "-0KMvRFwDWdVBeTpT11iHw",
+            "dates": [
+                "2012-07-13 21:43:57",
+                "2016-12-24 02:27:31",
+                "2017-08-31 00:35:26"
+            ]
+        },
+        {
+            "business_id": "-0LPtgJC31FWMrMv317p0Q",
+            "dates": [
+                "2013-04-13 12:35:33",
+                "2013-08-19 23:35:49",
+                "2013-10-04 19:14:56"
+            ]
+        },
+        {
+            "business_id": "-0M3o2uWBnQZwd3hmfEwuw",
+            "dates": [
+                "2016-09-10 19:26:19",
+                "2018-09-08 14:15:37",
+                "2019-09-13 22:47:25"
+            ]
+        },
+        {
+            "business_id": "-0RRiWDtfnS16AKCtfvBZg",
+            "dates": [
+                "2017-05-19 14:30:16",
+                "2017-05-19 14:30:25",
+                "2017-08-28 15:49:37",
+                "2017-09-20 20:19:51",
+                "2017-10-01 16:31:05",
+                "2017-10-01 16:56:27",
+                "2017-12-27 23:33:20"
+            ]
+        },
+        {
+            "business_id": "-0Soj75v-XoRcf2ERr8Bmg",
+            "dates": [
+                "2019-06-05 18:22:49"
+            ]
+        },
+        {
+            "business_id": "-0ZumLlFjMh4ZW1z2nXGug",
+            "dates": [
+                "2011-09-24 21:37:32",
+                "2014-03-10 20:20:07",
+                "2015-05-27 00:40:24",
+                "2015-08-29 17:58:15",
+                "2018-03-16 15:03:26"
+            ]
+        },
+        {
+            "business_id": "-0aOudcaAyac0VJbMX-L1g",
+            "dates": [
+                "2015-03-16 23:51:16",
+                "2015-12-21 04:48:01",
+                "2016-10-28 20:22:42",
+                "2016-10-28 20:23:00"
+            ]
+        },
+        {
+            "business_id": "-0b86isaXMY0v4g-V8GZ9Q",
+            "dates": [
+                "2013-10-22 16:49:21",
+                "2014-11-21 17:39:24"
+            ]
+        },
+        {
+            "business_id": "-0d-BfFSU0bwLcnMaGRxYw",
+            "dates": [
+                "2014-08-07 18:30:48",
+                "2014-09-16 20:41:45",
+                "2014-10-12 23:22:27",
+                "2015-07-21 20:43:56",
+                "2015-07-21 20:45:07"
+            ]
+        },
+        {
+            "business_id": "-0jz6c3C6i7RG7Ag22K-Pg",
+            "dates": [
+                "2015-05-02 19:49:05",
+                "2015-05-06 03:52:18",
+                "2015-09-26 01:13:19"
+            ]
+        },
+        {
+            "business_id": "-0y3MZU2oYP8r1ruDP1bfQ",
+            "dates": [
+                "2015-04-11 13:14:14",
+                "2015-11-21 16:05:56",
+                "2016-05-06 14:10:04",
+                "2017-08-09 15:15:10",
+                "2017-10-21 15:12:56"
+            ]
+        },
+        {
+            "business_id": "-1BPe8UjF2_l3nVk-DFUjA",
+            "dates": [
+                "2015-12-03 18:44:00",
+                "2016-03-17 18:19:21",
+                "2016-11-02 15:58:38"
+            ]
+        },
+        {
+            "business_id": "-1E2CQu_38mkghvmZgCCRw",
+            "dates": [
+                "2019-04-04 22:02:37"
+            ]
+        },
+        {
+            "business_id": "-1wzk43IZ5D9Ysu6kzb5xA",
+            "dates": [
+                "2019-02-27 14:03:08"
+            ]
+        },
+        {
+            "business_id": "-23R9P2eG7VTc6DVLjFKzA",
+            "dates": [
+                "2011-12-21 19:02:51",
+                "2012-04-15 04:21:39",
+                "2012-04-15 14:23:56",
+                "2013-06-30 22:39:51",
+                "2013-10-04 20:34:13",
+                "2014-07-16 02:28:40"
+            ]
+        },
+        {
+            "business_id": "-26MGfikhJiTfCI-GqmzhQ",
+            "dates": [
+                "2018-06-13 20:16:07"
+            ]
+        },
+        {
+            "business_id": "-2bLuJsMZ0WhI9daurVQNQ",
+            "dates": [
+                "2015-05-29 16:46:17",
+                "2015-06-01 15:03:53"
+            ]
+        },
+        {
+            "business_id": "-2hDBMaza_ldqnZdiU06LQ",
+            "dates": [
+                "2011-10-08 12:02:23",
+                "2014-08-18 02:11:11",
+                "2016-01-07 05:27:51",
+                "2016-10-21 20:15:55",
+                "2016-12-01 03:57:10",
+                "2016-12-29 01:54:42",
+                "2018-07-22 19:55:31",
+                "2018-09-07 01:42:54",
+                "2019-03-08 03:41:06"
+            ]
+        }
+    ]
+);
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/join-unnest-queries/use-case-1/use-case-1.3.update.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/join-unnest-queries/use-case-1/use-case-1.3.update.sqlpp
new file mode 100644
index 0000000..c1eca5d
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/join-unnest-queries/use-case-1/use-case-1.3.update.sqlpp
@@ -0,0 +1,132 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+USE TestYelp;
+
+INSERT INTO YelpCheckinDateMarkers (
+    [
+        { "marker": "4fa1e250-7a68-3adb-04ec-6569c36882f5", "datetime": "2016-04-26 19:49:16", "date": "2016-04-26", "time": "19:49:16" },
+        { "marker": "4fa1e250-7a68-3adb-04ec-6569c36882f6", "datetime": "2016-08-30 18:36:57", "date": "2016-08-30", "time": "18:36:57" },
+        { "marker": "4fa1e250-7a68-3adb-04ec-6569c36882f7", "datetime": "2016-10-15 02:45:18", "date": "2016-10-15", "time": "02:45:18" },
+        { "marker": "4fa1e250-7a68-3adb-04ec-6569c36882f8", "datetime": "2016-11-18 01:54:50", "date": "2016-11-18", "time": "01:54:50" },
+        { "marker": "4fa1e250-7a68-3adb-04ec-6569c36882f9", "datetime": "2017-04-20 18:39:06", "date": "2017-04-20", "time": "18:39:06" },
+        { "marker": "4fa1e250-7a68-3adb-04ec-6569c36882fa", "datetime": "2017-05-03 17:58:02", "date": "2017-05-03", "time": "17:58:02" },
+        { "marker": "4fa1e250-7a68-3adb-04ec-6569c36882fb", "datetime": "2019-03-19 22:04:48", "date": "2019-03-19", "time": "22:04:48" },
+        { "marker": "4fa1e250-7a68-3adb-04ec-6569c36882fc", "datetime": "2018-05-25 19:52:07", "date": "2018-05-25", "time": "19:52:07" },
+        { "marker": "4fa1e250-7a68-3adb-04ec-6569c36882fd", "datetime": "2018-09-18 16:09:44", "date": "2018-09-18", "time": "16:09:44" },
+        { "marker": "4fa1e250-7a68-3adb-04ec-6569c36882fe", "datetime": "2019-10-18 21:29:09", "date": "2019-10-18", "time": "21:29:09" },
+        { "marker": "4fa1e250-7a68-3adb-04ec-6569c36882ff", "datetime": "2011-05-03 20:54:05", "date": "2011-05-03", "time": "20:54:05" },
+        { "marker": "4fa1e250-7a68-3adb-04ec-6569c3688300", "datetime": "2011-08-23 20:49:45", "date": "2011-08-23", "time": "20:49:45" },
+        { "marker": "4fa1e250-7a68-3adb-04ec-6569c3688301", "datetime": "2014-12-04 06:13:01", "date": "2014-12-04", "time": "06:13:01" },
+        { "marker": "4fa1e250-7a68-3adb-04ec-6569c3688302", "datetime": "2016-11-16 19:25:55", "date": "2016-11-16", "time": "19:25:55" },
+        { "marker": "4fa1e250-7a68-3adb-04ec-6569c3688303", "datetime": "2016-06-18 21:35:45", "date": "2016-06-18", "time": "21:35:45" },
+        { "marker": "4fa1e250-7a68-3adb-04ec-6569c3688304", "datetime": "2016-10-15 18:17:51", "date": "2016-10-15", "time": "18:17:51" },
+        { "marker": "4fa1e250-7a68-3adb-04ec-6569c3688305", "datetime": "2014-08-27 17:49:18", "date": "2014-08-27", "time": "17:49:18" },
+        { "marker": "4fa1e250-7a68-3adb-04ec-6569c3688306", "datetime": "2015-12-19 21:30:31", "date": "2015-12-19", "time": "21:30:31" },
+        { "marker": "4fa1e250-7a68-3adb-04ec-6569c3688307", "datetime": "2018-11-27 15:53:50", "date": "2018-11-27", "time": "15:53:50" },
+        { "marker": "4fa1e250-7a68-3adb-04ec-6569c3688308", "datetime": "2019-04-11 18:30:12", "date": "2019-04-11", "time": "18:30:12" },
+        { "marker": "4fa1e250-7a68-3adb-04ec-6569c3688309", "datetime": "2015-06-06 20:01:06", "date": "2015-06-06", "time": "20:01:06" },
+        { "marker": "4fa1e250-7a68-3adb-04ec-6569c368830a", "datetime": "2019-03-14 22:01:52", "date": "2019-03-14", "time": "22:01:52" },
+        { "marker": "4fa1e250-7a68-3adb-04ec-6569c368830b", "datetime": "2012-07-13 21:43:57", "date": "2012-07-13", "time": "21:43:57" },
+        { "marker": "4fa1e250-7a68-3adb-04ec-6569c368830c", "datetime": "2016-12-24 02:27:31", "date": "2016-12-24", "time": "02:27:31" },
+        { "marker": "4fa1e250-7a68-3adb-04ec-6569c368830d", "datetime": "2017-08-31 00:35:26", "date": "2017-08-31", "time": "00:35:26" },
+        { "marker": "4fa1e250-7a68-3adb-04ec-6569c368830e", "datetime": "2013-04-13 12:35:33", "date": "2013-04-13", "time": "12:35:33" },
+        { "marker": "4fa1e250-7a68-3adb-04ec-6569c368830f", "datetime": "2013-08-19 23:35:49", "date": "2013-08-19", "time": "23:35:49" },
+        { "marker": "4fa1e250-7a68-3adb-04ec-6569c3688310", "datetime": "2013-10-04 19:14:56", "date": "2013-10-04", "time": "19:14:56" },
+        { "marker": "4fa1e250-7a68-3adb-04ec-6569c3688311", "datetime": "2016-09-10 19:26:19", "date": "2016-09-10", "time": "19:26:19" },
+        { "marker": "4fa1e250-7a68-3adb-04ec-6569c3688312", "datetime": "2018-09-08 14:15:37", "date": "2018-09-08", "time": "14:15:37" },
+        { "marker": "4fa1e250-7a68-3adb-04ec-6569c3688313", "datetime": "2019-09-13 22:47:25", "date": "2019-09-13", "time": "22:47:25" },
+        { "marker": "4fa1e250-7a68-3adb-04ec-6569c3688314", "datetime": "2019-06-05 18:22:49", "date": "2019-06-05", "time": "18:22:49" },
+        { "marker": "4fa1e250-7a68-3adb-04ec-6569c3688315", "datetime": "2015-03-16 23:51:16", "date": "2015-03-16", "time": "23:51:16" },
+        { "marker": "4fa1e250-7a68-3adb-04ec-6569c3688316", "datetime": "2015-12-21 04:48:01", "date": "2015-12-21", "time": "04:48:01" },
+        { "marker": "4fa1e250-7a68-3adb-04ec-6569c3688317", "datetime": "2016-10-28 20:22:42", "date": "2016-10-28", "time": "20:22:42" },
+        { "marker": "4fa1e250-7a68-3adb-04ec-6569c3688318", "datetime": "2016-10-28 20:23:00", "date": "2016-10-28", "time": "20:23:00" },
+        { "marker": "4fa1e250-7a68-3adb-04ec-6569c3688319", "datetime": "2014-08-07 18:30:48", "date": "2014-08-07", "time": "18:30:48" },
+        { "marker": "4fa1e250-7a68-3adb-04ec-6569c368831a", "datetime": "2014-09-16 20:41:45", "date": "2014-09-16", "time": "20:41:45" },
+        { "marker": "4fa1e250-7a68-3adb-04ec-6569c368831b", "datetime": "2014-10-12 23:22:27", "date": "2014-10-12", "time": "23:22:27" },
+        { "marker": "4fa1e250-7a68-3adb-04ec-6569c368831c", "datetime": "2015-07-21 20:43:56", "date": "2015-07-21", "time": "20:43:56" },
+        { "marker": "4fa1e250-7a68-3adb-04ec-6569c368831d", "datetime": "2015-07-21 20:45:07", "date": "2015-07-21", "time": "20:45:07" },
+        { "marker": "4fa1e250-7a68-3adb-04ec-6569c368831e", "datetime": "2015-05-02 19:49:05", "date": "2015-05-02", "time": "19:49:05" },
+        { "marker": "4fa1e250-7a68-3adb-04ec-6569c368831f", "datetime": "2015-05-06 03:52:18", "date": "2015-05-06", "time": "03:52:18" },
+        { "marker": "4fa1e250-7a68-3adb-04ec-6569c3688320", "datetime": "2015-09-26 01:13:19", "date": "2015-09-26", "time": "01:13:19" },
+        { "marker": "4fa1e250-7a68-3adb-04ec-6569c3688321", "datetime": "2019-04-04 22:02:37", "date": "2019-04-04", "time": "22:02:37" },
+        { "marker": "4fa1e250-7a68-3adb-04ec-6569c3688322", "datetime": "2011-12-21 19:02:51", "date": "2011-12-21", "time": "19:02:51" },
+        { "marker": "4fa1e250-7a68-3adb-04ec-6569c3688323", "datetime": "2012-04-15 04:21:39", "date": "2012-04-15", "time": "04:21:39" },
+        { "marker": "4fa1e250-7a68-3adb-04ec-6569c3688324", "datetime": "2012-04-15 14:23:56", "date": "2012-04-15", "time": "14:23:56" },
+        { "marker": "4fa1e250-7a68-3adb-04ec-6569c3688325", "datetime": "2013-06-30 22:39:51", "date": "2013-06-30", "time": "22:39:51" },
+        { "marker": "4fa1e250-7a68-3adb-04ec-6569c3688326", "datetime": "2013-10-04 20:34:13", "date": "2013-10-04", "time": "20:34:13" },
+        { "marker": "4fa1e250-7a68-3adb-04ec-6569c3688327", "datetime": "2014-07-16 02:28:40", "date": "2014-07-16", "time": "02:28:40" },
+        { "marker": "4fa1e250-7a68-3adb-04ec-6569c3688328", "datetime": "2018-06-13 20:16:07", "date": "2018-06-13", "time": "20:16:07" },
+        { "marker": "4fa1e250-7a68-3adb-04ec-6569c3688329", "datetime": "2011-10-08 12:02:23", "date": "2011-10-08", "time": "12:02:23" },
+        { "marker": "4fa1e250-7a68-3adb-04ec-6569c368832a", "datetime": "2014-08-18 02:11:11", "date": "2014-08-18", "time": "02:11:11" },
+        { "marker": "4fa1e250-7a68-3adb-04ec-6569c368832b", "datetime": "2016-01-07 05:27:51", "date": "2016-01-07", "time": "05:27:51" },
+        { "marker": "4fa1e250-7a68-3adb-04ec-6569c368832c", "datetime": "2016-10-21 20:15:55", "date": "2016-10-21", "time": "20:15:55" },
+        { "marker": "4fa1e250-7a68-3adb-04ec-6569c368832d", "datetime": "2016-12-01 03:57:10", "date": "2016-12-01", "time": "03:57:10" },
+        { "marker": "4fa1e250-7a68-3adb-04ec-6569c368832e", "datetime": "2016-12-29 01:54:42", "date": "2016-12-29", "time": "01:54:42" },
+        { "marker": "4fa1e250-7a68-3adb-04ec-6569c368832f", "datetime": "2018-07-22 19:55:31", "date": "2018-07-22", "time": "19:55:31" },
+        { "marker": "4fa1e250-7a68-3adb-04ec-6569c3688330", "datetime": "2018-09-07 01:42:54", "date": "2018-09-07", "time": "01:42:54" },
+        { "marker": "4fa1e250-7a68-3adb-04ec-6569c3688331", "datetime": "2019-03-08 03:41:06", "date": "2019-03-08", "time": "03:41:06" },
+        { "marker": "991ed2f7-3374-33a5-ee65-77f067848381", "datetime": "2019-06-07 17:54:58", "date": "2019-06-07", "time": "17:54:58" },
+        { "marker": "991ed2f7-3374-33a5-ee65-77f067848382", "datetime": "2015-04-02 21:45:17", "date": "2015-04-02", "time": "21:45:17" },
+        { "marker": "991ed2f7-3374-33a5-ee65-77f067848383", "datetime": "2014-01-28 20:56:04", "date": "2014-01-28", "time": "20:56:04" },
+        { "marker": "991ed2f7-3374-33a5-ee65-77f067848384", "datetime": "2014-11-16 16:11:58", "date": "2014-11-16", "time": "16:11:58" },
+        { "marker": "991ed2f7-3374-33a5-ee65-77f067848385", "datetime": "2015-11-15 19:21:53", "date": "2015-11-15", "time": "19:21:53" },
+        { "marker": "991ed2f7-3374-33a5-ee65-77f067848386", "datetime": "2015-11-15 19:33:39", "date": "2015-11-15", "time": "19:33:39" },
+        { "marker": "991ed2f7-3374-33a5-ee65-77f067848387", "datetime": "2018-09-29 18:55:17", "date": "2018-09-29", "time": "18:55:17" },
+        { "marker": "991ed2f7-3374-33a5-ee65-77f067848388", "datetime": "2018-10-20 16:48:05", "date": "2018-10-20", "time": "16:48:05" },
+        { "marker": "991ed2f7-3374-33a5-ee65-77f067848389", "datetime": "2018-10-20 22:20:24", "date": "2018-10-20", "time": "22:20:24" },
+        { "marker": "991ed2f7-3374-33a5-ee65-77f06784838a", "datetime": "2011-04-23 21:11:22", "date": "2011-04-23", "time": "21:11:22" },
+        { "marker": "991ed2f7-3374-33a5-ee65-77f06784838b", "datetime": "2014-05-04 19:42:48", "date": "2014-05-04", "time": "19:42:48" },
+        { "marker": "991ed2f7-3374-33a5-ee65-77f06784838c", "datetime": "2014-05-11 19:16:08", "date": "2014-05-11", "time": "19:16:08" },
+        { "marker": "991ed2f7-3374-33a5-ee65-77f06784838d", "datetime": "2014-06-04 19:14:18", "date": "2014-06-04", "time": "19:14:18" },
+        { "marker": "991ed2f7-3374-33a5-ee65-77f06784838e", "datetime": "2015-12-05 19:22:42", "date": "2015-12-05", "time": "19:22:42" },
+        { "marker": "991ed2f7-3374-33a5-ee65-77f06784838f", "datetime": "2017-05-15 23:19:00", "date": "2017-05-15", "time": "23:19:00" },
+        { "marker": "991ed2f7-3374-33a5-ee65-77f067848390", "datetime": "2017-05-19 14:30:16", "date": "2017-05-19", "time": "14:30:16" },
+        { "marker": "991ed2f7-3374-33a5-ee65-77f067848391", "datetime": "2017-05-19 14:30:25", "date": "2017-05-19", "time": "14:30:25" },
+        { "marker": "991ed2f7-3374-33a5-ee65-77f067848392", "datetime": "2017-08-28 15:49:37", "date": "2017-08-28", "time": "15:49:37" },
+        { "marker": "991ed2f7-3374-33a5-ee65-77f067848393", "datetime": "2017-09-20 20:19:51", "date": "2017-09-20", "time": "20:19:51" },
+        { "marker": "991ed2f7-3374-33a5-ee65-77f067848394", "datetime": "2017-10-01 16:31:05", "date": "2017-10-01", "time": "16:31:05" },
+        { "marker": "991ed2f7-3374-33a5-ee65-77f067848395", "datetime": "2017-10-01 16:56:27", "date": "2017-10-01", "time": "16:56:27" },
+        { "marker": "991ed2f7-3374-33a5-ee65-77f067848396", "datetime": "2017-12-27 23:33:20", "date": "2017-12-27", "time": "23:33:20" },
+        { "marker": "991ed2f7-3374-33a5-ee65-77f067848397", "datetime": "2011-09-24 21:37:32", "date": "2011-09-24", "time": "21:37:32" },
+        { "marker": "991ed2f7-3374-33a5-ee65-77f067848398", "datetime": "2014-03-10 20:20:07", "date": "2014-03-10", "time": "20:20:07" },
+        { "marker": "991ed2f7-3374-33a5-ee65-77f067848399", "datetime": "2015-05-27 00:40:24", "date": "2015-05-27", "time": "00:40:24" },
+        { "marker": "991ed2f7-3374-33a5-ee65-77f06784839a", "datetime": "2015-08-29 17:58:15", "date": "2015-08-29", "time": "17:58:15" },
+        { "marker": "991ed2f7-3374-33a5-ee65-77f06784839b", "datetime": "2018-03-16 15:03:26", "date": "2018-03-16", "time": "15:03:26" },
+        { "marker": "991ed2f7-3374-33a5-ee65-77f06784839c", "datetime": "2013-10-22 16:49:21", "date": "2013-10-22", "time": "16:49:21" },
+        { "marker": "991ed2f7-3374-33a5-ee65-77f06784839d", "datetime": "2014-11-21 17:39:24", "date": "2014-11-21", "time": "17:39:24" },
+        { "marker": "991ed2f7-3374-33a5-ee65-77f06784839e", "datetime": "2015-04-11 13:14:14", "date": "2015-04-11", "time": "13:14:14" },
+        { "marker": "991ed2f7-3374-33a5-ee65-77f06784839f", "datetime": "2015-11-21 16:05:56", "date": "2015-11-21", "time": "16:05:56" },
+        { "marker": "991ed2f7-3374-33a5-ee65-77f0678483a0", "datetime": "2016-05-06 14:10:04", "date": "2016-05-06", "time": "14:10:04" },
+        { "marker": "991ed2f7-3374-33a5-ee65-77f0678483a1", "datetime": "2017-08-09 15:15:10", "date": "2017-08-09", "time": "15:15:10" },
+        { "marker": "991ed2f7-3374-33a5-ee65-77f0678483a2", "datetime": "2017-10-21 15:12:56", "date": "2017-10-21", "time": "15:12:56" },
+        { "marker": "991ed2f7-3374-33a5-ee65-77f0678483a3", "datetime": "2015-12-03 18:44:00", "date": "2015-12-03", "time": "18:44:00" },
+        { "marker": "991ed2f7-3374-33a5-ee65-77f0678483a4", "datetime": "2016-03-17 18:19:21", "date": "2016-03-17", "time": "18:19:21" },
+        { "marker": "991ed2f7-3374-33a5-ee65-77f0678483a5", "datetime": "2016-11-02 15:58:38", "date": "2016-11-02", "time": "15:58:38" },
+        { "marker": "991ed2f7-3374-33a5-ee65-77f0678483a6", "datetime": "2019-02-27 14:03:08", "date": "2019-02-27", "time": "14:03:08" },
+        { "marker": "991ed2f7-3374-33a5-ee65-77f0678483a7", "datetime": "2015-05-29 16:46:17", "date": "2015-05-29", "time": "16:46:17" },
+        { "marker": "991ed2f7-3374-33a5-ee65-77f0678483a8", "datetime": "2015-06-01 15:03:53", "date": "2015-06-01", "time": "15:03:53" },
+        { "marker": "BAD_ENTRY_1", "datetime": "NA", "date": "NA", "time": "NA" },
+        { "marker": "BAD_ENTRY_2", "datetime": "NA", "date": "NA", "time": "NA" },
+        { "marker": "BAD_ENTRY_3", "datetime": "NA", "date": "NA", "time": "NA" },
+        { "marker": "BAD_ENTRY_4", "datetime": "NA", "date": "NA", "time": "NA" },
+        { "marker": "BAD_ENTRY_5", "datetime": "NA", "date": "NA", "time": "NA" },
+        { "marker": "BAD_ENTRY_6", "datetime": "NA", "date": "NA", "time": "NA" }
+    ]
+);
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/join-unnest-queries/use-case-1/use-case-1.4.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/join-unnest-queries/use-case-1/use-case-1.4.ddl.sqlpp
new file mode 100644
index 0000000..a0fc1bf
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/join-unnest-queries/use-case-1/use-case-1.4.ddl.sqlpp
@@ -0,0 +1,22 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+USE TestYelp;
+
+CREATE INDEX IdxYelpCheckinDates ON YelpCheckin (UNNEST dates);
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/join-unnest-queries/use-case-1/use-case-1.5.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/join-unnest-queries/use-case-1/use-case-1.5.query.sqlpp
new file mode 100644
index 0000000..bb9eea4
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/join-unnest-queries/use-case-1/use-case-1.5.query.sqlpp
@@ -0,0 +1,26 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+SET `compiler.arrayindex` "true";
+
+USE TestYelp;
+
+SELECT M.marker
+FROM YelpCheckinDateMarkers M, YelpCheckin C, C.dates D
+WHERE D /*+ indexnl */ = M.datetime
+      AND C.business_id = "--Ni3oJ4VOqfOEu7Sj2Vzg";
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/join-unnest-queries/use-case-1/use-case-1.6.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/join-unnest-queries/use-case-1/use-case-1.6.query.sqlpp
new file mode 100644
index 0000000..a66da8f
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/join-unnest-queries/use-case-1/use-case-1.6.query.sqlpp
@@ -0,0 +1,26 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+SET `compiler.arrayindex` "true";
+
+USE TestYelp;
+
+SELECT M.marker
+FROM YelpCheckinDateMarkers M, YelpCheckin C, C.dates D
+WHERE D /*+ indexnl */ = M.datetime
+      AND M.time = "19:49:16";
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/join-unnest-queries/use-case-1/use-case-1.7.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/join-unnest-queries/use-case-1/use-case-1.7.query.sqlpp
new file mode 100644
index 0000000..8bdf70e
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/join-unnest-queries/use-case-1/use-case-1.7.query.sqlpp
@@ -0,0 +1,28 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+SET `compiler.arrayindex` "true";
+
+USE TestYelp;
+
+SELECT COUNT(*)
+FROM YelpCheckinDateMarkers M
+INNER JOIN (
+    SELECT VALUE D
+    FROM YelpCheckin C, C.dates D
+) AS CD ON CD /*+ indexnl */ = M.datetime;
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/join-unnest-queries/use-case-2/use-case-2.1.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/join-unnest-queries/use-case-2/use-case-2.1.ddl.sqlpp
new file mode 100644
index 0000000..a00ffca
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/join-unnest-queries/use-case-2/use-case-2.1.ddl.sqlpp
@@ -0,0 +1,48 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+/*
+ * Description: Verify that we are able to correctly perform the following types of queries using an record->array->atomic index.
+ *     1) INNER JOIN query, with additional predicate on inner collection before the unnest.
+ *     2) INNER JOIN query, with additional predicate on the outer collection.
+ *     3) Explicit INNER JOIN query using UNNEST subquery, no additional predicate.
+ *     TODO (GLENN) 4) Explicit LEFT OUTER JOIN query using UNNEST subquery, no additional predicate.
+ */
+
+DROP DATAVERSE TestYelp IF EXISTS;
+CREATE DATAVERSE TestYelp;
+USE TestYelp;
+
+CREATE TYPE CheckinType AS {
+    checkin_id: uuid,
+    business_id: string,
+    checkin_times: {
+        dates: [string],
+        times: [string]
+    }
+};
+CREATE TYPE CheckinDatesToMarkersType AS {
+    marker: string,
+    datetime: string,
+    date: string,
+    time: string
+};
+
+CREATE DATASET YelpCheckin(CheckinType) PRIMARY KEY checkin_id AUTOGENERATED;
+CREATE DATASET YelpCheckinDateMarkers (CheckinDatesToMarkersType) PRIMARY KEY marker;
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/join-unnest-queries/use-case-2/use-case-2.2.update.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/join-unnest-queries/use-case-2/use-case-2.2.update.sqlpp
new file mode 100644
index 0000000..2549e8f
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/join-unnest-queries/use-case-2/use-case-2.2.update.sqlpp
@@ -0,0 +1,497 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+USE TestYelp;
+
+INSERT INTO YelpCheckin (
+    [
+        {
+            "business_id": "--1UhMGODdWsrMastO9DZw",
+            "checkin_times": {
+                "dates": [
+                    "2016-04-26",
+                    "2016-08-30",
+                    "2016-10-15",
+                    "2016-11-18",
+                    "2017-04-20",
+                    "2017-05-03",
+                    "2019-03-19"
+                ],
+                "times": [
+                    "19:49:16",
+                    "18:36:57",
+                    "02:45:18",
+                    "01:54:50",
+                    "18:39:06",
+                    "17:58:02",
+                    "22:04:48"
+                ]
+            }
+        },
+        {
+            "business_id": "--EF5N7P70J_UYBTPypYlA",
+            "checkin_times": {
+                "dates": [
+                    "2018-05-25",
+                    "2018-09-18",
+                    "2019-10-18"
+                ],
+                "times": [
+                    "19:52:07",
+                    "16:09:44",
+                    "21:29:09"
+                ]
+            }
+        },
+        {
+            "business_id": "--Ni3oJ4VOqfOEu7Sj2Vzg",
+            "checkin_times": {
+                "dates": [
+                    "2019-06-07"
+                ],
+                "times": [
+                    "17:54:58"
+                ]
+            }
+        },
+        {
+            "business_id": "--Y1Adl1YUWfYIRSd8vkmA",
+            "checkin_times": {
+                "dates": [
+                    "2011-05-03",
+                    "2011-08-23",
+                    "2014-12-04",
+                    "2016-11-16"
+                ],
+                "times": [
+                    "20:54:05",
+                    "20:49:45",
+                    "06:13:01",
+                    "19:25:55"
+                ]
+            }
+        },
+        {
+            "business_id": "--YPwqIlRJrhHkJcjY3eiA",
+            "checkin_times": {
+                "dates": [
+                    "2016-06-18",
+                    "2016-10-15"
+                ],
+                "times": [
+                    "21:35:45",
+                    "18:17:51"
+                ]
+            }
+        },
+        {
+            "business_id": "--e8PjCNhEz32pprnPhCwQ",
+            "checkin_times": {
+                "dates": [
+                    "2015-04-02"
+                ],
+                "times": [
+                    "21:45:17"
+                ]
+            }
+        },
+        {
+            "business_id": "--kinfHwmtdjz03g8B8z8Q",
+            "checkin_times": {
+                "dates": [
+                    "2014-08-27",
+                    "2015-12-19",
+                    "2018-11-27"
+                ],
+                "times": [
+                    "17:49:18",
+                    "21:30:31",
+                    "15:53:50"
+                ]
+            }
+        },
+        {
+            "business_id": "--q6datkI-f0EoVheXNEeQ",
+            "checkin_times": {
+                "dates": [
+                    "2014-01-28",
+                    "2014-11-16",
+                    "2015-11-15",
+                    "2015-11-15"
+                ],
+                "times": [
+                    "20:56:04",
+                    "16:11:58",
+                    "19:21:53",
+                    "19:33:39"
+                ]
+            }
+        },
+        {
+            "business_id": "--qvQS4MigHPykD2GV0-zw",
+            "checkin_times": {
+                "dates": [
+                    "2019-04-11"
+                ],
+                "times": [
+                    "18:30:12"
+                ]
+            }
+        },
+        {
+            "business_id": "--wIGbLEhlpl_UeAIyDmZQ",
+            "checkin_times": {
+                "dates": [
+                    "2015-06-06",
+                    "2019-03-14"
+                ],
+                "times": [
+                    "20:01:06",
+                    "22:01:52"
+                ]
+            }
+        },
+        {
+            "business_id": "-0FA-Qdi3SPYIoJz9UQw-A",
+            "checkin_times": {
+                "dates": [
+                    "2018-09-29",
+                    "2018-10-20",
+                    "2018-10-20"
+                ],
+                "times": [
+                    "18:55:17",
+                    "16:48:05",
+                    "22:20:24"
+                ]
+            }
+        },
+        {
+            "business_id": "-0Hj1hb_XW6ybWq2M7QhGA",
+            "checkin_times": {
+                "dates": [
+                    "2011-04-23",
+                    "2014-05-04",
+                    "2014-05-11",
+                    "2014-06-04",
+                    "2015-12-05",
+                    "2017-05-15"
+                ],
+                "times": [
+                    "21:11:22",
+                    "19:42:48",
+                    "19:16:08",
+                    "19:14:18",
+                    "19:22:42",
+                    "23:19:00"
+                ]
+            }
+        },
+        {
+            "business_id": "-0KMvRFwDWdVBeTpT11iHw",
+            "checkin_times": {
+                "dates": [
+                    "2012-07-13",
+                    "2016-12-24",
+                    "2017-08-31"
+                ],
+                "times": [
+                    "21:43:57",
+                    "02:27:31",
+                    "00:35:26"
+                ]
+            }
+        },
+        {
+            "business_id": "-0LPtgJC31FWMrMv317p0Q",
+            "checkin_times": {
+                "dates": [
+                    "2013-04-13",
+                    "2013-08-19",
+                    "2013-10-04"
+                ],
+                "times": [
+                    "12:35:33",
+                    "23:35:49",
+                    "19:14:56"
+                ]
+            }
+        },
+        {
+            "business_id": "-0M3o2uWBnQZwd3hmfEwuw",
+            "checkin_times": {
+                "dates": [
+                    "2016-09-10",
+                    "2018-09-08",
+                    "2019-09-13"
+                ],
+                "times": [
+                    "19:26:19",
+                    "14:15:37",
+                    "22:47:25"
+                ]
+            }
+        },
+        {
+            "business_id": "-0RRiWDtfnS16AKCtfvBZg",
+            "checkin_times": {
+                "dates": [
+                    "2017-05-19",
+                    "2017-05-19",
+                    "2017-08-28",
+                    "2017-09-20",
+                    "2017-10-01",
+                    "2017-10-01",
+                    "2017-12-27"
+                ],
+                "times": [
+                    "14:30:16",
+                    "14:30:25",
+                    "15:49:37",
+                    "20:19:51",
+                    "16:31:05",
+                    "16:56:27",
+                    "23:33:20"
+                ]
+            }
+        },
+        {
+            "business_id": "-0Soj75v-XoRcf2ERr8Bmg",
+            "checkin_times": {
+                "dates": [
+                    "2019-06-05"
+                ],
+                "times": [
+                    "18:22:49"
+                ]
+            }
+        },
+        {
+            "business_id": "-0ZumLlFjMh4ZW1z2nXGug",
+            "checkin_times": {
+                "dates": [
+                    "2011-09-24",
+                    "2014-03-10",
+                    "2015-05-27",
+                    "2015-08-29",
+                    "2018-03-16"
+                ],
+                "times": [
+                    "21:37:32",
+                    "20:20:07",
+                    "00:40:24",
+                    "17:58:15",
+                    "15:03:26"
+                ]
+            }
+        },
+        {
+            "business_id": "-0aOudcaAyac0VJbMX-L1g",
+            "checkin_times": {
+                "dates": [
+                    "2015-03-16",
+                    "2015-12-21",
+                    "2016-10-28",
+                    "2016-10-28"
+                ],
+                "times": [
+                    "23:51:16",
+                    "04:48:01",
+                    "20:22:42",
+                    "20:23:00"
+                ]
+            }
+        },
+        {
+            "business_id": "-0b86isaXMY0v4g-V8GZ9Q",
+            "checkin_times": {
+                "dates": [
+                    "2013-10-22",
+                    "2014-11-21"
+                ],
+                "times": [
+                    "16:49:21",
+                    "17:39:24"
+                ]
+            }
+        },
+        {
+            "business_id": "-0d-BfFSU0bwLcnMaGRxYw",
+            "checkin_times": {
+                "dates": [
+                    "2014-08-07",
+                    "2014-09-16",
+                    "2014-10-12",
+                    "2015-07-21",
+                    "2015-07-21"
+                ],
+                "times": [
+                    "18:30:48",
+                    "20:41:45",
+                    "23:22:27",
+                    "20:43:56",
+                    "20:45:07"
+                ]
+            }
+        },
+        {
+            "business_id": "-0jz6c3C6i7RG7Ag22K-Pg",
+            "checkin_times": {
+                "dates": [
+                    "2015-05-02",
+                    "2015-05-06",
+                    "2015-09-26"
+                ],
+                "times": [
+                    "19:49:05",
+                    "03:52:18",
+                    "01:13:19"
+                ]
+            }
+        },
+        {
+            "business_id": "-0y3MZU2oYP8r1ruDP1bfQ",
+            "checkin_times": {
+                "dates": [
+                    "2015-04-11",
+                    "2015-11-21",
+                    "2016-05-06",
+                    "2017-08-09",
+                    "2017-10-21"
+                ],
+                "times": [
+                    "13:14:14",
+                    "16:05:56",
+                    "14:10:04",
+                    "15:15:10",
+                    "15:12:56"
+                ]
+            }
+        },
+        {
+            "business_id": "-1BPe8UjF2_l3nVk-DFUjA",
+            "checkin_times": {
+                "dates": [
+                    "2015-12-03",
+                    "2016-03-17",
+                    "2016-11-02"
+                ],
+                "times": [
+                    "18:44:00",
+                    "18:19:21",
+                    "15:58:38"
+                ]
+            }
+        },
+        {
+            "business_id": "-1E2CQu_38mkghvmZgCCRw",
+            "checkin_times": {
+                "dates": [
+                    "2019-04-04"
+                ],
+                "times": [
+                    "22:02:37"
+                ]
+            }
+        },
+        {
+            "business_id": "-1wzk43IZ5D9Ysu6kzb5xA",
+            "checkin_times": {
+                "dates": [
+                    "2019-02-27"
+                ],
+                "times": [
+                    "14:03:08"
+                ]
+            }
+        },
+        {
+            "business_id": "-23R9P2eG7VTc6DVLjFKzA",
+            "checkin_times": {
+                "dates": [
+                    "2011-12-21",
+                    "2012-04-15",
+                    "2012-04-15",
+                    "2013-06-30",
+                    "2013-10-04",
+                    "2014-07-16"
+                ],
+                "times": [
+                    "19:02:51",
+                    "04:21:39",
+                    "14:23:56",
+                    "22:39:51",
+                    "20:34:13",
+                    "02:28:40"
+                ]
+            }
+        },
+        {
+            "business_id": "-26MGfikhJiTfCI-GqmzhQ",
+            "checkin_times": {
+                "dates": [
+                    "2018-06-13"
+                ],
+                "times": [
+                    "20:16:07"
+                ]
+            }
+        },
+        {
+            "business_id": "-2bLuJsMZ0WhI9daurVQNQ",
+            "checkin_times": {
+                "dates": [
+                    "2015-05-29",
+                    "2015-06-01"
+                ],
+                "times": [
+                    "16:46:17",
+                    "15:03:53"
+                ]
+            }
+        },
+        {
+            "business_id": "-2hDBMaza_ldqnZdiU06LQ",
+            "checkin_times": {
+                "dates": [
+                    "2011-10-08",
+                    "2014-08-18",
+                    "2016-01-07",
+                    "2016-10-21",
+                    "2016-12-01",
+                    "2016-12-29",
+                    "2018-07-22",
+                    "2018-09-07",
+                    "2019-03-08"
+                ],
+                "times": [
+                    "12:02:23",
+                    "02:11:11",
+                    "05:27:51",
+                    "20:15:55",
+                    "03:57:10",
+                    "01:54:42",
+                    "19:55:31",
+                    "01:42:54",
+                    "03:41:06"
+                ]
+            }
+        }
+    ]
+);
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/join-unnest-queries/use-case-2/use-case-2.3.update.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/join-unnest-queries/use-case-2/use-case-2.3.update.sqlpp
new file mode 100644
index 0000000..c1eca5d
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/join-unnest-queries/use-case-2/use-case-2.3.update.sqlpp
@@ -0,0 +1,132 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+USE TestYelp;
+
+INSERT INTO YelpCheckinDateMarkers (
+    [
+        { "marker": "4fa1e250-7a68-3adb-04ec-6569c36882f5", "datetime": "2016-04-26 19:49:16", "date": "2016-04-26", "time": "19:49:16" },
+        { "marker": "4fa1e250-7a68-3adb-04ec-6569c36882f6", "datetime": "2016-08-30 18:36:57", "date": "2016-08-30", "time": "18:36:57" },
+        { "marker": "4fa1e250-7a68-3adb-04ec-6569c36882f7", "datetime": "2016-10-15 02:45:18", "date": "2016-10-15", "time": "02:45:18" },
+        { "marker": "4fa1e250-7a68-3adb-04ec-6569c36882f8", "datetime": "2016-11-18 01:54:50", "date": "2016-11-18", "time": "01:54:50" },
+        { "marker": "4fa1e250-7a68-3adb-04ec-6569c36882f9", "datetime": "2017-04-20 18:39:06", "date": "2017-04-20", "time": "18:39:06" },
+        { "marker": "4fa1e250-7a68-3adb-04ec-6569c36882fa", "datetime": "2017-05-03 17:58:02", "date": "2017-05-03", "time": "17:58:02" },
+        { "marker": "4fa1e250-7a68-3adb-04ec-6569c36882fb", "datetime": "2019-03-19 22:04:48", "date": "2019-03-19", "time": "22:04:48" },
+        { "marker": "4fa1e250-7a68-3adb-04ec-6569c36882fc", "datetime": "2018-05-25 19:52:07", "date": "2018-05-25", "time": "19:52:07" },
+        { "marker": "4fa1e250-7a68-3adb-04ec-6569c36882fd", "datetime": "2018-09-18 16:09:44", "date": "2018-09-18", "time": "16:09:44" },
+        { "marker": "4fa1e250-7a68-3adb-04ec-6569c36882fe", "datetime": "2019-10-18 21:29:09", "date": "2019-10-18", "time": "21:29:09" },
+        { "marker": "4fa1e250-7a68-3adb-04ec-6569c36882ff", "datetime": "2011-05-03 20:54:05", "date": "2011-05-03", "time": "20:54:05" },
+        { "marker": "4fa1e250-7a68-3adb-04ec-6569c3688300", "datetime": "2011-08-23 20:49:45", "date": "2011-08-23", "time": "20:49:45" },
+        { "marker": "4fa1e250-7a68-3adb-04ec-6569c3688301", "datetime": "2014-12-04 06:13:01", "date": "2014-12-04", "time": "06:13:01" },
+        { "marker": "4fa1e250-7a68-3adb-04ec-6569c3688302", "datetime": "2016-11-16 19:25:55", "date": "2016-11-16", "time": "19:25:55" },
+        { "marker": "4fa1e250-7a68-3adb-04ec-6569c3688303", "datetime": "2016-06-18 21:35:45", "date": "2016-06-18", "time": "21:35:45" },
+        { "marker": "4fa1e250-7a68-3adb-04ec-6569c3688304", "datetime": "2016-10-15 18:17:51", "date": "2016-10-15", "time": "18:17:51" },
+        { "marker": "4fa1e250-7a68-3adb-04ec-6569c3688305", "datetime": "2014-08-27 17:49:18", "date": "2014-08-27", "time": "17:49:18" },
+        { "marker": "4fa1e250-7a68-3adb-04ec-6569c3688306", "datetime": "2015-12-19 21:30:31", "date": "2015-12-19", "time": "21:30:31" },
+        { "marker": "4fa1e250-7a68-3adb-04ec-6569c3688307", "datetime": "2018-11-27 15:53:50", "date": "2018-11-27", "time": "15:53:50" },
+        { "marker": "4fa1e250-7a68-3adb-04ec-6569c3688308", "datetime": "2019-04-11 18:30:12", "date": "2019-04-11", "time": "18:30:12" },
+        { "marker": "4fa1e250-7a68-3adb-04ec-6569c3688309", "datetime": "2015-06-06 20:01:06", "date": "2015-06-06", "time": "20:01:06" },
+        { "marker": "4fa1e250-7a68-3adb-04ec-6569c368830a", "datetime": "2019-03-14 22:01:52", "date": "2019-03-14", "time": "22:01:52" },
+        { "marker": "4fa1e250-7a68-3adb-04ec-6569c368830b", "datetime": "2012-07-13 21:43:57", "date": "2012-07-13", "time": "21:43:57" },
+        { "marker": "4fa1e250-7a68-3adb-04ec-6569c368830c", "datetime": "2016-12-24 02:27:31", "date": "2016-12-24", "time": "02:27:31" },
+        { "marker": "4fa1e250-7a68-3adb-04ec-6569c368830d", "datetime": "2017-08-31 00:35:26", "date": "2017-08-31", "time": "00:35:26" },
+        { "marker": "4fa1e250-7a68-3adb-04ec-6569c368830e", "datetime": "2013-04-13 12:35:33", "date": "2013-04-13", "time": "12:35:33" },
+        { "marker": "4fa1e250-7a68-3adb-04ec-6569c368830f", "datetime": "2013-08-19 23:35:49", "date": "2013-08-19", "time": "23:35:49" },
+        { "marker": "4fa1e250-7a68-3adb-04ec-6569c3688310", "datetime": "2013-10-04 19:14:56", "date": "2013-10-04", "time": "19:14:56" },
+        { "marker": "4fa1e250-7a68-3adb-04ec-6569c3688311", "datetime": "2016-09-10 19:26:19", "date": "2016-09-10", "time": "19:26:19" },
+        { "marker": "4fa1e250-7a68-3adb-04ec-6569c3688312", "datetime": "2018-09-08 14:15:37", "date": "2018-09-08", "time": "14:15:37" },
+        { "marker": "4fa1e250-7a68-3adb-04ec-6569c3688313", "datetime": "2019-09-13 22:47:25", "date": "2019-09-13", "time": "22:47:25" },
+        { "marker": "4fa1e250-7a68-3adb-04ec-6569c3688314", "datetime": "2019-06-05 18:22:49", "date": "2019-06-05", "time": "18:22:49" },
+        { "marker": "4fa1e250-7a68-3adb-04ec-6569c3688315", "datetime": "2015-03-16 23:51:16", "date": "2015-03-16", "time": "23:51:16" },
+        { "marker": "4fa1e250-7a68-3adb-04ec-6569c3688316", "datetime": "2015-12-21 04:48:01", "date": "2015-12-21", "time": "04:48:01" },
+        { "marker": "4fa1e250-7a68-3adb-04ec-6569c3688317", "datetime": "2016-10-28 20:22:42", "date": "2016-10-28", "time": "20:22:42" },
+        { "marker": "4fa1e250-7a68-3adb-04ec-6569c3688318", "datetime": "2016-10-28 20:23:00", "date": "2016-10-28", "time": "20:23:00" },
+        { "marker": "4fa1e250-7a68-3adb-04ec-6569c3688319", "datetime": "2014-08-07 18:30:48", "date": "2014-08-07", "time": "18:30:48" },
+        { "marker": "4fa1e250-7a68-3adb-04ec-6569c368831a", "datetime": "2014-09-16 20:41:45", "date": "2014-09-16", "time": "20:41:45" },
+        { "marker": "4fa1e250-7a68-3adb-04ec-6569c368831b", "datetime": "2014-10-12 23:22:27", "date": "2014-10-12", "time": "23:22:27" },
+        { "marker": "4fa1e250-7a68-3adb-04ec-6569c368831c", "datetime": "2015-07-21 20:43:56", "date": "2015-07-21", "time": "20:43:56" },
+        { "marker": "4fa1e250-7a68-3adb-04ec-6569c368831d", "datetime": "2015-07-21 20:45:07", "date": "2015-07-21", "time": "20:45:07" },
+        { "marker": "4fa1e250-7a68-3adb-04ec-6569c368831e", "datetime": "2015-05-02 19:49:05", "date": "2015-05-02", "time": "19:49:05" },
+        { "marker": "4fa1e250-7a68-3adb-04ec-6569c368831f", "datetime": "2015-05-06 03:52:18", "date": "2015-05-06", "time": "03:52:18" },
+        { "marker": "4fa1e250-7a68-3adb-04ec-6569c3688320", "datetime": "2015-09-26 01:13:19", "date": "2015-09-26", "time": "01:13:19" },
+        { "marker": "4fa1e250-7a68-3adb-04ec-6569c3688321", "datetime": "2019-04-04 22:02:37", "date": "2019-04-04", "time": "22:02:37" },
+        { "marker": "4fa1e250-7a68-3adb-04ec-6569c3688322", "datetime": "2011-12-21 19:02:51", "date": "2011-12-21", "time": "19:02:51" },
+        { "marker": "4fa1e250-7a68-3adb-04ec-6569c3688323", "datetime": "2012-04-15 04:21:39", "date": "2012-04-15", "time": "04:21:39" },
+        { "marker": "4fa1e250-7a68-3adb-04ec-6569c3688324", "datetime": "2012-04-15 14:23:56", "date": "2012-04-15", "time": "14:23:56" },
+        { "marker": "4fa1e250-7a68-3adb-04ec-6569c3688325", "datetime": "2013-06-30 22:39:51", "date": "2013-06-30", "time": "22:39:51" },
+        { "marker": "4fa1e250-7a68-3adb-04ec-6569c3688326", "datetime": "2013-10-04 20:34:13", "date": "2013-10-04", "time": "20:34:13" },
+        { "marker": "4fa1e250-7a68-3adb-04ec-6569c3688327", "datetime": "2014-07-16 02:28:40", "date": "2014-07-16", "time": "02:28:40" },
+        { "marker": "4fa1e250-7a68-3adb-04ec-6569c3688328", "datetime": "2018-06-13 20:16:07", "date": "2018-06-13", "time": "20:16:07" },
+        { "marker": "4fa1e250-7a68-3adb-04ec-6569c3688329", "datetime": "2011-10-08 12:02:23", "date": "2011-10-08", "time": "12:02:23" },
+        { "marker": "4fa1e250-7a68-3adb-04ec-6569c368832a", "datetime": "2014-08-18 02:11:11", "date": "2014-08-18", "time": "02:11:11" },
+        { "marker": "4fa1e250-7a68-3adb-04ec-6569c368832b", "datetime": "2016-01-07 05:27:51", "date": "2016-01-07", "time": "05:27:51" },
+        { "marker": "4fa1e250-7a68-3adb-04ec-6569c368832c", "datetime": "2016-10-21 20:15:55", "date": "2016-10-21", "time": "20:15:55" },
+        { "marker": "4fa1e250-7a68-3adb-04ec-6569c368832d", "datetime": "2016-12-01 03:57:10", "date": "2016-12-01", "time": "03:57:10" },
+        { "marker": "4fa1e250-7a68-3adb-04ec-6569c368832e", "datetime": "2016-12-29 01:54:42", "date": "2016-12-29", "time": "01:54:42" },
+        { "marker": "4fa1e250-7a68-3adb-04ec-6569c368832f", "datetime": "2018-07-22 19:55:31", "date": "2018-07-22", "time": "19:55:31" },
+        { "marker": "4fa1e250-7a68-3adb-04ec-6569c3688330", "datetime": "2018-09-07 01:42:54", "date": "2018-09-07", "time": "01:42:54" },
+        { "marker": "4fa1e250-7a68-3adb-04ec-6569c3688331", "datetime": "2019-03-08 03:41:06", "date": "2019-03-08", "time": "03:41:06" },
+        { "marker": "991ed2f7-3374-33a5-ee65-77f067848381", "datetime": "2019-06-07 17:54:58", "date": "2019-06-07", "time": "17:54:58" },
+        { "marker": "991ed2f7-3374-33a5-ee65-77f067848382", "datetime": "2015-04-02 21:45:17", "date": "2015-04-02", "time": "21:45:17" },
+        { "marker": "991ed2f7-3374-33a5-ee65-77f067848383", "datetime": "2014-01-28 20:56:04", "date": "2014-01-28", "time": "20:56:04" },
+        { "marker": "991ed2f7-3374-33a5-ee65-77f067848384", "datetime": "2014-11-16 16:11:58", "date": "2014-11-16", "time": "16:11:58" },
+        { "marker": "991ed2f7-3374-33a5-ee65-77f067848385", "datetime": "2015-11-15 19:21:53", "date": "2015-11-15", "time": "19:21:53" },
+        { "marker": "991ed2f7-3374-33a5-ee65-77f067848386", "datetime": "2015-11-15 19:33:39", "date": "2015-11-15", "time": "19:33:39" },
+        { "marker": "991ed2f7-3374-33a5-ee65-77f067848387", "datetime": "2018-09-29 18:55:17", "date": "2018-09-29", "time": "18:55:17" },
+        { "marker": "991ed2f7-3374-33a5-ee65-77f067848388", "datetime": "2018-10-20 16:48:05", "date": "2018-10-20", "time": "16:48:05" },
+        { "marker": "991ed2f7-3374-33a5-ee65-77f067848389", "datetime": "2018-10-20 22:20:24", "date": "2018-10-20", "time": "22:20:24" },
+        { "marker": "991ed2f7-3374-33a5-ee65-77f06784838a", "datetime": "2011-04-23 21:11:22", "date": "2011-04-23", "time": "21:11:22" },
+        { "marker": "991ed2f7-3374-33a5-ee65-77f06784838b", "datetime": "2014-05-04 19:42:48", "date": "2014-05-04", "time": "19:42:48" },
+        { "marker": "991ed2f7-3374-33a5-ee65-77f06784838c", "datetime": "2014-05-11 19:16:08", "date": "2014-05-11", "time": "19:16:08" },
+        { "marker": "991ed2f7-3374-33a5-ee65-77f06784838d", "datetime": "2014-06-04 19:14:18", "date": "2014-06-04", "time": "19:14:18" },
+        { "marker": "991ed2f7-3374-33a5-ee65-77f06784838e", "datetime": "2015-12-05 19:22:42", "date": "2015-12-05", "time": "19:22:42" },
+        { "marker": "991ed2f7-3374-33a5-ee65-77f06784838f", "datetime": "2017-05-15 23:19:00", "date": "2017-05-15", "time": "23:19:00" },
+        { "marker": "991ed2f7-3374-33a5-ee65-77f067848390", "datetime": "2017-05-19 14:30:16", "date": "2017-05-19", "time": "14:30:16" },
+        { "marker": "991ed2f7-3374-33a5-ee65-77f067848391", "datetime": "2017-05-19 14:30:25", "date": "2017-05-19", "time": "14:30:25" },
+        { "marker": "991ed2f7-3374-33a5-ee65-77f067848392", "datetime": "2017-08-28 15:49:37", "date": "2017-08-28", "time": "15:49:37" },
+        { "marker": "991ed2f7-3374-33a5-ee65-77f067848393", "datetime": "2017-09-20 20:19:51", "date": "2017-09-20", "time": "20:19:51" },
+        { "marker": "991ed2f7-3374-33a5-ee65-77f067848394", "datetime": "2017-10-01 16:31:05", "date": "2017-10-01", "time": "16:31:05" },
+        { "marker": "991ed2f7-3374-33a5-ee65-77f067848395", "datetime": "2017-10-01 16:56:27", "date": "2017-10-01", "time": "16:56:27" },
+        { "marker": "991ed2f7-3374-33a5-ee65-77f067848396", "datetime": "2017-12-27 23:33:20", "date": "2017-12-27", "time": "23:33:20" },
+        { "marker": "991ed2f7-3374-33a5-ee65-77f067848397", "datetime": "2011-09-24 21:37:32", "date": "2011-09-24", "time": "21:37:32" },
+        { "marker": "991ed2f7-3374-33a5-ee65-77f067848398", "datetime": "2014-03-10 20:20:07", "date": "2014-03-10", "time": "20:20:07" },
+        { "marker": "991ed2f7-3374-33a5-ee65-77f067848399", "datetime": "2015-05-27 00:40:24", "date": "2015-05-27", "time": "00:40:24" },
+        { "marker": "991ed2f7-3374-33a5-ee65-77f06784839a", "datetime": "2015-08-29 17:58:15", "date": "2015-08-29", "time": "17:58:15" },
+        { "marker": "991ed2f7-3374-33a5-ee65-77f06784839b", "datetime": "2018-03-16 15:03:26", "date": "2018-03-16", "time": "15:03:26" },
+        { "marker": "991ed2f7-3374-33a5-ee65-77f06784839c", "datetime": "2013-10-22 16:49:21", "date": "2013-10-22", "time": "16:49:21" },
+        { "marker": "991ed2f7-3374-33a5-ee65-77f06784839d", "datetime": "2014-11-21 17:39:24", "date": "2014-11-21", "time": "17:39:24" },
+        { "marker": "991ed2f7-3374-33a5-ee65-77f06784839e", "datetime": "2015-04-11 13:14:14", "date": "2015-04-11", "time": "13:14:14" },
+        { "marker": "991ed2f7-3374-33a5-ee65-77f06784839f", "datetime": "2015-11-21 16:05:56", "date": "2015-11-21", "time": "16:05:56" },
+        { "marker": "991ed2f7-3374-33a5-ee65-77f0678483a0", "datetime": "2016-05-06 14:10:04", "date": "2016-05-06", "time": "14:10:04" },
+        { "marker": "991ed2f7-3374-33a5-ee65-77f0678483a1", "datetime": "2017-08-09 15:15:10", "date": "2017-08-09", "time": "15:15:10" },
+        { "marker": "991ed2f7-3374-33a5-ee65-77f0678483a2", "datetime": "2017-10-21 15:12:56", "date": "2017-10-21", "time": "15:12:56" },
+        { "marker": "991ed2f7-3374-33a5-ee65-77f0678483a3", "datetime": "2015-12-03 18:44:00", "date": "2015-12-03", "time": "18:44:00" },
+        { "marker": "991ed2f7-3374-33a5-ee65-77f0678483a4", "datetime": "2016-03-17 18:19:21", "date": "2016-03-17", "time": "18:19:21" },
+        { "marker": "991ed2f7-3374-33a5-ee65-77f0678483a5", "datetime": "2016-11-02 15:58:38", "date": "2016-11-02", "time": "15:58:38" },
+        { "marker": "991ed2f7-3374-33a5-ee65-77f0678483a6", "datetime": "2019-02-27 14:03:08", "date": "2019-02-27", "time": "14:03:08" },
+        { "marker": "991ed2f7-3374-33a5-ee65-77f0678483a7", "datetime": "2015-05-29 16:46:17", "date": "2015-05-29", "time": "16:46:17" },
+        { "marker": "991ed2f7-3374-33a5-ee65-77f0678483a8", "datetime": "2015-06-01 15:03:53", "date": "2015-06-01", "time": "15:03:53" },
+        { "marker": "BAD_ENTRY_1", "datetime": "NA", "date": "NA", "time": "NA" },
+        { "marker": "BAD_ENTRY_2", "datetime": "NA", "date": "NA", "time": "NA" },
+        { "marker": "BAD_ENTRY_3", "datetime": "NA", "date": "NA", "time": "NA" },
+        { "marker": "BAD_ENTRY_4", "datetime": "NA", "date": "NA", "time": "NA" },
+        { "marker": "BAD_ENTRY_5", "datetime": "NA", "date": "NA", "time": "NA" },
+        { "marker": "BAD_ENTRY_6", "datetime": "NA", "date": "NA", "time": "NA" }
+    ]
+);
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/join-unnest-queries/use-case-2/use-case-2.4.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/join-unnest-queries/use-case-2/use-case-2.4.ddl.sqlpp
new file mode 100644
index 0000000..0ab72fe
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/join-unnest-queries/use-case-2/use-case-2.4.ddl.sqlpp
@@ -0,0 +1,22 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+USE TestYelp;
+
+CREATE INDEX IdxYelpCheckinDates ON YelpCheckin (UNNEST checkin_times.dates);
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/join-unnest-queries/use-case-2/use-case-2.5.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/join-unnest-queries/use-case-2/use-case-2.5.query.sqlpp
new file mode 100644
index 0000000..93777ba
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/join-unnest-queries/use-case-2/use-case-2.5.query.sqlpp
@@ -0,0 +1,26 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+SET `compiler.arrayindex` "true";
+
+USE TestYelp;
+
+SELECT M.marker
+FROM YelpCheckinDateMarkers M, YelpCheckin C, C.checkin_times.dates D
+WHERE D /*+ indexnl */ = M.date
+      AND C.business_id = "--Ni3oJ4VOqfOEu7Sj2Vzg";
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/join-unnest-queries/use-case-2/use-case-2.6.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/join-unnest-queries/use-case-2/use-case-2.6.query.sqlpp
new file mode 100644
index 0000000..dba4085
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/join-unnest-queries/use-case-2/use-case-2.6.query.sqlpp
@@ -0,0 +1,26 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+SET `compiler.arrayindex` "true";
+
+USE TestYelp;
+
+SELECT M.marker
+FROM YelpCheckinDateMarkers M, YelpCheckin C, C.checkin_times.dates D
+WHERE D /*+ indexnl */ = M.date
+      AND M.time = "19:49:16";
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/join-unnest-queries/use-case-2/use-case-2.7.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/join-unnest-queries/use-case-2/use-case-2.7.query.sqlpp
new file mode 100644
index 0000000..c5598d4
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/join-unnest-queries/use-case-2/use-case-2.7.query.sqlpp
@@ -0,0 +1,28 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+SET `compiler.arrayindex` "true";
+
+USE TestYelp;
+
+SELECT COUNT(*)
+FROM YelpCheckinDateMarkers M
+INNER JOIN (
+    SELECT VALUE D
+    FROM YelpCheckin C, C.checkin_times.dates D
+) AS CD ON CD /*+ indexnl */ = M.date;
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/join-unnest-queries/use-case-3/use-case-3.1.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/join-unnest-queries/use-case-3/use-case-3.1.ddl.sqlpp
new file mode 100644
index 0000000..fed0346
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/join-unnest-queries/use-case-3/use-case-3.1.ddl.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.
+ */
+
+/*
+ * Description: Verify that we are able to correctly perform the following types of queries using an array->record->atomic index.
+ *     1) INNER JOIN query, with additional predicate on inner collection before the unnest.
+ *     2) INNER JOIN query, with additional predicate on inner collection after the unnest.
+ *     3) INNER JOIN query, with additional predicate on the outer collection.
+ *     4) Explicit INNER JOIN query using UNNEST subquery, no additional predicate.
+ *     TODO (GLENN) 5) Explicit LEFT OUTER JOIN query using UNNEST subquery, no additional predicate.
+ */
+
+DROP DATAVERSE TestYelp IF EXISTS;
+CREATE DATAVERSE TestYelp;
+USE TestYelp;
+
+CREATE TYPE CheckinType AS {
+    checkin_id: uuid,
+    business_id: string,
+    checkin_times: [{
+        date: string,
+        time: string
+    }]
+};
+CREATE TYPE CheckinDatesToMarkersType AS {
+    marker: string,
+    datetime: string,
+    date: string,
+    time: string
+};
+
+CREATE DATASET YelpCheckin(CheckinType) PRIMARY KEY checkin_id AUTOGENERATED;
+CREATE DATASET YelpCheckinDateMarkers (CheckinDatesToMarkersType) PRIMARY KEY marker;
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/join-unnest-queries/use-case-3/use-case-3.2.update.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/join-unnest-queries/use-case-3/use-case-3.2.update.sqlpp
new file mode 100644
index 0000000..02ddf21
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/join-unnest-queries/use-case-3/use-case-3.2.update.sqlpp
@@ -0,0 +1,276 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+USE TestYelp;
+
+INSERT INTO YelpCheckin (
+    [
+        {
+            "business_id": "--1UhMGODdWsrMastO9DZw",
+            "checkin_times": [ 
+                {"date": "2016-04-26", "time": "19:49:16"},
+                {"date": "2016-08-30", "time": "18:36:57"},
+                {"date": "2016-10-15", "time": "02:45:18"},
+                {"date": "2016-11-18", "time": "01:54:50"},
+                {"date": "2017-04-20", "time": "18:39:06"},
+                {"date": "2017-05-03", "time": "17:58:02"},
+                {"date": "2019-03-19", "time": "22:04:48"}
+             ]
+        },
+        {
+            "business_id": "--EF5N7P70J_UYBTPypYlA",
+            "checkin_times": [ 
+                {"date": "2018-05-25", "time": "19:52:07"},
+                {"date": "2018-09-18", "time": "16:09:44"},
+                {"date": "2019-10-18", "time": "21:29:09"}
+             ]
+        },
+        {
+            "business_id": "--Ni3oJ4VOqfOEu7Sj2Vzg",
+            "checkin_times": [ 
+                {"date": "2019-06-07", "time": "17:54:58"}
+             ]
+        },
+        {
+            "business_id": "--Y1Adl1YUWfYIRSd8vkmA",
+            "checkin_times": [ 
+                {"date": "2011-05-03", "time": "20:54:05"},
+                {"date": "2011-08-23", "time": "20:49:45"},
+                {"date": "2014-12-04", "time": "06:13:01"},
+                {"date": "2016-11-16", "time": "19:25:55"}
+             ]
+        },
+        {
+            "business_id": "--YPwqIlRJrhHkJcjY3eiA",
+            "checkin_times": [ 
+                {"date": "2016-06-18", "time": "21:35:45"},
+                {"date": "2016-10-15", "time": "18:17:51"}
+             ]
+        },
+        {
+            "business_id": "--e8PjCNhEz32pprnPhCwQ",
+            "checkin_times": [ 
+                {"date": "2015-04-02", "time": "21:45:17"}
+             ]
+        },
+        {
+            "business_id": "--kinfHwmtdjz03g8B8z8Q",
+            "checkin_times": [ 
+                {"date": "2014-08-27", "time": "17:49:18"},
+                {"date": "2015-12-19", "time": "21:30:31"},
+                {"date": "2018-11-27", "time": "15:53:50"}
+             ]
+        },
+        {
+            "business_id": "--q6datkI-f0EoVheXNEeQ",
+            "checkin_times": [ 
+                {"date": "2014-01-28", "time": "20:56:04"},
+                {"date": "2014-11-16", "time": "16:11:58"},
+                {"date": "2015-11-15", "time": "19:21:53"},
+                {"date": "2015-11-15", "time": "19:33:39"}
+             ]
+        },
+        {
+            "business_id": "--qvQS4MigHPykD2GV0-zw",
+            "checkin_times": [ 
+                {"date": "2019-04-11", "time": "18:30:12"}
+             ]
+        },
+        {
+            "business_id": "--wIGbLEhlpl_UeAIyDmZQ",
+            "checkin_times": [ 
+                {"date": "2015-06-06", "time": "20:01:06"},
+                {"date": "2019-03-14", "time": "22:01:52"}
+             ]
+        },
+        {
+            "business_id": "-0FA-Qdi3SPYIoJz9UQw-A",
+            "checkin_times": [ 
+                {"date": "2018-09-29", "time": "18:55:17"},
+                {"date": "2018-10-20", "time": "16:48:05"},
+                {"date": "2018-10-20", "time": "22:20:24"}
+             ]
+        },
+        {
+            "business_id": "-0Hj1hb_XW6ybWq2M7QhGA",
+            "checkin_times": [ 
+                {"date": "2011-04-23", "time": "21:11:22"},
+                {"date": "2014-05-04", "time": "19:42:48"},
+                {"date": "2014-05-11", "time": "19:16:08"},
+                {"date": "2014-06-04", "time": "19:14:18"},
+                {"date": "2015-12-05", "time": "19:22:42"},
+                {"date": "2017-05-15", "time": "23:19:00"}
+             ]
+        },
+        {
+            "business_id": "-0KMvRFwDWdVBeTpT11iHw",
+            "checkin_times": [ 
+                {"date": "2012-07-13", "time": "21:43:57"},
+                {"date": "2016-12-24", "time": "02:27:31"},
+                {"date": "2017-08-31", "time": "00:35:26"}
+             ]
+        },
+        {
+            "business_id": "-0LPtgJC31FWMrMv317p0Q",
+            "checkin_times": [ 
+                {"date": "2013-04-13", "time": "12:35:33"},
+                {"date": "2013-08-19", "time": "23:35:49"},
+                {"date": "2013-10-04", "time": "19:14:56"}
+             ]
+        },
+        {
+            "business_id": "-0M3o2uWBnQZwd3hmfEwuw",
+            "checkin_times": [ 
+                {"date": "2016-09-10", "time": "19:26:19"},
+                {"date": "2018-09-08", "time": "14:15:37"},
+                {"date": "2019-09-13", "time": "22:47:25"}
+             ]
+        },
+        {
+            "business_id": "-0RRiWDtfnS16AKCtfvBZg",
+            "checkin_times": [ 
+                {"date": "2017-05-19", "time": "14:30:16"},
+                {"date": "2017-05-19", "time": "14:30:25"},
+                {"date": "2017-08-28", "time": "15:49:37"},
+                {"date": "2017-09-20", "time": "20:19:51"},
+                {"date": "2017-10-01", "time": "16:31:05"},
+                {"date": "2017-10-01", "time": "16:56:27"},
+                {"date": "2017-12-27", "time": "23:33:20"}
+             ]
+        },
+        {
+            "business_id": "-0Soj75v-XoRcf2ERr8Bmg",
+            "checkin_times": [ 
+                {"date": "2019-06-05", "time": "18:22:49"}
+             ]
+        },
+        {
+            "business_id": "-0ZumLlFjMh4ZW1z2nXGug",
+            "checkin_times": [ 
+                {"date": "2011-09-24", "time": "21:37:32"},
+                {"date": "2014-03-10", "time": "20:20:07"},
+                {"date": "2015-05-27", "time": "00:40:24"},
+                {"date": "2015-08-29", "time": "17:58:15"},
+                {"date": "2018-03-16", "time": "15:03:26"}
+             ]
+        },
+        {
+            "business_id": "-0aOudcaAyac0VJbMX-L1g",
+            "checkin_times": [ 
+                {"date": "2015-03-16", "time": "23:51:16"},
+                {"date": "2015-12-21", "time": "04:48:01"},
+                {"date": "2016-10-28", "time": "20:22:42"},
+                {"date": "2016-10-28", "time": "20:23:00"}
+             ]
+        },
+        {
+            "business_id": "-0b86isaXMY0v4g-V8GZ9Q",
+            "checkin_times": [ 
+                {"date": "2013-10-22", "time": "16:49:21"},
+                {"date": "2014-11-21", "time": "17:39:24"}
+             ]
+        },
+        {
+            "business_id": "-0d-BfFSU0bwLcnMaGRxYw",
+            "checkin_times": [ 
+                {"date": "2014-08-07", "time": "18:30:48"},
+                {"date": "2014-09-16", "time": "20:41:45"},
+                {"date": "2014-10-12", "time": "23:22:27"},
+                {"date": "2015-07-21", "time": "20:43:56"},
+                {"date": "2015-07-21", "time": "20:45:07"}
+             ]
+        },
+        {
+            "business_id": "-0jz6c3C6i7RG7Ag22K-Pg",
+            "checkin_times": [ 
+                {"date": "2015-05-02", "time": "19:49:05"},
+                {"date": "2015-05-06", "time": "03:52:18"},
+                {"date": "2015-09-26", "time": "01:13:19"}
+             ]
+        },
+        {
+            "business_id": "-0y3MZU2oYP8r1ruDP1bfQ",
+            "checkin_times": [ 
+                {"date": "2015-04-11", "time": "13:14:14"},
+                {"date": "2015-11-21", "time": "16:05:56"},
+                {"date": "2016-05-06", "time": "14:10:04"},
+                {"date": "2017-08-09", "time": "15:15:10"},
+                {"date": "2017-10-21", "time": "15:12:56"}
+             ]
+        },
+        {
+            "business_id": "-1BPe8UjF2_l3nVk-DFUjA",
+            "checkin_times": [ 
+                {"date": "2015-12-03", "time": "18:44:00"},
+                {"date": "2016-03-17", "time": "18:19:21"},
+                {"date": "2016-11-02", "time": "15:58:38"}
+             ]
+        },
+        {
+            "business_id": "-1E2CQu_38mkghvmZgCCRw",
+            "checkin_times": [ 
+                {"date": "2019-04-04", "time": "22:02:37"}
+             ]
+        },
+        {
+            "business_id": "-1wzk43IZ5D9Ysu6kzb5xA",
+            "checkin_times": [ 
+                {"date": "2019-02-27", "time": "14:03:08"}
+             ]
+        },
+        {
+            "business_id": "-23R9P2eG7VTc6DVLjFKzA",
+            "checkin_times": [ 
+                {"date": "2011-12-21", "time": "19:02:51"},
+                {"date": "2012-04-15", "time": "04:21:39"},
+                {"date": "2012-04-15", "time": "14:23:56"},
+                {"date": "2013-06-30", "time": "22:39:51"},
+                {"date": "2013-10-04", "time": "20:34:13"},
+                {"date": "2014-07-16", "time": "02:28:40"}
+             ]
+        },
+        {
+            "business_id": "-26MGfikhJiTfCI-GqmzhQ",
+            "checkin_times": [ 
+                {"date": "2018-06-13", "time": "20:16:07"}
+             ]
+        },
+        {
+            "business_id": "-2bLuJsMZ0WhI9daurVQNQ",
+            "checkin_times": [ 
+                {"date": "2015-05-29", "time": "16:46:17"},
+                {"date": "2015-06-01", "time": "15:03:53"}
+             ]
+        },
+        {
+            "business_id": "-2hDBMaza_ldqnZdiU06LQ",
+            "checkin_times": [ 
+                {"date": "2011-10-08", "time": "12:02:23"},
+                {"date": "2014-08-18", "time": "02:11:11"},
+                {"date": "2016-01-07", "time": "05:27:51"},
+                {"date": "2016-10-21", "time": "20:15:55"},
+                {"date": "2016-12-01", "time": "03:57:10"},
+                {"date": "2016-12-29", "time": "01:54:42"},
+                {"date": "2018-07-22", "time": "19:55:31"},
+                {"date": "2018-09-07", "time": "01:42:54"},
+                {"date": "2019-03-08", "time": "03:41:06"}
+             ]
+        }
+    ]
+);
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/join-unnest-queries/use-case-3/use-case-3.3.update.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/join-unnest-queries/use-case-3/use-case-3.3.update.sqlpp
new file mode 100644
index 0000000..c1eca5d
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/join-unnest-queries/use-case-3/use-case-3.3.update.sqlpp
@@ -0,0 +1,132 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+USE TestYelp;
+
+INSERT INTO YelpCheckinDateMarkers (
+    [
+        { "marker": "4fa1e250-7a68-3adb-04ec-6569c36882f5", "datetime": "2016-04-26 19:49:16", "date": "2016-04-26", "time": "19:49:16" },
+        { "marker": "4fa1e250-7a68-3adb-04ec-6569c36882f6", "datetime": "2016-08-30 18:36:57", "date": "2016-08-30", "time": "18:36:57" },
+        { "marker": "4fa1e250-7a68-3adb-04ec-6569c36882f7", "datetime": "2016-10-15 02:45:18", "date": "2016-10-15", "time": "02:45:18" },
+        { "marker": "4fa1e250-7a68-3adb-04ec-6569c36882f8", "datetime": "2016-11-18 01:54:50", "date": "2016-11-18", "time": "01:54:50" },
+        { "marker": "4fa1e250-7a68-3adb-04ec-6569c36882f9", "datetime": "2017-04-20 18:39:06", "date": "2017-04-20", "time": "18:39:06" },
+        { "marker": "4fa1e250-7a68-3adb-04ec-6569c36882fa", "datetime": "2017-05-03 17:58:02", "date": "2017-05-03", "time": "17:58:02" },
+        { "marker": "4fa1e250-7a68-3adb-04ec-6569c36882fb", "datetime": "2019-03-19 22:04:48", "date": "2019-03-19", "time": "22:04:48" },
+        { "marker": "4fa1e250-7a68-3adb-04ec-6569c36882fc", "datetime": "2018-05-25 19:52:07", "date": "2018-05-25", "time": "19:52:07" },
+        { "marker": "4fa1e250-7a68-3adb-04ec-6569c36882fd", "datetime": "2018-09-18 16:09:44", "date": "2018-09-18", "time": "16:09:44" },
+        { "marker": "4fa1e250-7a68-3adb-04ec-6569c36882fe", "datetime": "2019-10-18 21:29:09", "date": "2019-10-18", "time": "21:29:09" },
+        { "marker": "4fa1e250-7a68-3adb-04ec-6569c36882ff", "datetime": "2011-05-03 20:54:05", "date": "2011-05-03", "time": "20:54:05" },
+        { "marker": "4fa1e250-7a68-3adb-04ec-6569c3688300", "datetime": "2011-08-23 20:49:45", "date": "2011-08-23", "time": "20:49:45" },
+        { "marker": "4fa1e250-7a68-3adb-04ec-6569c3688301", "datetime": "2014-12-04 06:13:01", "date": "2014-12-04", "time": "06:13:01" },
+        { "marker": "4fa1e250-7a68-3adb-04ec-6569c3688302", "datetime": "2016-11-16 19:25:55", "date": "2016-11-16", "time": "19:25:55" },
+        { "marker": "4fa1e250-7a68-3adb-04ec-6569c3688303", "datetime": "2016-06-18 21:35:45", "date": "2016-06-18", "time": "21:35:45" },
+        { "marker": "4fa1e250-7a68-3adb-04ec-6569c3688304", "datetime": "2016-10-15 18:17:51", "date": "2016-10-15", "time": "18:17:51" },
+        { "marker": "4fa1e250-7a68-3adb-04ec-6569c3688305", "datetime": "2014-08-27 17:49:18", "date": "2014-08-27", "time": "17:49:18" },
+        { "marker": "4fa1e250-7a68-3adb-04ec-6569c3688306", "datetime": "2015-12-19 21:30:31", "date": "2015-12-19", "time": "21:30:31" },
+        { "marker": "4fa1e250-7a68-3adb-04ec-6569c3688307", "datetime": "2018-11-27 15:53:50", "date": "2018-11-27", "time": "15:53:50" },
+        { "marker": "4fa1e250-7a68-3adb-04ec-6569c3688308", "datetime": "2019-04-11 18:30:12", "date": "2019-04-11", "time": "18:30:12" },
+        { "marker": "4fa1e250-7a68-3adb-04ec-6569c3688309", "datetime": "2015-06-06 20:01:06", "date": "2015-06-06", "time": "20:01:06" },
+        { "marker": "4fa1e250-7a68-3adb-04ec-6569c368830a", "datetime": "2019-03-14 22:01:52", "date": "2019-03-14", "time": "22:01:52" },
+        { "marker": "4fa1e250-7a68-3adb-04ec-6569c368830b", "datetime": "2012-07-13 21:43:57", "date": "2012-07-13", "time": "21:43:57" },
+        { "marker": "4fa1e250-7a68-3adb-04ec-6569c368830c", "datetime": "2016-12-24 02:27:31", "date": "2016-12-24", "time": "02:27:31" },
+        { "marker": "4fa1e250-7a68-3adb-04ec-6569c368830d", "datetime": "2017-08-31 00:35:26", "date": "2017-08-31", "time": "00:35:26" },
+        { "marker": "4fa1e250-7a68-3adb-04ec-6569c368830e", "datetime": "2013-04-13 12:35:33", "date": "2013-04-13", "time": "12:35:33" },
+        { "marker": "4fa1e250-7a68-3adb-04ec-6569c368830f", "datetime": "2013-08-19 23:35:49", "date": "2013-08-19", "time": "23:35:49" },
+        { "marker": "4fa1e250-7a68-3adb-04ec-6569c3688310", "datetime": "2013-10-04 19:14:56", "date": "2013-10-04", "time": "19:14:56" },
+        { "marker": "4fa1e250-7a68-3adb-04ec-6569c3688311", "datetime": "2016-09-10 19:26:19", "date": "2016-09-10", "time": "19:26:19" },
+        { "marker": "4fa1e250-7a68-3adb-04ec-6569c3688312", "datetime": "2018-09-08 14:15:37", "date": "2018-09-08", "time": "14:15:37" },
+        { "marker": "4fa1e250-7a68-3adb-04ec-6569c3688313", "datetime": "2019-09-13 22:47:25", "date": "2019-09-13", "time": "22:47:25" },
+        { "marker": "4fa1e250-7a68-3adb-04ec-6569c3688314", "datetime": "2019-06-05 18:22:49", "date": "2019-06-05", "time": "18:22:49" },
+        { "marker": "4fa1e250-7a68-3adb-04ec-6569c3688315", "datetime": "2015-03-16 23:51:16", "date": "2015-03-16", "time": "23:51:16" },
+        { "marker": "4fa1e250-7a68-3adb-04ec-6569c3688316", "datetime": "2015-12-21 04:48:01", "date": "2015-12-21", "time": "04:48:01" },
+        { "marker": "4fa1e250-7a68-3adb-04ec-6569c3688317", "datetime": "2016-10-28 20:22:42", "date": "2016-10-28", "time": "20:22:42" },
+        { "marker": "4fa1e250-7a68-3adb-04ec-6569c3688318", "datetime": "2016-10-28 20:23:00", "date": "2016-10-28", "time": "20:23:00" },
+        { "marker": "4fa1e250-7a68-3adb-04ec-6569c3688319", "datetime": "2014-08-07 18:30:48", "date": "2014-08-07", "time": "18:30:48" },
+        { "marker": "4fa1e250-7a68-3adb-04ec-6569c368831a", "datetime": "2014-09-16 20:41:45", "date": "2014-09-16", "time": "20:41:45" },
+        { "marker": "4fa1e250-7a68-3adb-04ec-6569c368831b", "datetime": "2014-10-12 23:22:27", "date": "2014-10-12", "time": "23:22:27" },
+        { "marker": "4fa1e250-7a68-3adb-04ec-6569c368831c", "datetime": "2015-07-21 20:43:56", "date": "2015-07-21", "time": "20:43:56" },
+        { "marker": "4fa1e250-7a68-3adb-04ec-6569c368831d", "datetime": "2015-07-21 20:45:07", "date": "2015-07-21", "time": "20:45:07" },
+        { "marker": "4fa1e250-7a68-3adb-04ec-6569c368831e", "datetime": "2015-05-02 19:49:05", "date": "2015-05-02", "time": "19:49:05" },
+        { "marker": "4fa1e250-7a68-3adb-04ec-6569c368831f", "datetime": "2015-05-06 03:52:18", "date": "2015-05-06", "time": "03:52:18" },
+        { "marker": "4fa1e250-7a68-3adb-04ec-6569c3688320", "datetime": "2015-09-26 01:13:19", "date": "2015-09-26", "time": "01:13:19" },
+        { "marker": "4fa1e250-7a68-3adb-04ec-6569c3688321", "datetime": "2019-04-04 22:02:37", "date": "2019-04-04", "time": "22:02:37" },
+        { "marker": "4fa1e250-7a68-3adb-04ec-6569c3688322", "datetime": "2011-12-21 19:02:51", "date": "2011-12-21", "time": "19:02:51" },
+        { "marker": "4fa1e250-7a68-3adb-04ec-6569c3688323", "datetime": "2012-04-15 04:21:39", "date": "2012-04-15", "time": "04:21:39" },
+        { "marker": "4fa1e250-7a68-3adb-04ec-6569c3688324", "datetime": "2012-04-15 14:23:56", "date": "2012-04-15", "time": "14:23:56" },
+        { "marker": "4fa1e250-7a68-3adb-04ec-6569c3688325", "datetime": "2013-06-30 22:39:51", "date": "2013-06-30", "time": "22:39:51" },
+        { "marker": "4fa1e250-7a68-3adb-04ec-6569c3688326", "datetime": "2013-10-04 20:34:13", "date": "2013-10-04", "time": "20:34:13" },
+        { "marker": "4fa1e250-7a68-3adb-04ec-6569c3688327", "datetime": "2014-07-16 02:28:40", "date": "2014-07-16", "time": "02:28:40" },
+        { "marker": "4fa1e250-7a68-3adb-04ec-6569c3688328", "datetime": "2018-06-13 20:16:07", "date": "2018-06-13", "time": "20:16:07" },
+        { "marker": "4fa1e250-7a68-3adb-04ec-6569c3688329", "datetime": "2011-10-08 12:02:23", "date": "2011-10-08", "time": "12:02:23" },
+        { "marker": "4fa1e250-7a68-3adb-04ec-6569c368832a", "datetime": "2014-08-18 02:11:11", "date": "2014-08-18", "time": "02:11:11" },
+        { "marker": "4fa1e250-7a68-3adb-04ec-6569c368832b", "datetime": "2016-01-07 05:27:51", "date": "2016-01-07", "time": "05:27:51" },
+        { "marker": "4fa1e250-7a68-3adb-04ec-6569c368832c", "datetime": "2016-10-21 20:15:55", "date": "2016-10-21", "time": "20:15:55" },
+        { "marker": "4fa1e250-7a68-3adb-04ec-6569c368832d", "datetime": "2016-12-01 03:57:10", "date": "2016-12-01", "time": "03:57:10" },
+        { "marker": "4fa1e250-7a68-3adb-04ec-6569c368832e", "datetime": "2016-12-29 01:54:42", "date": "2016-12-29", "time": "01:54:42" },
+        { "marker": "4fa1e250-7a68-3adb-04ec-6569c368832f", "datetime": "2018-07-22 19:55:31", "date": "2018-07-22", "time": "19:55:31" },
+        { "marker": "4fa1e250-7a68-3adb-04ec-6569c3688330", "datetime": "2018-09-07 01:42:54", "date": "2018-09-07", "time": "01:42:54" },
+        { "marker": "4fa1e250-7a68-3adb-04ec-6569c3688331", "datetime": "2019-03-08 03:41:06", "date": "2019-03-08", "time": "03:41:06" },
+        { "marker": "991ed2f7-3374-33a5-ee65-77f067848381", "datetime": "2019-06-07 17:54:58", "date": "2019-06-07", "time": "17:54:58" },
+        { "marker": "991ed2f7-3374-33a5-ee65-77f067848382", "datetime": "2015-04-02 21:45:17", "date": "2015-04-02", "time": "21:45:17" },
+        { "marker": "991ed2f7-3374-33a5-ee65-77f067848383", "datetime": "2014-01-28 20:56:04", "date": "2014-01-28", "time": "20:56:04" },
+        { "marker": "991ed2f7-3374-33a5-ee65-77f067848384", "datetime": "2014-11-16 16:11:58", "date": "2014-11-16", "time": "16:11:58" },
+        { "marker": "991ed2f7-3374-33a5-ee65-77f067848385", "datetime": "2015-11-15 19:21:53", "date": "2015-11-15", "time": "19:21:53" },
+        { "marker": "991ed2f7-3374-33a5-ee65-77f067848386", "datetime": "2015-11-15 19:33:39", "date": "2015-11-15", "time": "19:33:39" },
+        { "marker": "991ed2f7-3374-33a5-ee65-77f067848387", "datetime": "2018-09-29 18:55:17", "date": "2018-09-29", "time": "18:55:17" },
+        { "marker": "991ed2f7-3374-33a5-ee65-77f067848388", "datetime": "2018-10-20 16:48:05", "date": "2018-10-20", "time": "16:48:05" },
+        { "marker": "991ed2f7-3374-33a5-ee65-77f067848389", "datetime": "2018-10-20 22:20:24", "date": "2018-10-20", "time": "22:20:24" },
+        { "marker": "991ed2f7-3374-33a5-ee65-77f06784838a", "datetime": "2011-04-23 21:11:22", "date": "2011-04-23", "time": "21:11:22" },
+        { "marker": "991ed2f7-3374-33a5-ee65-77f06784838b", "datetime": "2014-05-04 19:42:48", "date": "2014-05-04", "time": "19:42:48" },
+        { "marker": "991ed2f7-3374-33a5-ee65-77f06784838c", "datetime": "2014-05-11 19:16:08", "date": "2014-05-11", "time": "19:16:08" },
+        { "marker": "991ed2f7-3374-33a5-ee65-77f06784838d", "datetime": "2014-06-04 19:14:18", "date": "2014-06-04", "time": "19:14:18" },
+        { "marker": "991ed2f7-3374-33a5-ee65-77f06784838e", "datetime": "2015-12-05 19:22:42", "date": "2015-12-05", "time": "19:22:42" },
+        { "marker": "991ed2f7-3374-33a5-ee65-77f06784838f", "datetime": "2017-05-15 23:19:00", "date": "2017-05-15", "time": "23:19:00" },
+        { "marker": "991ed2f7-3374-33a5-ee65-77f067848390", "datetime": "2017-05-19 14:30:16", "date": "2017-05-19", "time": "14:30:16" },
+        { "marker": "991ed2f7-3374-33a5-ee65-77f067848391", "datetime": "2017-05-19 14:30:25", "date": "2017-05-19", "time": "14:30:25" },
+        { "marker": "991ed2f7-3374-33a5-ee65-77f067848392", "datetime": "2017-08-28 15:49:37", "date": "2017-08-28", "time": "15:49:37" },
+        { "marker": "991ed2f7-3374-33a5-ee65-77f067848393", "datetime": "2017-09-20 20:19:51", "date": "2017-09-20", "time": "20:19:51" },
+        { "marker": "991ed2f7-3374-33a5-ee65-77f067848394", "datetime": "2017-10-01 16:31:05", "date": "2017-10-01", "time": "16:31:05" },
+        { "marker": "991ed2f7-3374-33a5-ee65-77f067848395", "datetime": "2017-10-01 16:56:27", "date": "2017-10-01", "time": "16:56:27" },
+        { "marker": "991ed2f7-3374-33a5-ee65-77f067848396", "datetime": "2017-12-27 23:33:20", "date": "2017-12-27", "time": "23:33:20" },
+        { "marker": "991ed2f7-3374-33a5-ee65-77f067848397", "datetime": "2011-09-24 21:37:32", "date": "2011-09-24", "time": "21:37:32" },
+        { "marker": "991ed2f7-3374-33a5-ee65-77f067848398", "datetime": "2014-03-10 20:20:07", "date": "2014-03-10", "time": "20:20:07" },
+        { "marker": "991ed2f7-3374-33a5-ee65-77f067848399", "datetime": "2015-05-27 00:40:24", "date": "2015-05-27", "time": "00:40:24" },
+        { "marker": "991ed2f7-3374-33a5-ee65-77f06784839a", "datetime": "2015-08-29 17:58:15", "date": "2015-08-29", "time": "17:58:15" },
+        { "marker": "991ed2f7-3374-33a5-ee65-77f06784839b", "datetime": "2018-03-16 15:03:26", "date": "2018-03-16", "time": "15:03:26" },
+        { "marker": "991ed2f7-3374-33a5-ee65-77f06784839c", "datetime": "2013-10-22 16:49:21", "date": "2013-10-22", "time": "16:49:21" },
+        { "marker": "991ed2f7-3374-33a5-ee65-77f06784839d", "datetime": "2014-11-21 17:39:24", "date": "2014-11-21", "time": "17:39:24" },
+        { "marker": "991ed2f7-3374-33a5-ee65-77f06784839e", "datetime": "2015-04-11 13:14:14", "date": "2015-04-11", "time": "13:14:14" },
+        { "marker": "991ed2f7-3374-33a5-ee65-77f06784839f", "datetime": "2015-11-21 16:05:56", "date": "2015-11-21", "time": "16:05:56" },
+        { "marker": "991ed2f7-3374-33a5-ee65-77f0678483a0", "datetime": "2016-05-06 14:10:04", "date": "2016-05-06", "time": "14:10:04" },
+        { "marker": "991ed2f7-3374-33a5-ee65-77f0678483a1", "datetime": "2017-08-09 15:15:10", "date": "2017-08-09", "time": "15:15:10" },
+        { "marker": "991ed2f7-3374-33a5-ee65-77f0678483a2", "datetime": "2017-10-21 15:12:56", "date": "2017-10-21", "time": "15:12:56" },
+        { "marker": "991ed2f7-3374-33a5-ee65-77f0678483a3", "datetime": "2015-12-03 18:44:00", "date": "2015-12-03", "time": "18:44:00" },
+        { "marker": "991ed2f7-3374-33a5-ee65-77f0678483a4", "datetime": "2016-03-17 18:19:21", "date": "2016-03-17", "time": "18:19:21" },
+        { "marker": "991ed2f7-3374-33a5-ee65-77f0678483a5", "datetime": "2016-11-02 15:58:38", "date": "2016-11-02", "time": "15:58:38" },
+        { "marker": "991ed2f7-3374-33a5-ee65-77f0678483a6", "datetime": "2019-02-27 14:03:08", "date": "2019-02-27", "time": "14:03:08" },
+        { "marker": "991ed2f7-3374-33a5-ee65-77f0678483a7", "datetime": "2015-05-29 16:46:17", "date": "2015-05-29", "time": "16:46:17" },
+        { "marker": "991ed2f7-3374-33a5-ee65-77f0678483a8", "datetime": "2015-06-01 15:03:53", "date": "2015-06-01", "time": "15:03:53" },
+        { "marker": "BAD_ENTRY_1", "datetime": "NA", "date": "NA", "time": "NA" },
+        { "marker": "BAD_ENTRY_2", "datetime": "NA", "date": "NA", "time": "NA" },
+        { "marker": "BAD_ENTRY_3", "datetime": "NA", "date": "NA", "time": "NA" },
+        { "marker": "BAD_ENTRY_4", "datetime": "NA", "date": "NA", "time": "NA" },
+        { "marker": "BAD_ENTRY_5", "datetime": "NA", "date": "NA", "time": "NA" },
+        { "marker": "BAD_ENTRY_6", "datetime": "NA", "date": "NA", "time": "NA" }
+    ]
+);
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/join-unnest-queries/use-case-3/use-case-3.4.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/join-unnest-queries/use-case-3/use-case-3.4.ddl.sqlpp
new file mode 100644
index 0000000..ee911b4
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/join-unnest-queries/use-case-3/use-case-3.4.ddl.sqlpp
@@ -0,0 +1,22 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+USE TestYelp;
+
+CREATE INDEX IdxYelpCheckinDates ON YelpCheckin (UNNEST checkin_times SELECT date);
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/join-unnest-queries/use-case-3/use-case-3.5.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/join-unnest-queries/use-case-3/use-case-3.5.query.sqlpp
new file mode 100644
index 0000000..147f99b
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/join-unnest-queries/use-case-3/use-case-3.5.query.sqlpp
@@ -0,0 +1,26 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+SET `compiler.arrayindex` "true";
+
+USE TestYelp;
+
+SELECT M.marker
+FROM YelpCheckinDateMarkers M, YelpCheckin C, C.checkin_times D
+WHERE D.date /*+ indexnl */ = M.date
+      AND C.business_id = "--Ni3oJ4VOqfOEu7Sj2Vzg";
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/join-unnest-queries/use-case-3/use-case-3.6.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/join-unnest-queries/use-case-3/use-case-3.6.query.sqlpp
new file mode 100644
index 0000000..5a380e4
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/join-unnest-queries/use-case-3/use-case-3.6.query.sqlpp
@@ -0,0 +1,26 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+SET `compiler.arrayindex` "true";
+
+USE TestYelp;
+
+SELECT M.marker
+FROM YelpCheckinDateMarkers M, YelpCheckin C, C.checkin_times D
+WHERE D.date /*+ indexnl */ = M.date
+      AND D.time = "19:49:16";
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/join-unnest-queries/use-case-3/use-case-3.7.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/join-unnest-queries/use-case-3/use-case-3.7.query.sqlpp
new file mode 100644
index 0000000..eb7687f
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/join-unnest-queries/use-case-3/use-case-3.7.query.sqlpp
@@ -0,0 +1,26 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+SET `compiler.arrayindex` "true";
+
+USE TestYelp;
+
+SELECT M.marker
+FROM YelpCheckinDateMarkers M, YelpCheckin C, C.checkin_times D
+WHERE D.date /*+ indexnl */ = M.date
+      AND M.time = "19:49:16";
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/join-unnest-queries/use-case-3/use-case-3.8.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/join-unnest-queries/use-case-3/use-case-3.8.query.sqlpp
new file mode 100644
index 0000000..2719057
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/join-unnest-queries/use-case-3/use-case-3.8.query.sqlpp
@@ -0,0 +1,28 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+SET `compiler.arrayindex` "true";
+
+USE TestYelp;
+
+SELECT COUNT(*)
+FROM YelpCheckinDateMarkers M
+INNER JOIN (
+    SELECT VALUE D.date
+    FROM YelpCheckin C, C.checkin_times D
+) AS CD ON CD /*+ indexnl */ = M.date;
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/join-unnest-queries/use-case-4/use-case-4.1.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/join-unnest-queries/use-case-4/use-case-4.1.ddl.sqlpp
new file mode 100644
index 0000000..41d36e9
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/join-unnest-queries/use-case-4/use-case-4.1.ddl.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.
+ */
+
+/*
+ * Description: Verify that we are able to correctly perform the following types of queries using an array->record->array->atomic index.
+ *     1) INNER JOIN query, with additional predicate on inner collection before the first unnest.
+ *     2) INNER JOIN query, with additional predicate on inner collection before the first unnest and before the final unnest.
+ *     3) INNER JOIN query, with additional predicate on the outer collection.
+ *     4) Explicit INNER JOIN query using UNNEST subquery, no additional predicate.
+ *     TODO (GLENN) 5) Explicit LEFT OUTER JOIN query using UNNEST subquery, no additional predicate.
+ */
+
+DROP DATAVERSE TestYelp IF EXISTS;
+CREATE DATAVERSE TestYelp;
+USE TestYelp;
+
+CREATE TYPE CheckinType AS {
+    checkin_id: uuid,
+    business_id: string,
+    checkin_times: [{
+        checkin_times_id: int,
+        dates: [string],
+        times: [string]
+    }]
+};
+CREATE TYPE CheckinDatesToMarkersType AS {
+    marker: string,
+    datetime: string,
+    date: string,
+    time: string
+};
+
+CREATE DATASET YelpCheckin(CheckinType) PRIMARY KEY checkin_id AUTOGENERATED;
+CREATE DATASET YelpCheckinDateMarkers (CheckinDatesToMarkersType) PRIMARY KEY marker;
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/join-unnest-queries/use-case-4/use-case-4.2.update.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/join-unnest-queries/use-case-4/use-case-4.2.update.sqlpp
new file mode 100644
index 0000000..21105b9
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/join-unnest-queries/use-case-4/use-case-4.2.update.sqlpp
@@ -0,0 +1,487 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+USE TestYelp;
+
+INSERT INTO YelpCheckin (
+    [
+        {
+            "business_id": "--1UhMGODdWsrMastO9DZw",
+            "checkin_times": [ 
+                {
+                    "checkin_times_id": 1,
+                    "dates": ["2016-04-26", "2016-08-30", "2016-10-15", "2016-11-18"],
+                    "times": ["19:49:16", "18:36:57", "02:45:18", "01:54:50"]
+                },
+                {
+                    "checkin_times_id": 2,
+                    "dates": ["2017-04-20", "2017-05-03"],
+                    "times": ["18:39:06", "17:58:02"]
+                },
+                {
+                    "checkin_times_id": 3,
+                    "dates": ["2019-03-19"],
+                    "times": ["22:04:48"]
+                }
+             ]
+        },
+        {
+            "business_id": "--EF5N7P70J_UYBTPypYlA",
+            "checkin_times": [ 
+                {
+                    "checkin_times_id": 1,
+                    "dates": ["2018-05-25", "2018-09-18"],
+                    "times": ["19:52:07", "16:09:44"]
+                },
+                {
+                    "checkin_times_id": 2,
+                    "dates": ["2019-10-18"],
+                    "times": ["21:29:09"]
+                }
+             ]
+        },
+        {
+            "business_id": "--Ni3oJ4VOqfOEu7Sj2Vzg",
+            "checkin_times": [ 
+                {
+                    "checkin_times_id": 1,
+                    "dates": ["2019-06-07"],
+                    "times": ["17:54:58"]
+                }
+             ]
+        },
+        {
+            "business_id": "--Y1Adl1YUWfYIRSd8vkmA",
+            "checkin_times": [ 
+                {
+                    "checkin_times_id": 1,
+                    "dates": ["2011-05-03", "2011-08-23"],
+                    "times": ["20:54:05", "20:49:45"]
+                },
+                {
+                    "checkin_times_id": 2,
+                    "dates": ["2014-12-04"],
+                    "times": ["06:13:01"]
+                },
+                {
+                    "checkin_times_id": 3,
+                    "dates": ["2016-11-16"],
+                    "times": ["19:25:55"]
+                }
+             ]
+        },
+        {
+            "business_id": "--YPwqIlRJrhHkJcjY3eiA",
+            "checkin_times": [ 
+                {
+                    "checkin_times_id": 1,
+                    "dates": ["2016-06-18", "2016-10-15"],
+                    "times": ["21:35:45", "18:17:51"]
+                }
+             ]
+        },
+        {
+            "business_id": "--e8PjCNhEz32pprnPhCwQ",
+            "checkin_times": [ 
+                {
+                    "checkin_times_id": 1,
+                    "dates": ["2015-04-02"],
+                    "times": ["21:45:17"]
+                }
+             ]
+        },
+        {
+            "business_id": "--kinfHwmtdjz03g8B8z8Q",
+            "checkin_times": [ 
+                {
+                    "checkin_times_id": 1,
+                    "dates": ["2014-08-27"],
+                    "times": ["17:49:18"]
+                },
+                   {
+                    "checkin_times_id": 2,
+                    "dates": ["2015-12-19"],
+                    "times": ["21:30:31"]
+                },
+                {
+                    "checkin_times_id": 3,
+                    "dates": ["2018-11-27"],
+                    "times": ["15:53:50"]
+                }
+             ]
+        },
+        {
+            "business_id": "--q6datkI-f0EoVheXNEeQ",
+            "checkin_times": [ 
+                {
+                    "checkin_times_id": 1,
+                    "dates": ["2014-01-28", "2014-11-16"],
+                    "times": ["20:56:04", "16:11:58"]
+                },
+                {
+                    "checkin_times_id": 2,
+                    "dates": ["2015-11-15", "2015-11-15"],
+                    "times": ["19:21:53", "19:33:39"]
+                }
+             ]
+        },
+        {
+            "business_id": "--qvQS4MigHPykD2GV0-zw",
+            "checkin_times": [ 
+                {
+                    "checkin_times_id": 1,
+                    "dates": ["2019-04-11"],
+                    "times": ["18:30:12"]
+                }
+             ]
+        },
+        {
+            "business_id": "--wIGbLEhlpl_UeAIyDmZQ",
+            "checkin_times": [ 
+                {
+                    "checkin_times_id": 1,
+                    "dates": ["2015-06-06"],
+                    "times": ["20:01:06"]
+                },
+                {
+                    "checkin_times_id": 2,
+                    "dates": ["2019-03-14"],
+                    "times": ["22:01:52"]
+                }
+             ]
+        },
+        {
+            "business_id": "-0FA-Qdi3SPYIoJz9UQw-A",
+            "checkin_times": [ 
+                {
+                    "checkin_times_id": 1,
+                    "dates": ["2018-09-29", "2018-10-20", "2018-10-20"],
+                    "times": ["18:55:17", "16:48:05", "22:20:24"]
+                }
+             ]
+        },
+        {
+            "business_id": "-0Hj1hb_XW6ybWq2M7QhGA",
+            "checkin_times": [ 
+                {
+                    "checkin_times_id": 1,
+                    "dates": ["2011-04-23"],
+                    "times": ["21:11:22"]
+                },
+                {
+                    "checkin_times_id": 2,
+                    "dates": ["2014-05-04", "2014-05-11", "2014-06-04"],
+                    "times": ["19:42:48", "19:16:08", "19:14:18"]
+                },
+                {
+                    "checkin_times_id": 3,
+                    "dates": ["2015-12-05"],
+                    "times": ["19:22:42"]
+                },
+                {
+                    "checkin_times_id": 4,
+                    "dates": ["2017-05-15"],
+                    "times": ["23:19:00"]
+                }
+             ]
+        },
+        {
+            "business_id": "-0KMvRFwDWdVBeTpT11iHw",
+            "checkin_times": [ 
+                {
+                    "checkin_times_id": 1,
+                    "dates": ["2012-07-13"],
+                    "times": ["21:43:57"]
+                },
+                {
+                    "checkin_times_id": 2,
+                    "dates": ["2016-12-24"],
+                    "times": ["02:27:31"]
+                },
+                {
+                    "checkin_times_id": 3,
+                    "dates": ["2017-08-31"],
+                    "times": ["00:35:26"]
+                }
+             ]
+        },
+        {
+            "business_id": "-0LPtgJC31FWMrMv317p0Q",
+            "checkin_times": [ 
+                {
+                    "checkin_times_id": 1,
+                    "dates": ["2013-04-13", "2013-08-19", "2013-10-04"],
+                    "times": ["12:35:33", "23:35:49", "19:14:56"]
+                }
+
+             ]
+        },
+        {
+            "business_id": "-0M3o2uWBnQZwd3hmfEwuw",
+            "checkin_times": [ 
+                {
+                    "checkin_times_id": 1,
+                    "dates": ["2016-09-10"],
+                    "times": ["19:26:19"]
+                },
+                {
+                    "checkin_times_id": 2,
+                    "dates": ["2018-09-08"],
+                    "times": ["14:15:37"]
+                },
+                {
+                    "checkin_times_id": 3,
+                    "dates": ["2019-09-13"],
+                    "times": ["22:47:25"]
+                }
+             ]
+        },
+        {
+            "business_id": "-0RRiWDtfnS16AKCtfvBZg",
+            "checkin_times": [ 
+                {
+                    "checkin_times_id": 1,
+                    "dates": ["2017-05-19", "2017-05-19", "2017-08-28", "2017-09-20", "2017-10-01", "2017-10-01", "2017-12-27"],
+                    "times": ["14:30:16", "14:30:25", "15:49:37", "20:19:51", "16:31:05", "16:56:27", "23:33:20"]
+                }
+             ]
+        },
+        {
+            "business_id": "-0Soj75v-XoRcf2ERr8Bmg",
+            "checkin_times": [ 
+                {
+                    "checkin_times_id": 1,
+                    "dates": ["2019-06-05"],
+                    "times": ["18:22:49"]
+                }
+             ]
+        },
+        {
+            "business_id": "-0ZumLlFjMh4ZW1z2nXGug",
+            "checkin_times": [ 
+                {
+                    "checkin_times_id": 1,
+                    "dates": ["2011-09-24"],
+                    "times": ["21:37:32"]
+                },
+                {
+                    "checkin_times_id": 2,
+                    "dates": ["2014-03-10"],
+                    "times": ["20:20:07"]
+                },
+                {
+                    "checkin_times_id": 3,
+                    "dates": ["2015-05-27", "2015-08-29"],
+                    "times": ["00:40:24", "17:58:15"]
+                },
+                {
+                    "checkin_times_id": 4,
+                    "dates": ["2018-03-16"],
+                    "times": ["15:03:26"]
+                }
+             ]
+        },
+        {
+            "business_id": "-0aOudcaAyac0VJbMX-L1g",
+            "checkin_times": [ 
+                {
+                    "checkin_times_id": 1,
+                    "dates": ["2015-03-16", "2015-12-21"],
+                    "times": ["23:51:16", "04:48:01"]
+                },
+                {
+                    "checkin_times_id": 2,
+                    "dates": ["2016-10-28", "2016-10-28"],
+                    "times": ["20:22:42", "20:23:00"]
+                }
+             ]
+        },
+        {
+            "business_id": "-0b86isaXMY0v4g-V8GZ9Q",
+            "checkin_times": [ 
+                {
+                    "checkin_times_id": 1,
+                    "dates": ["2013-10-22"],
+                    "times": ["16:49:21"]
+                },
+                {
+                    "checkin_times_id": 2,
+                    "dates": ["2014-11-21"],
+                    "times": ["17:39:24"]
+                }
+             ]
+        },
+        {
+            "business_id": "-0d-BfFSU0bwLcnMaGRxYw",
+            "checkin_times": [ 
+                {
+                    "checkin_times_id": 1,
+                    "dates": ["2014-08-07", "2014-09-16", "2014-10-12"],
+                    "times": ["18:30:48", "20:41:45", "23:22:27"]
+                },
+                {
+                    "checkin_times_id": 2,
+                    "dates": ["2015-07-21", "2015-07-21"],
+                    "times": ["20:43:56", "20:45:07"]
+                }
+             ]
+        },
+        {
+            "business_id": "-0jz6c3C6i7RG7Ag22K-Pg",
+            "checkin_times": [ 
+                {
+                    "checkin_times_id": 1,
+                    "dates": ["2015-05-02", "2015-05-06", "2015-09-26"],
+                    "times": ["19:49:05", "03:52:18", "01:13:19"]
+                }
+
+             ]
+        },
+        {
+            "business_id": "-0y3MZU2oYP8r1ruDP1bfQ",
+            "checkin_times": [ 
+                {
+                    "checkin_times_id": 1,
+                    "dates": ["2015-04-11", "2015-11-21"],
+                    "times": ["13:14:14", "16:05:56"]
+                },
+                {
+                    "checkin_times_id": 2,
+                    "dates": ["2016-05-06"],
+                    "times": ["14:10:04"]
+                },
+                {
+                    "checkin_times_id": 3,
+                    "dates": ["2017-08-09", "2017-10-21"],
+                    "times": ["15:15:10", "15:12:56"]
+                }
+             ]
+        },
+        {
+            "business_id": "-1BPe8UjF2_l3nVk-DFUjA",
+            "checkin_times": [ 
+                {
+                    "checkin_times_id": 1,
+                    "dates": ["2015-12-03"],
+                    "times": ["18:44:00"]
+                },
+                {
+                    "checkin_times_id": 2,
+                    "dates": ["2016-03-17", "2016-11-02"],
+                    "times": ["18:19:21", "15:58:38"]
+                }
+             ]
+        },
+        {
+            "business_id": "-1E2CQu_38mkghvmZgCCRw",
+            "checkin_times": [ 
+                {
+                    "checkin_times_id": 1,
+                    "dates": ["2019-04-04"],
+                    "times": ["22:02:37"]
+                }
+             ]
+        },
+        {
+            "business_id": "-1wzk43IZ5D9Ysu6kzb5xA",
+            "checkin_times": [ 
+                {
+                    "checkin_times_id": 1,
+                    "dates": ["2019-02-27"],
+                    "times": ["14:03:08"]
+                }
+             ]
+        },
+        {
+            "business_id": "-23R9P2eG7VTc6DVLjFKzA",
+            "checkin_times": [ 
+                {
+                    "checkin_times_id": 1,
+                    "dates": ["2011-12-21"],
+                    "times": ["19:02:51"]
+                },
+                {
+                    "checkin_times_id": 2,
+                    "dates": ["2012-04-15", "2012-04-15"],
+                    "times": ["04:21:39", "14:23:56"]
+                },
+                {
+                    "checkin_times_id": 3,
+                    "dates": ["2013-06-30", "2013-10-04"],
+                    "times": ["22:39:51", "20:34:13"]
+                },
+                {
+                    "checkin_times_id": 4,
+                    "dates": ["2014-07-16"],
+                    "times": ["02:28:40"]
+                }
+             ]
+        },
+        {
+            "business_id": "-26MGfikhJiTfCI-GqmzhQ",
+            "checkin_times": [ 
+                {
+                    "checkin_times_id": 1,
+                    "dates": ["2018-06-13"],
+                    "times": ["20:16:07"]
+                }
+             ]
+        },
+        {
+            "business_id": "-2bLuJsMZ0WhI9daurVQNQ",
+            "checkin_times": [ 
+                {
+                    "checkin_times_id": 1,
+                    "dates": ["2015-05-29", "2015-06-01"],
+                    "times": ["16:46:17", "15:03:53"]
+                }
+             ]
+        },
+        {
+            "business_id": "-2hDBMaza_ldqnZdiU06LQ",
+            "checkin_times": [ 
+                {
+                    "checkin_times_id": 1,
+                    "dates": ["2011-10-08"],
+                    "times": ["12:02:23"]
+                },
+                {
+                    "checkin_times_id": 2,
+                    "dates": ["2014-08-18"],
+                    "times": ["02:11:11"]
+                },
+                {
+                    "checkin_times_id": 3,
+                    "dates": ["2016-01-07", "2016-10-21", "2016-12-01", "2016-12-29"],
+                    "times": ["05:27:51", "20:15:55", "03:57:10", "01:54:42"]
+                },
+                {
+                    "checkin_times_id": 4,
+                    "dates": ["2018-07-22", "2018-09-07"],
+                    "times": ["19:55:31", "01:42:54"]
+                },
+                {
+                    "checkin_times_id": 5,
+                    "dates": ["2019-03-08"],
+                    "times": ["03:41:06"]
+                }
+             ]
+        }
+    ]
+);
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/join-unnest-queries/use-case-4/use-case-4.3.update.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/join-unnest-queries/use-case-4/use-case-4.3.update.sqlpp
new file mode 100644
index 0000000..c1eca5d
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/join-unnest-queries/use-case-4/use-case-4.3.update.sqlpp
@@ -0,0 +1,132 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+USE TestYelp;
+
+INSERT INTO YelpCheckinDateMarkers (
+    [
+        { "marker": "4fa1e250-7a68-3adb-04ec-6569c36882f5", "datetime": "2016-04-26 19:49:16", "date": "2016-04-26", "time": "19:49:16" },
+        { "marker": "4fa1e250-7a68-3adb-04ec-6569c36882f6", "datetime": "2016-08-30 18:36:57", "date": "2016-08-30", "time": "18:36:57" },
+        { "marker": "4fa1e250-7a68-3adb-04ec-6569c36882f7", "datetime": "2016-10-15 02:45:18", "date": "2016-10-15", "time": "02:45:18" },
+        { "marker": "4fa1e250-7a68-3adb-04ec-6569c36882f8", "datetime": "2016-11-18 01:54:50", "date": "2016-11-18", "time": "01:54:50" },
+        { "marker": "4fa1e250-7a68-3adb-04ec-6569c36882f9", "datetime": "2017-04-20 18:39:06", "date": "2017-04-20", "time": "18:39:06" },
+        { "marker": "4fa1e250-7a68-3adb-04ec-6569c36882fa", "datetime": "2017-05-03 17:58:02", "date": "2017-05-03", "time": "17:58:02" },
+        { "marker": "4fa1e250-7a68-3adb-04ec-6569c36882fb", "datetime": "2019-03-19 22:04:48", "date": "2019-03-19", "time": "22:04:48" },
+        { "marker": "4fa1e250-7a68-3adb-04ec-6569c36882fc", "datetime": "2018-05-25 19:52:07", "date": "2018-05-25", "time": "19:52:07" },
+        { "marker": "4fa1e250-7a68-3adb-04ec-6569c36882fd", "datetime": "2018-09-18 16:09:44", "date": "2018-09-18", "time": "16:09:44" },
+        { "marker": "4fa1e250-7a68-3adb-04ec-6569c36882fe", "datetime": "2019-10-18 21:29:09", "date": "2019-10-18", "time": "21:29:09" },
+        { "marker": "4fa1e250-7a68-3adb-04ec-6569c36882ff", "datetime": "2011-05-03 20:54:05", "date": "2011-05-03", "time": "20:54:05" },
+        { "marker": "4fa1e250-7a68-3adb-04ec-6569c3688300", "datetime": "2011-08-23 20:49:45", "date": "2011-08-23", "time": "20:49:45" },
+        { "marker": "4fa1e250-7a68-3adb-04ec-6569c3688301", "datetime": "2014-12-04 06:13:01", "date": "2014-12-04", "time": "06:13:01" },
+        { "marker": "4fa1e250-7a68-3adb-04ec-6569c3688302", "datetime": "2016-11-16 19:25:55", "date": "2016-11-16", "time": "19:25:55" },
+        { "marker": "4fa1e250-7a68-3adb-04ec-6569c3688303", "datetime": "2016-06-18 21:35:45", "date": "2016-06-18", "time": "21:35:45" },
+        { "marker": "4fa1e250-7a68-3adb-04ec-6569c3688304", "datetime": "2016-10-15 18:17:51", "date": "2016-10-15", "time": "18:17:51" },
+        { "marker": "4fa1e250-7a68-3adb-04ec-6569c3688305", "datetime": "2014-08-27 17:49:18", "date": "2014-08-27", "time": "17:49:18" },
+        { "marker": "4fa1e250-7a68-3adb-04ec-6569c3688306", "datetime": "2015-12-19 21:30:31", "date": "2015-12-19", "time": "21:30:31" },
+        { "marker": "4fa1e250-7a68-3adb-04ec-6569c3688307", "datetime": "2018-11-27 15:53:50", "date": "2018-11-27", "time": "15:53:50" },
+        { "marker": "4fa1e250-7a68-3adb-04ec-6569c3688308", "datetime": "2019-04-11 18:30:12", "date": "2019-04-11", "time": "18:30:12" },
+        { "marker": "4fa1e250-7a68-3adb-04ec-6569c3688309", "datetime": "2015-06-06 20:01:06", "date": "2015-06-06", "time": "20:01:06" },
+        { "marker": "4fa1e250-7a68-3adb-04ec-6569c368830a", "datetime": "2019-03-14 22:01:52", "date": "2019-03-14", "time": "22:01:52" },
+        { "marker": "4fa1e250-7a68-3adb-04ec-6569c368830b", "datetime": "2012-07-13 21:43:57", "date": "2012-07-13", "time": "21:43:57" },
+        { "marker": "4fa1e250-7a68-3adb-04ec-6569c368830c", "datetime": "2016-12-24 02:27:31", "date": "2016-12-24", "time": "02:27:31" },
+        { "marker": "4fa1e250-7a68-3adb-04ec-6569c368830d", "datetime": "2017-08-31 00:35:26", "date": "2017-08-31", "time": "00:35:26" },
+        { "marker": "4fa1e250-7a68-3adb-04ec-6569c368830e", "datetime": "2013-04-13 12:35:33", "date": "2013-04-13", "time": "12:35:33" },
+        { "marker": "4fa1e250-7a68-3adb-04ec-6569c368830f", "datetime": "2013-08-19 23:35:49", "date": "2013-08-19", "time": "23:35:49" },
+        { "marker": "4fa1e250-7a68-3adb-04ec-6569c3688310", "datetime": "2013-10-04 19:14:56", "date": "2013-10-04", "time": "19:14:56" },
+        { "marker": "4fa1e250-7a68-3adb-04ec-6569c3688311", "datetime": "2016-09-10 19:26:19", "date": "2016-09-10", "time": "19:26:19" },
+        { "marker": "4fa1e250-7a68-3adb-04ec-6569c3688312", "datetime": "2018-09-08 14:15:37", "date": "2018-09-08", "time": "14:15:37" },
+        { "marker": "4fa1e250-7a68-3adb-04ec-6569c3688313", "datetime": "2019-09-13 22:47:25", "date": "2019-09-13", "time": "22:47:25" },
+        { "marker": "4fa1e250-7a68-3adb-04ec-6569c3688314", "datetime": "2019-06-05 18:22:49", "date": "2019-06-05", "time": "18:22:49" },
+        { "marker": "4fa1e250-7a68-3adb-04ec-6569c3688315", "datetime": "2015-03-16 23:51:16", "date": "2015-03-16", "time": "23:51:16" },
+        { "marker": "4fa1e250-7a68-3adb-04ec-6569c3688316", "datetime": "2015-12-21 04:48:01", "date": "2015-12-21", "time": "04:48:01" },
+        { "marker": "4fa1e250-7a68-3adb-04ec-6569c3688317", "datetime": "2016-10-28 20:22:42", "date": "2016-10-28", "time": "20:22:42" },
+        { "marker": "4fa1e250-7a68-3adb-04ec-6569c3688318", "datetime": "2016-10-28 20:23:00", "date": "2016-10-28", "time": "20:23:00" },
+        { "marker": "4fa1e250-7a68-3adb-04ec-6569c3688319", "datetime": "2014-08-07 18:30:48", "date": "2014-08-07", "time": "18:30:48" },
+        { "marker": "4fa1e250-7a68-3adb-04ec-6569c368831a", "datetime": "2014-09-16 20:41:45", "date": "2014-09-16", "time": "20:41:45" },
+        { "marker": "4fa1e250-7a68-3adb-04ec-6569c368831b", "datetime": "2014-10-12 23:22:27", "date": "2014-10-12", "time": "23:22:27" },
+        { "marker": "4fa1e250-7a68-3adb-04ec-6569c368831c", "datetime": "2015-07-21 20:43:56", "date": "2015-07-21", "time": "20:43:56" },
+        { "marker": "4fa1e250-7a68-3adb-04ec-6569c368831d", "datetime": "2015-07-21 20:45:07", "date": "2015-07-21", "time": "20:45:07" },
+        { "marker": "4fa1e250-7a68-3adb-04ec-6569c368831e", "datetime": "2015-05-02 19:49:05", "date": "2015-05-02", "time": "19:49:05" },
+        { "marker": "4fa1e250-7a68-3adb-04ec-6569c368831f", "datetime": "2015-05-06 03:52:18", "date": "2015-05-06", "time": "03:52:18" },
+        { "marker": "4fa1e250-7a68-3adb-04ec-6569c3688320", "datetime": "2015-09-26 01:13:19", "date": "2015-09-26", "time": "01:13:19" },
+        { "marker": "4fa1e250-7a68-3adb-04ec-6569c3688321", "datetime": "2019-04-04 22:02:37", "date": "2019-04-04", "time": "22:02:37" },
+        { "marker": "4fa1e250-7a68-3adb-04ec-6569c3688322", "datetime": "2011-12-21 19:02:51", "date": "2011-12-21", "time": "19:02:51" },
+        { "marker": "4fa1e250-7a68-3adb-04ec-6569c3688323", "datetime": "2012-04-15 04:21:39", "date": "2012-04-15", "time": "04:21:39" },
+        { "marker": "4fa1e250-7a68-3adb-04ec-6569c3688324", "datetime": "2012-04-15 14:23:56", "date": "2012-04-15", "time": "14:23:56" },
+        { "marker": "4fa1e250-7a68-3adb-04ec-6569c3688325", "datetime": "2013-06-30 22:39:51", "date": "2013-06-30", "time": "22:39:51" },
+        { "marker": "4fa1e250-7a68-3adb-04ec-6569c3688326", "datetime": "2013-10-04 20:34:13", "date": "2013-10-04", "time": "20:34:13" },
+        { "marker": "4fa1e250-7a68-3adb-04ec-6569c3688327", "datetime": "2014-07-16 02:28:40", "date": "2014-07-16", "time": "02:28:40" },
+        { "marker": "4fa1e250-7a68-3adb-04ec-6569c3688328", "datetime": "2018-06-13 20:16:07", "date": "2018-06-13", "time": "20:16:07" },
+        { "marker": "4fa1e250-7a68-3adb-04ec-6569c3688329", "datetime": "2011-10-08 12:02:23", "date": "2011-10-08", "time": "12:02:23" },
+        { "marker": "4fa1e250-7a68-3adb-04ec-6569c368832a", "datetime": "2014-08-18 02:11:11", "date": "2014-08-18", "time": "02:11:11" },
+        { "marker": "4fa1e250-7a68-3adb-04ec-6569c368832b", "datetime": "2016-01-07 05:27:51", "date": "2016-01-07", "time": "05:27:51" },
+        { "marker": "4fa1e250-7a68-3adb-04ec-6569c368832c", "datetime": "2016-10-21 20:15:55", "date": "2016-10-21", "time": "20:15:55" },
+        { "marker": "4fa1e250-7a68-3adb-04ec-6569c368832d", "datetime": "2016-12-01 03:57:10", "date": "2016-12-01", "time": "03:57:10" },
+        { "marker": "4fa1e250-7a68-3adb-04ec-6569c368832e", "datetime": "2016-12-29 01:54:42", "date": "2016-12-29", "time": "01:54:42" },
+        { "marker": "4fa1e250-7a68-3adb-04ec-6569c368832f", "datetime": "2018-07-22 19:55:31", "date": "2018-07-22", "time": "19:55:31" },
+        { "marker": "4fa1e250-7a68-3adb-04ec-6569c3688330", "datetime": "2018-09-07 01:42:54", "date": "2018-09-07", "time": "01:42:54" },
+        { "marker": "4fa1e250-7a68-3adb-04ec-6569c3688331", "datetime": "2019-03-08 03:41:06", "date": "2019-03-08", "time": "03:41:06" },
+        { "marker": "991ed2f7-3374-33a5-ee65-77f067848381", "datetime": "2019-06-07 17:54:58", "date": "2019-06-07", "time": "17:54:58" },
+        { "marker": "991ed2f7-3374-33a5-ee65-77f067848382", "datetime": "2015-04-02 21:45:17", "date": "2015-04-02", "time": "21:45:17" },
+        { "marker": "991ed2f7-3374-33a5-ee65-77f067848383", "datetime": "2014-01-28 20:56:04", "date": "2014-01-28", "time": "20:56:04" },
+        { "marker": "991ed2f7-3374-33a5-ee65-77f067848384", "datetime": "2014-11-16 16:11:58", "date": "2014-11-16", "time": "16:11:58" },
+        { "marker": "991ed2f7-3374-33a5-ee65-77f067848385", "datetime": "2015-11-15 19:21:53", "date": "2015-11-15", "time": "19:21:53" },
+        { "marker": "991ed2f7-3374-33a5-ee65-77f067848386", "datetime": "2015-11-15 19:33:39", "date": "2015-11-15", "time": "19:33:39" },
+        { "marker": "991ed2f7-3374-33a5-ee65-77f067848387", "datetime": "2018-09-29 18:55:17", "date": "2018-09-29", "time": "18:55:17" },
+        { "marker": "991ed2f7-3374-33a5-ee65-77f067848388", "datetime": "2018-10-20 16:48:05", "date": "2018-10-20", "time": "16:48:05" },
+        { "marker": "991ed2f7-3374-33a5-ee65-77f067848389", "datetime": "2018-10-20 22:20:24", "date": "2018-10-20", "time": "22:20:24" },
+        { "marker": "991ed2f7-3374-33a5-ee65-77f06784838a", "datetime": "2011-04-23 21:11:22", "date": "2011-04-23", "time": "21:11:22" },
+        { "marker": "991ed2f7-3374-33a5-ee65-77f06784838b", "datetime": "2014-05-04 19:42:48", "date": "2014-05-04", "time": "19:42:48" },
+        { "marker": "991ed2f7-3374-33a5-ee65-77f06784838c", "datetime": "2014-05-11 19:16:08", "date": "2014-05-11", "time": "19:16:08" },
+        { "marker": "991ed2f7-3374-33a5-ee65-77f06784838d", "datetime": "2014-06-04 19:14:18", "date": "2014-06-04", "time": "19:14:18" },
+        { "marker": "991ed2f7-3374-33a5-ee65-77f06784838e", "datetime": "2015-12-05 19:22:42", "date": "2015-12-05", "time": "19:22:42" },
+        { "marker": "991ed2f7-3374-33a5-ee65-77f06784838f", "datetime": "2017-05-15 23:19:00", "date": "2017-05-15", "time": "23:19:00" },
+        { "marker": "991ed2f7-3374-33a5-ee65-77f067848390", "datetime": "2017-05-19 14:30:16", "date": "2017-05-19", "time": "14:30:16" },
+        { "marker": "991ed2f7-3374-33a5-ee65-77f067848391", "datetime": "2017-05-19 14:30:25", "date": "2017-05-19", "time": "14:30:25" },
+        { "marker": "991ed2f7-3374-33a5-ee65-77f067848392", "datetime": "2017-08-28 15:49:37", "date": "2017-08-28", "time": "15:49:37" },
+        { "marker": "991ed2f7-3374-33a5-ee65-77f067848393", "datetime": "2017-09-20 20:19:51", "date": "2017-09-20", "time": "20:19:51" },
+        { "marker": "991ed2f7-3374-33a5-ee65-77f067848394", "datetime": "2017-10-01 16:31:05", "date": "2017-10-01", "time": "16:31:05" },
+        { "marker": "991ed2f7-3374-33a5-ee65-77f067848395", "datetime": "2017-10-01 16:56:27", "date": "2017-10-01", "time": "16:56:27" },
+        { "marker": "991ed2f7-3374-33a5-ee65-77f067848396", "datetime": "2017-12-27 23:33:20", "date": "2017-12-27", "time": "23:33:20" },
+        { "marker": "991ed2f7-3374-33a5-ee65-77f067848397", "datetime": "2011-09-24 21:37:32", "date": "2011-09-24", "time": "21:37:32" },
+        { "marker": "991ed2f7-3374-33a5-ee65-77f067848398", "datetime": "2014-03-10 20:20:07", "date": "2014-03-10", "time": "20:20:07" },
+        { "marker": "991ed2f7-3374-33a5-ee65-77f067848399", "datetime": "2015-05-27 00:40:24", "date": "2015-05-27", "time": "00:40:24" },
+        { "marker": "991ed2f7-3374-33a5-ee65-77f06784839a", "datetime": "2015-08-29 17:58:15", "date": "2015-08-29", "time": "17:58:15" },
+        { "marker": "991ed2f7-3374-33a5-ee65-77f06784839b", "datetime": "2018-03-16 15:03:26", "date": "2018-03-16", "time": "15:03:26" },
+        { "marker": "991ed2f7-3374-33a5-ee65-77f06784839c", "datetime": "2013-10-22 16:49:21", "date": "2013-10-22", "time": "16:49:21" },
+        { "marker": "991ed2f7-3374-33a5-ee65-77f06784839d", "datetime": "2014-11-21 17:39:24", "date": "2014-11-21", "time": "17:39:24" },
+        { "marker": "991ed2f7-3374-33a5-ee65-77f06784839e", "datetime": "2015-04-11 13:14:14", "date": "2015-04-11", "time": "13:14:14" },
+        { "marker": "991ed2f7-3374-33a5-ee65-77f06784839f", "datetime": "2015-11-21 16:05:56", "date": "2015-11-21", "time": "16:05:56" },
+        { "marker": "991ed2f7-3374-33a5-ee65-77f0678483a0", "datetime": "2016-05-06 14:10:04", "date": "2016-05-06", "time": "14:10:04" },
+        { "marker": "991ed2f7-3374-33a5-ee65-77f0678483a1", "datetime": "2017-08-09 15:15:10", "date": "2017-08-09", "time": "15:15:10" },
+        { "marker": "991ed2f7-3374-33a5-ee65-77f0678483a2", "datetime": "2017-10-21 15:12:56", "date": "2017-10-21", "time": "15:12:56" },
+        { "marker": "991ed2f7-3374-33a5-ee65-77f0678483a3", "datetime": "2015-12-03 18:44:00", "date": "2015-12-03", "time": "18:44:00" },
+        { "marker": "991ed2f7-3374-33a5-ee65-77f0678483a4", "datetime": "2016-03-17 18:19:21", "date": "2016-03-17", "time": "18:19:21" },
+        { "marker": "991ed2f7-3374-33a5-ee65-77f0678483a5", "datetime": "2016-11-02 15:58:38", "date": "2016-11-02", "time": "15:58:38" },
+        { "marker": "991ed2f7-3374-33a5-ee65-77f0678483a6", "datetime": "2019-02-27 14:03:08", "date": "2019-02-27", "time": "14:03:08" },
+        { "marker": "991ed2f7-3374-33a5-ee65-77f0678483a7", "datetime": "2015-05-29 16:46:17", "date": "2015-05-29", "time": "16:46:17" },
+        { "marker": "991ed2f7-3374-33a5-ee65-77f0678483a8", "datetime": "2015-06-01 15:03:53", "date": "2015-06-01", "time": "15:03:53" },
+        { "marker": "BAD_ENTRY_1", "datetime": "NA", "date": "NA", "time": "NA" },
+        { "marker": "BAD_ENTRY_2", "datetime": "NA", "date": "NA", "time": "NA" },
+        { "marker": "BAD_ENTRY_3", "datetime": "NA", "date": "NA", "time": "NA" },
+        { "marker": "BAD_ENTRY_4", "datetime": "NA", "date": "NA", "time": "NA" },
+        { "marker": "BAD_ENTRY_5", "datetime": "NA", "date": "NA", "time": "NA" },
+        { "marker": "BAD_ENTRY_6", "datetime": "NA", "date": "NA", "time": "NA" }
+    ]
+);
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/join-unnest-queries/use-case-4/use-case-4.4.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/join-unnest-queries/use-case-4/use-case-4.4.ddl.sqlpp
new file mode 100644
index 0000000..05a7c76
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/join-unnest-queries/use-case-4/use-case-4.4.ddl.sqlpp
@@ -0,0 +1,22 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+USE TestYelp;
+
+CREATE INDEX IdxYelpCheckinDates ON YelpCheckin (UNNEST checkin_times UNNEST dates);
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/join-unnest-queries/use-case-4/use-case-4.5.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/join-unnest-queries/use-case-4/use-case-4.5.query.sqlpp
new file mode 100644
index 0000000..7220d42
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/join-unnest-queries/use-case-4/use-case-4.5.query.sqlpp
@@ -0,0 +1,26 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+SET `compiler.arrayindex` "true";
+
+USE TestYelp;
+
+SELECT M.marker
+FROM YelpCheckinDateMarkers M, YelpCheckin C, C.checkin_times CT, CT.dates D
+WHERE D /*+ indexnl */ = M.date
+      AND C.business_id = "--Ni3oJ4VOqfOEu7Sj2Vzg";
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/join-unnest-queries/use-case-4/use-case-4.6.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/join-unnest-queries/use-case-4/use-case-4.6.query.sqlpp
new file mode 100644
index 0000000..cc884df
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/join-unnest-queries/use-case-4/use-case-4.6.query.sqlpp
@@ -0,0 +1,27 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+SET `compiler.arrayindex` "true";
+
+USE TestYelp;
+
+SELECT M.marker
+FROM YelpCheckinDateMarkers M, YelpCheckin C, C.checkin_times CT, CT.dates D
+WHERE D /*+ indexnl */ = M.date
+      AND CT.checkin_times_id = 1
+      AND C.business_id = "--Ni3oJ4VOqfOEu7Sj2Vzg";
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/join-unnest-queries/use-case-4/use-case-4.7.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/join-unnest-queries/use-case-4/use-case-4.7.query.sqlpp
new file mode 100644
index 0000000..7e6c12b
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/join-unnest-queries/use-case-4/use-case-4.7.query.sqlpp
@@ -0,0 +1,26 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+SET `compiler.arrayindex` "true";
+
+USE TestYelp;
+
+SELECT M.marker
+FROM YelpCheckinDateMarkers M, YelpCheckin C, C.checkin_times CT, CT.dates D
+WHERE D /*+ indexnl */ = M.date
+      AND M.time = "19:49:16";
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/join-unnest-queries/use-case-4/use-case-4.8.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/join-unnest-queries/use-case-4/use-case-4.8.query.sqlpp
new file mode 100644
index 0000000..5f6ed5c
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/join-unnest-queries/use-case-4/use-case-4.8.query.sqlpp
@@ -0,0 +1,28 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+SET `compiler.arrayindex` "true";
+
+USE TestYelp;
+
+SELECT COUNT(*)
+FROM YelpCheckinDateMarkers M
+INNER JOIN (
+    SELECT VALUE D
+    FROM YelpCheckin C, C.checkin_times CT, CT.dates D
+) AS CD ON CD /*+ indexnl */ = M.date;
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/join-unnest-queries/with-open-index/with-open-index.1.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/join-unnest-queries/with-open-index/with-open-index.1.ddl.sqlpp
new file mode 100644
index 0000000..039df8c
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/join-unnest-queries/with-open-index/with-open-index.1.ddl.sqlpp
@@ -0,0 +1,43 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+/*
+ * Description: Verify that we are able to correctly perform the following types of queries using an array->atomic index.
+ *     1) Implicit INNER JOIN query, with additional predicate on inner collection before the unnest.
+ *     2) Implicit INNER JOIN query, with additional predicate on the outer collection.
+ *     3) Explicit INNER JOIN query using UNNEST subquery, no additional predicate.
+ *     TODO (GLENN) 4) Explicit LEFT OUTER JOIN query using UNNEST subquery, no additional predicate.
+ */
+
+DROP DATAVERSE TestYelp IF EXISTS;
+CREATE DATAVERSE TestYelp;
+USE TestYelp;
+
+CREATE TYPE CheckinType AS {
+    checkin_id: uuid
+};
+CREATE TYPE CheckinDatesToMarkersType AS {
+    marker: string,
+    datetime: string,
+    date: string,
+    time: string
+};
+
+CREATE DATASET YelpCheckin(CheckinType) PRIMARY KEY checkin_id AUTOGENERATED;
+CREATE DATASET YelpCheckinDateMarkers (CheckinDatesToMarkersType) PRIMARY KEY marker;
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/join-unnest-queries/with-open-index/with-open-index.2.update.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/join-unnest-queries/with-open-index/with-open-index.2.update.sqlpp
new file mode 100644
index 0000000..b75bf1b
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/join-unnest-queries/with-open-index/with-open-index.2.update.sqlpp
@@ -0,0 +1,276 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+USE TestYelp;
+
+INSERT INTO YelpCheckin (
+    [
+        {
+            "business_id": "--1UhMGODdWsrMastO9DZw",
+            "dates": [
+                "2016-04-26 19:49:16",
+                "2016-08-30 18:36:57",
+                "2016-10-15 02:45:18",
+                "2016-11-18 01:54:50",
+                "2017-04-20 18:39:06",
+                "2017-05-03 17:58:02",
+                "2019-03-19 22:04:48"
+            ]
+        },
+        {
+            "business_id": "--EF5N7P70J_UYBTPypYlA",
+            "dates": [
+                "2018-05-25 19:52:07",
+                "2018-09-18 16:09:44",
+                "2019-10-18 21:29:09"
+            ]
+        },
+        {
+            "business_id": "--Ni3oJ4VOqfOEu7Sj2Vzg",
+            "dates": [
+                "2019-06-07 17:54:58"
+            ]
+        },
+        {
+            "business_id": "--Y1Adl1YUWfYIRSd8vkmA",
+            "dates": [
+                "2011-05-03 20:54:05",
+                "2011-08-23 20:49:45",
+                "2014-12-04 06:13:01",
+                "2016-11-16 19:25:55"
+            ]
+        },
+        {
+            "business_id": "--YPwqIlRJrhHkJcjY3eiA",
+            "dates": [
+                "2016-06-18 21:35:45",
+                "2016-10-15 18:17:51"
+            ]
+        },
+        {
+            "business_id": "--e8PjCNhEz32pprnPhCwQ",
+            "dates": [
+                "2015-04-02 21:45:17"
+            ]
+        },
+        {
+            "business_id": "--kinfHwmtdjz03g8B8z8Q",
+            "dates": [
+                "2014-08-27 17:49:18",
+                "2015-12-19 21:30:31",
+                "2018-11-27 15:53:50"
+            ]
+        },
+        {
+            "business_id": "--q6datkI-f0EoVheXNEeQ",
+            "dates": [
+                "2014-01-28 20:56:04",
+                "2014-11-16 16:11:58",
+                "2015-11-15 19:21:53",
+                "2015-11-15 19:33:39"
+            ]
+        },
+        {
+            "business_id": "--qvQS4MigHPykD2GV0-zw",
+            "dates": [
+                "2019-04-11 18:30:12"
+            ]
+        },
+        {
+            "business_id": "--wIGbLEhlpl_UeAIyDmZQ",
+            "dates": [
+                "2015-06-06 20:01:06",
+                "2019-03-14 22:01:52"
+            ]
+        },
+        {
+            "business_id": "-0FA-Qdi3SPYIoJz9UQw-A",
+            "dates": [
+                "2018-09-29 18:55:17",
+                "2018-10-20 16:48:05",
+                "2018-10-20 22:20:24"
+            ]
+        },
+        {
+            "business_id": "-0Hj1hb_XW6ybWq2M7QhGA",
+            "dates": [
+                "2011-04-23 21:11:22",
+                "2014-05-04 19:42:48",
+                "2014-05-11 19:16:08",
+                "2014-06-04 19:14:18",
+                "2015-12-05 19:22:42",
+                "2017-05-15 23:19:00"
+            ]
+        },
+        {
+            "business_id": "-0KMvRFwDWdVBeTpT11iHw",
+            "dates": [
+                "2012-07-13 21:43:57",
+                "2016-12-24 02:27:31",
+                "2017-08-31 00:35:26"
+            ]
+        },
+        {
+            "business_id": "-0LPtgJC31FWMrMv317p0Q",
+            "dates": [
+                "2013-04-13 12:35:33",
+                "2013-08-19 23:35:49",
+                "2013-10-04 19:14:56"
+            ]
+        },
+        {
+            "business_id": "-0M3o2uWBnQZwd3hmfEwuw",
+            "dates": [
+                "2016-09-10 19:26:19",
+                "2018-09-08 14:15:37",
+                "2019-09-13 22:47:25"
+            ]
+        },
+        {
+            "business_id": "-0RRiWDtfnS16AKCtfvBZg",
+            "dates": [
+                "2017-05-19 14:30:16",
+                "2017-05-19 14:30:25",
+                "2017-08-28 15:49:37",
+                "2017-09-20 20:19:51",
+                "2017-10-01 16:31:05",
+                "2017-10-01 16:56:27",
+                "2017-12-27 23:33:20"
+            ]
+        },
+        {
+            "business_id": "-0Soj75v-XoRcf2ERr8Bmg",
+            "dates": [
+                "2019-06-05 18:22:49"
+            ]
+        },
+        {
+            "business_id": "-0ZumLlFjMh4ZW1z2nXGug",
+            "dates": [
+                "2011-09-24 21:37:32",
+                "2014-03-10 20:20:07",
+                "2015-05-27 00:40:24",
+                "2015-08-29 17:58:15",
+                "2018-03-16 15:03:26"
+            ]
+        },
+        {
+            "business_id": "-0aOudcaAyac0VJbMX-L1g",
+            "dates": [
+                "2015-03-16 23:51:16",
+                "2015-12-21 04:48:01",
+                "2016-10-28 20:22:42",
+                "2016-10-28 20:23:00"
+            ]
+        },
+        {
+            "business_id": "-0b86isaXMY0v4g-V8GZ9Q",
+            "dates": [
+                "2013-10-22 16:49:21",
+                "2014-11-21 17:39:24"
+            ]
+        },
+        {
+            "business_id": "-0d-BfFSU0bwLcnMaGRxYw",
+            "dates": [
+                "2014-08-07 18:30:48",
+                "2014-09-16 20:41:45",
+                "2014-10-12 23:22:27",
+                "2015-07-21 20:43:56",
+                "2015-07-21 20:45:07"
+            ]
+        },
+        {
+            "business_id": "-0jz6c3C6i7RG7Ag22K-Pg",
+            "dates": [
+                "2015-05-02 19:49:05",
+                "2015-05-06 03:52:18",
+                "2015-09-26 01:13:19"
+            ]
+        },
+        {
+            "business_id": "-0y3MZU2oYP8r1ruDP1bfQ",
+            "dates": [
+                "2015-04-11 13:14:14",
+                "2015-11-21 16:05:56",
+                "2016-05-06 14:10:04",
+                "2017-08-09 15:15:10",
+                "2017-10-21 15:12:56"
+            ]
+        },
+        {
+            "business_id": "-1BPe8UjF2_l3nVk-DFUjA",
+            "dates": [
+                "2015-12-03 18:44:00",
+                "2016-03-17 18:19:21",
+                "2016-11-02 15:58:38"
+            ]
+        },
+        {
+            "business_id": "-1E2CQu_38mkghvmZgCCRw",
+            "dates": [
+                "2019-04-04 22:02:37"
+            ]
+        },
+        {
+            "business_id": "-1wzk43IZ5D9Ysu6kzb5xA",
+            "dates": [
+                "2019-02-27 14:03:08"
+            ]
+        },
+        {
+            "business_id": "-23R9P2eG7VTc6DVLjFKzA",
+            "dates": [
+                "2011-12-21 19:02:51",
+                "2012-04-15 04:21:39",
+                "2012-04-15 14:23:56",
+                "2013-06-30 22:39:51",
+                "2013-10-04 20:34:13",
+                "2014-07-16 02:28:40"
+            ]
+        },
+        {
+            "business_id": "-26MGfikhJiTfCI-GqmzhQ",
+            "dates": [
+                "2018-06-13 20:16:07"
+            ]
+        },
+        {
+            "business_id": "-2bLuJsMZ0WhI9daurVQNQ",
+            "dates": [
+                "2015-05-29 16:46:17",
+                "2015-06-01 15:03:53"
+            ]
+        },
+        {
+            "business_id": "-2hDBMaza_ldqnZdiU06LQ",
+            "dates": [
+                "2011-10-08 12:02:23",
+                "2014-08-18 02:11:11",
+                "2016-01-07 05:27:51",
+                "2016-10-21 20:15:55",
+                "2016-12-01 03:57:10",
+                "2016-12-29 01:54:42",
+                "2018-07-22 19:55:31",
+                "2018-09-07 01:42:54",
+                "2019-03-08 03:41:06"
+            ]
+        }
+    ]
+);
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/join-unnest-queries/with-open-index/with-open-index.3.update.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/join-unnest-queries/with-open-index/with-open-index.3.update.sqlpp
new file mode 100644
index 0000000..c1eca5d
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/join-unnest-queries/with-open-index/with-open-index.3.update.sqlpp
@@ -0,0 +1,132 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+USE TestYelp;
+
+INSERT INTO YelpCheckinDateMarkers (
+    [
+        { "marker": "4fa1e250-7a68-3adb-04ec-6569c36882f5", "datetime": "2016-04-26 19:49:16", "date": "2016-04-26", "time": "19:49:16" },
+        { "marker": "4fa1e250-7a68-3adb-04ec-6569c36882f6", "datetime": "2016-08-30 18:36:57", "date": "2016-08-30", "time": "18:36:57" },
+        { "marker": "4fa1e250-7a68-3adb-04ec-6569c36882f7", "datetime": "2016-10-15 02:45:18", "date": "2016-10-15", "time": "02:45:18" },
+        { "marker": "4fa1e250-7a68-3adb-04ec-6569c36882f8", "datetime": "2016-11-18 01:54:50", "date": "2016-11-18", "time": "01:54:50" },
+        { "marker": "4fa1e250-7a68-3adb-04ec-6569c36882f9", "datetime": "2017-04-20 18:39:06", "date": "2017-04-20", "time": "18:39:06" },
+        { "marker": "4fa1e250-7a68-3adb-04ec-6569c36882fa", "datetime": "2017-05-03 17:58:02", "date": "2017-05-03", "time": "17:58:02" },
+        { "marker": "4fa1e250-7a68-3adb-04ec-6569c36882fb", "datetime": "2019-03-19 22:04:48", "date": "2019-03-19", "time": "22:04:48" },
+        { "marker": "4fa1e250-7a68-3adb-04ec-6569c36882fc", "datetime": "2018-05-25 19:52:07", "date": "2018-05-25", "time": "19:52:07" },
+        { "marker": "4fa1e250-7a68-3adb-04ec-6569c36882fd", "datetime": "2018-09-18 16:09:44", "date": "2018-09-18", "time": "16:09:44" },
+        { "marker": "4fa1e250-7a68-3adb-04ec-6569c36882fe", "datetime": "2019-10-18 21:29:09", "date": "2019-10-18", "time": "21:29:09" },
+        { "marker": "4fa1e250-7a68-3adb-04ec-6569c36882ff", "datetime": "2011-05-03 20:54:05", "date": "2011-05-03", "time": "20:54:05" },
+        { "marker": "4fa1e250-7a68-3adb-04ec-6569c3688300", "datetime": "2011-08-23 20:49:45", "date": "2011-08-23", "time": "20:49:45" },
+        { "marker": "4fa1e250-7a68-3adb-04ec-6569c3688301", "datetime": "2014-12-04 06:13:01", "date": "2014-12-04", "time": "06:13:01" },
+        { "marker": "4fa1e250-7a68-3adb-04ec-6569c3688302", "datetime": "2016-11-16 19:25:55", "date": "2016-11-16", "time": "19:25:55" },
+        { "marker": "4fa1e250-7a68-3adb-04ec-6569c3688303", "datetime": "2016-06-18 21:35:45", "date": "2016-06-18", "time": "21:35:45" },
+        { "marker": "4fa1e250-7a68-3adb-04ec-6569c3688304", "datetime": "2016-10-15 18:17:51", "date": "2016-10-15", "time": "18:17:51" },
+        { "marker": "4fa1e250-7a68-3adb-04ec-6569c3688305", "datetime": "2014-08-27 17:49:18", "date": "2014-08-27", "time": "17:49:18" },
+        { "marker": "4fa1e250-7a68-3adb-04ec-6569c3688306", "datetime": "2015-12-19 21:30:31", "date": "2015-12-19", "time": "21:30:31" },
+        { "marker": "4fa1e250-7a68-3adb-04ec-6569c3688307", "datetime": "2018-11-27 15:53:50", "date": "2018-11-27", "time": "15:53:50" },
+        { "marker": "4fa1e250-7a68-3adb-04ec-6569c3688308", "datetime": "2019-04-11 18:30:12", "date": "2019-04-11", "time": "18:30:12" },
+        { "marker": "4fa1e250-7a68-3adb-04ec-6569c3688309", "datetime": "2015-06-06 20:01:06", "date": "2015-06-06", "time": "20:01:06" },
+        { "marker": "4fa1e250-7a68-3adb-04ec-6569c368830a", "datetime": "2019-03-14 22:01:52", "date": "2019-03-14", "time": "22:01:52" },
+        { "marker": "4fa1e250-7a68-3adb-04ec-6569c368830b", "datetime": "2012-07-13 21:43:57", "date": "2012-07-13", "time": "21:43:57" },
+        { "marker": "4fa1e250-7a68-3adb-04ec-6569c368830c", "datetime": "2016-12-24 02:27:31", "date": "2016-12-24", "time": "02:27:31" },
+        { "marker": "4fa1e250-7a68-3adb-04ec-6569c368830d", "datetime": "2017-08-31 00:35:26", "date": "2017-08-31", "time": "00:35:26" },
+        { "marker": "4fa1e250-7a68-3adb-04ec-6569c368830e", "datetime": "2013-04-13 12:35:33", "date": "2013-04-13", "time": "12:35:33" },
+        { "marker": "4fa1e250-7a68-3adb-04ec-6569c368830f", "datetime": "2013-08-19 23:35:49", "date": "2013-08-19", "time": "23:35:49" },
+        { "marker": "4fa1e250-7a68-3adb-04ec-6569c3688310", "datetime": "2013-10-04 19:14:56", "date": "2013-10-04", "time": "19:14:56" },
+        { "marker": "4fa1e250-7a68-3adb-04ec-6569c3688311", "datetime": "2016-09-10 19:26:19", "date": "2016-09-10", "time": "19:26:19" },
+        { "marker": "4fa1e250-7a68-3adb-04ec-6569c3688312", "datetime": "2018-09-08 14:15:37", "date": "2018-09-08", "time": "14:15:37" },
+        { "marker": "4fa1e250-7a68-3adb-04ec-6569c3688313", "datetime": "2019-09-13 22:47:25", "date": "2019-09-13", "time": "22:47:25" },
+        { "marker": "4fa1e250-7a68-3adb-04ec-6569c3688314", "datetime": "2019-06-05 18:22:49", "date": "2019-06-05", "time": "18:22:49" },
+        { "marker": "4fa1e250-7a68-3adb-04ec-6569c3688315", "datetime": "2015-03-16 23:51:16", "date": "2015-03-16", "time": "23:51:16" },
+        { "marker": "4fa1e250-7a68-3adb-04ec-6569c3688316", "datetime": "2015-12-21 04:48:01", "date": "2015-12-21", "time": "04:48:01" },
+        { "marker": "4fa1e250-7a68-3adb-04ec-6569c3688317", "datetime": "2016-10-28 20:22:42", "date": "2016-10-28", "time": "20:22:42" },
+        { "marker": "4fa1e250-7a68-3adb-04ec-6569c3688318", "datetime": "2016-10-28 20:23:00", "date": "2016-10-28", "time": "20:23:00" },
+        { "marker": "4fa1e250-7a68-3adb-04ec-6569c3688319", "datetime": "2014-08-07 18:30:48", "date": "2014-08-07", "time": "18:30:48" },
+        { "marker": "4fa1e250-7a68-3adb-04ec-6569c368831a", "datetime": "2014-09-16 20:41:45", "date": "2014-09-16", "time": "20:41:45" },
+        { "marker": "4fa1e250-7a68-3adb-04ec-6569c368831b", "datetime": "2014-10-12 23:22:27", "date": "2014-10-12", "time": "23:22:27" },
+        { "marker": "4fa1e250-7a68-3adb-04ec-6569c368831c", "datetime": "2015-07-21 20:43:56", "date": "2015-07-21", "time": "20:43:56" },
+        { "marker": "4fa1e250-7a68-3adb-04ec-6569c368831d", "datetime": "2015-07-21 20:45:07", "date": "2015-07-21", "time": "20:45:07" },
+        { "marker": "4fa1e250-7a68-3adb-04ec-6569c368831e", "datetime": "2015-05-02 19:49:05", "date": "2015-05-02", "time": "19:49:05" },
+        { "marker": "4fa1e250-7a68-3adb-04ec-6569c368831f", "datetime": "2015-05-06 03:52:18", "date": "2015-05-06", "time": "03:52:18" },
+        { "marker": "4fa1e250-7a68-3adb-04ec-6569c3688320", "datetime": "2015-09-26 01:13:19", "date": "2015-09-26", "time": "01:13:19" },
+        { "marker": "4fa1e250-7a68-3adb-04ec-6569c3688321", "datetime": "2019-04-04 22:02:37", "date": "2019-04-04", "time": "22:02:37" },
+        { "marker": "4fa1e250-7a68-3adb-04ec-6569c3688322", "datetime": "2011-12-21 19:02:51", "date": "2011-12-21", "time": "19:02:51" },
+        { "marker": "4fa1e250-7a68-3adb-04ec-6569c3688323", "datetime": "2012-04-15 04:21:39", "date": "2012-04-15", "time": "04:21:39" },
+        { "marker": "4fa1e250-7a68-3adb-04ec-6569c3688324", "datetime": "2012-04-15 14:23:56", "date": "2012-04-15", "time": "14:23:56" },
+        { "marker": "4fa1e250-7a68-3adb-04ec-6569c3688325", "datetime": "2013-06-30 22:39:51", "date": "2013-06-30", "time": "22:39:51" },
+        { "marker": "4fa1e250-7a68-3adb-04ec-6569c3688326", "datetime": "2013-10-04 20:34:13", "date": "2013-10-04", "time": "20:34:13" },
+        { "marker": "4fa1e250-7a68-3adb-04ec-6569c3688327", "datetime": "2014-07-16 02:28:40", "date": "2014-07-16", "time": "02:28:40" },
+        { "marker": "4fa1e250-7a68-3adb-04ec-6569c3688328", "datetime": "2018-06-13 20:16:07", "date": "2018-06-13", "time": "20:16:07" },
+        { "marker": "4fa1e250-7a68-3adb-04ec-6569c3688329", "datetime": "2011-10-08 12:02:23", "date": "2011-10-08", "time": "12:02:23" },
+        { "marker": "4fa1e250-7a68-3adb-04ec-6569c368832a", "datetime": "2014-08-18 02:11:11", "date": "2014-08-18", "time": "02:11:11" },
+        { "marker": "4fa1e250-7a68-3adb-04ec-6569c368832b", "datetime": "2016-01-07 05:27:51", "date": "2016-01-07", "time": "05:27:51" },
+        { "marker": "4fa1e250-7a68-3adb-04ec-6569c368832c", "datetime": "2016-10-21 20:15:55", "date": "2016-10-21", "time": "20:15:55" },
+        { "marker": "4fa1e250-7a68-3adb-04ec-6569c368832d", "datetime": "2016-12-01 03:57:10", "date": "2016-12-01", "time": "03:57:10" },
+        { "marker": "4fa1e250-7a68-3adb-04ec-6569c368832e", "datetime": "2016-12-29 01:54:42", "date": "2016-12-29", "time": "01:54:42" },
+        { "marker": "4fa1e250-7a68-3adb-04ec-6569c368832f", "datetime": "2018-07-22 19:55:31", "date": "2018-07-22", "time": "19:55:31" },
+        { "marker": "4fa1e250-7a68-3adb-04ec-6569c3688330", "datetime": "2018-09-07 01:42:54", "date": "2018-09-07", "time": "01:42:54" },
+        { "marker": "4fa1e250-7a68-3adb-04ec-6569c3688331", "datetime": "2019-03-08 03:41:06", "date": "2019-03-08", "time": "03:41:06" },
+        { "marker": "991ed2f7-3374-33a5-ee65-77f067848381", "datetime": "2019-06-07 17:54:58", "date": "2019-06-07", "time": "17:54:58" },
+        { "marker": "991ed2f7-3374-33a5-ee65-77f067848382", "datetime": "2015-04-02 21:45:17", "date": "2015-04-02", "time": "21:45:17" },
+        { "marker": "991ed2f7-3374-33a5-ee65-77f067848383", "datetime": "2014-01-28 20:56:04", "date": "2014-01-28", "time": "20:56:04" },
+        { "marker": "991ed2f7-3374-33a5-ee65-77f067848384", "datetime": "2014-11-16 16:11:58", "date": "2014-11-16", "time": "16:11:58" },
+        { "marker": "991ed2f7-3374-33a5-ee65-77f067848385", "datetime": "2015-11-15 19:21:53", "date": "2015-11-15", "time": "19:21:53" },
+        { "marker": "991ed2f7-3374-33a5-ee65-77f067848386", "datetime": "2015-11-15 19:33:39", "date": "2015-11-15", "time": "19:33:39" },
+        { "marker": "991ed2f7-3374-33a5-ee65-77f067848387", "datetime": "2018-09-29 18:55:17", "date": "2018-09-29", "time": "18:55:17" },
+        { "marker": "991ed2f7-3374-33a5-ee65-77f067848388", "datetime": "2018-10-20 16:48:05", "date": "2018-10-20", "time": "16:48:05" },
+        { "marker": "991ed2f7-3374-33a5-ee65-77f067848389", "datetime": "2018-10-20 22:20:24", "date": "2018-10-20", "time": "22:20:24" },
+        { "marker": "991ed2f7-3374-33a5-ee65-77f06784838a", "datetime": "2011-04-23 21:11:22", "date": "2011-04-23", "time": "21:11:22" },
+        { "marker": "991ed2f7-3374-33a5-ee65-77f06784838b", "datetime": "2014-05-04 19:42:48", "date": "2014-05-04", "time": "19:42:48" },
+        { "marker": "991ed2f7-3374-33a5-ee65-77f06784838c", "datetime": "2014-05-11 19:16:08", "date": "2014-05-11", "time": "19:16:08" },
+        { "marker": "991ed2f7-3374-33a5-ee65-77f06784838d", "datetime": "2014-06-04 19:14:18", "date": "2014-06-04", "time": "19:14:18" },
+        { "marker": "991ed2f7-3374-33a5-ee65-77f06784838e", "datetime": "2015-12-05 19:22:42", "date": "2015-12-05", "time": "19:22:42" },
+        { "marker": "991ed2f7-3374-33a5-ee65-77f06784838f", "datetime": "2017-05-15 23:19:00", "date": "2017-05-15", "time": "23:19:00" },
+        { "marker": "991ed2f7-3374-33a5-ee65-77f067848390", "datetime": "2017-05-19 14:30:16", "date": "2017-05-19", "time": "14:30:16" },
+        { "marker": "991ed2f7-3374-33a5-ee65-77f067848391", "datetime": "2017-05-19 14:30:25", "date": "2017-05-19", "time": "14:30:25" },
+        { "marker": "991ed2f7-3374-33a5-ee65-77f067848392", "datetime": "2017-08-28 15:49:37", "date": "2017-08-28", "time": "15:49:37" },
+        { "marker": "991ed2f7-3374-33a5-ee65-77f067848393", "datetime": "2017-09-20 20:19:51", "date": "2017-09-20", "time": "20:19:51" },
+        { "marker": "991ed2f7-3374-33a5-ee65-77f067848394", "datetime": "2017-10-01 16:31:05", "date": "2017-10-01", "time": "16:31:05" },
+        { "marker": "991ed2f7-3374-33a5-ee65-77f067848395", "datetime": "2017-10-01 16:56:27", "date": "2017-10-01", "time": "16:56:27" },
+        { "marker": "991ed2f7-3374-33a5-ee65-77f067848396", "datetime": "2017-12-27 23:33:20", "date": "2017-12-27", "time": "23:33:20" },
+        { "marker": "991ed2f7-3374-33a5-ee65-77f067848397", "datetime": "2011-09-24 21:37:32", "date": "2011-09-24", "time": "21:37:32" },
+        { "marker": "991ed2f7-3374-33a5-ee65-77f067848398", "datetime": "2014-03-10 20:20:07", "date": "2014-03-10", "time": "20:20:07" },
+        { "marker": "991ed2f7-3374-33a5-ee65-77f067848399", "datetime": "2015-05-27 00:40:24", "date": "2015-05-27", "time": "00:40:24" },
+        { "marker": "991ed2f7-3374-33a5-ee65-77f06784839a", "datetime": "2015-08-29 17:58:15", "date": "2015-08-29", "time": "17:58:15" },
+        { "marker": "991ed2f7-3374-33a5-ee65-77f06784839b", "datetime": "2018-03-16 15:03:26", "date": "2018-03-16", "time": "15:03:26" },
+        { "marker": "991ed2f7-3374-33a5-ee65-77f06784839c", "datetime": "2013-10-22 16:49:21", "date": "2013-10-22", "time": "16:49:21" },
+        { "marker": "991ed2f7-3374-33a5-ee65-77f06784839d", "datetime": "2014-11-21 17:39:24", "date": "2014-11-21", "time": "17:39:24" },
+        { "marker": "991ed2f7-3374-33a5-ee65-77f06784839e", "datetime": "2015-04-11 13:14:14", "date": "2015-04-11", "time": "13:14:14" },
+        { "marker": "991ed2f7-3374-33a5-ee65-77f06784839f", "datetime": "2015-11-21 16:05:56", "date": "2015-11-21", "time": "16:05:56" },
+        { "marker": "991ed2f7-3374-33a5-ee65-77f0678483a0", "datetime": "2016-05-06 14:10:04", "date": "2016-05-06", "time": "14:10:04" },
+        { "marker": "991ed2f7-3374-33a5-ee65-77f0678483a1", "datetime": "2017-08-09 15:15:10", "date": "2017-08-09", "time": "15:15:10" },
+        { "marker": "991ed2f7-3374-33a5-ee65-77f0678483a2", "datetime": "2017-10-21 15:12:56", "date": "2017-10-21", "time": "15:12:56" },
+        { "marker": "991ed2f7-3374-33a5-ee65-77f0678483a3", "datetime": "2015-12-03 18:44:00", "date": "2015-12-03", "time": "18:44:00" },
+        { "marker": "991ed2f7-3374-33a5-ee65-77f0678483a4", "datetime": "2016-03-17 18:19:21", "date": "2016-03-17", "time": "18:19:21" },
+        { "marker": "991ed2f7-3374-33a5-ee65-77f0678483a5", "datetime": "2016-11-02 15:58:38", "date": "2016-11-02", "time": "15:58:38" },
+        { "marker": "991ed2f7-3374-33a5-ee65-77f0678483a6", "datetime": "2019-02-27 14:03:08", "date": "2019-02-27", "time": "14:03:08" },
+        { "marker": "991ed2f7-3374-33a5-ee65-77f0678483a7", "datetime": "2015-05-29 16:46:17", "date": "2015-05-29", "time": "16:46:17" },
+        { "marker": "991ed2f7-3374-33a5-ee65-77f0678483a8", "datetime": "2015-06-01 15:03:53", "date": "2015-06-01", "time": "15:03:53" },
+        { "marker": "BAD_ENTRY_1", "datetime": "NA", "date": "NA", "time": "NA" },
+        { "marker": "BAD_ENTRY_2", "datetime": "NA", "date": "NA", "time": "NA" },
+        { "marker": "BAD_ENTRY_3", "datetime": "NA", "date": "NA", "time": "NA" },
+        { "marker": "BAD_ENTRY_4", "datetime": "NA", "date": "NA", "time": "NA" },
+        { "marker": "BAD_ENTRY_5", "datetime": "NA", "date": "NA", "time": "NA" },
+        { "marker": "BAD_ENTRY_6", "datetime": "NA", "date": "NA", "time": "NA" }
+    ]
+);
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/join-unnest-queries/with-open-index/with-open-index.4.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/join-unnest-queries/with-open-index/with-open-index.4.ddl.sqlpp
new file mode 100644
index 0000000..b933d8a
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/join-unnest-queries/with-open-index/with-open-index.4.ddl.sqlpp
@@ -0,0 +1,22 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+USE TestYelp;
+
+CREATE INDEX IdxYelpCheckinDates ON YelpCheckin (UNNEST dates : string ?) ;
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/join-unnest-queries/with-open-index/with-open-index.5.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/join-unnest-queries/with-open-index/with-open-index.5.query.sqlpp
new file mode 100644
index 0000000..bb9eea4
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/join-unnest-queries/with-open-index/with-open-index.5.query.sqlpp
@@ -0,0 +1,26 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+SET `compiler.arrayindex` "true";
+
+USE TestYelp;
+
+SELECT M.marker
+FROM YelpCheckinDateMarkers M, YelpCheckin C, C.dates D
+WHERE D /*+ indexnl */ = M.datetime
+      AND C.business_id = "--Ni3oJ4VOqfOEu7Sj2Vzg";
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/join-unnest-queries/with-open-index/with-open-index.6.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/join-unnest-queries/with-open-index/with-open-index.6.query.sqlpp
new file mode 100644
index 0000000..a66da8f
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/join-unnest-queries/with-open-index/with-open-index.6.query.sqlpp
@@ -0,0 +1,26 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+SET `compiler.arrayindex` "true";
+
+USE TestYelp;
+
+SELECT M.marker
+FROM YelpCheckinDateMarkers M, YelpCheckin C, C.dates D
+WHERE D /*+ indexnl */ = M.datetime
+      AND M.time = "19:49:16";
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/join-unnest-queries/with-open-index/with-open-index.7.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/join-unnest-queries/with-open-index/with-open-index.7.query.sqlpp
new file mode 100644
index 0000000..8bdf70e
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/join-unnest-queries/with-open-index/with-open-index.7.query.sqlpp
@@ -0,0 +1,28 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+SET `compiler.arrayindex` "true";
+
+USE TestYelp;
+
+SELECT COUNT(*)
+FROM YelpCheckinDateMarkers M
+INNER JOIN (
+    SELECT VALUE D
+    FROM YelpCheckin C, C.dates D
+) AS CD ON CD /*+ indexnl */ = M.datetime;
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/metadata/closed/use-case-1/use-case-1.1.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/metadata/closed/use-case-1/use-case-1.1.ddl.sqlpp
new file mode 100644
index 0000000..5e18a83
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/metadata/closed/use-case-1/use-case-1.1.ddl.sqlpp
@@ -0,0 +1,37 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+/*
+ * Description: Verify the index metadata for an array->atomic index. 
+ */
+
+DROP DATAVERSE TestYelp IF EXISTS;
+CREATE DATAVERSE TestYelp;
+USE TestYelp;
+
+CREATE TYPE CheckinType AS {
+    checkin_id: uuid,
+    business_id: string,
+    dates: [string]
+};
+
+CREATE DATASET YelpCheckin(CheckinType) PRIMARY KEY checkin_id AUTOGENERATED;
+CREATE INDEX IdxYelpCheckinBusinessID ON YelpCheckin (business_id);
+CREATE INDEX IdxYelpCheckinDates ON YelpCheckin (UNNEST dates);
+
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/metadata/closed/use-case-1/use-case-1.2.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/metadata/closed/use-case-1/use-case-1.2.query.sqlpp
new file mode 100644
index 0000000..8bb71f2
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/metadata/closed/use-case-1/use-case-1.2.query.sqlpp
@@ -0,0 +1,22 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+SELECT D.SearchKey, D.SearchKeyElements, D.SearchKeyType
+FROM Metadata.`Index` D
+WHERE D.IndexName = "IdxYelpCheckinDates" AND D.DataverseName = "TestYelp";
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/metadata/closed/use-case-2/use-case-2.1.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/metadata/closed/use-case-2/use-case-2.1.ddl.sqlpp
new file mode 100644
index 0000000..fcdcde7
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/metadata/closed/use-case-2/use-case-2.1.ddl.sqlpp
@@ -0,0 +1,39 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+/*
+ * Description: Verify the index metadata for a record->array->atomic index. 
+ */
+
+DROP DATAVERSE TestYelp IF EXISTS;
+CREATE DATAVERSE TestYelp;
+USE TestYelp;
+
+CREATE TYPE CheckinType AS {
+    checkin_id: uuid,
+    business_id: string,
+    checkin_times: {
+        dates: [string],
+        times: [string]
+    }
+};
+
+CREATE DATASET YelpCheckin(CheckinType) PRIMARY KEY checkin_id AUTOGENERATED;
+CREATE INDEX IdxYelpCheckinBusinessID ON YelpCheckin (business_id);
+CREATE INDEX IdxYelpCheckinDates ON YelpCheckin (UNNEST checkin_times.dates);
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/metadata/closed/use-case-2/use-case-2.2.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/metadata/closed/use-case-2/use-case-2.2.query.sqlpp
new file mode 100644
index 0000000..8bb71f2
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/metadata/closed/use-case-2/use-case-2.2.query.sqlpp
@@ -0,0 +1,22 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+SELECT D.SearchKey, D.SearchKeyElements, D.SearchKeyType
+FROM Metadata.`Index` D
+WHERE D.IndexName = "IdxYelpCheckinDates" AND D.DataverseName = "TestYelp";
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/metadata/closed/use-case-3/use-case-3.1.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/metadata/closed/use-case-3/use-case-3.1.ddl.sqlpp
new file mode 100644
index 0000000..4b98d54
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/metadata/closed/use-case-3/use-case-3.1.ddl.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.
+ */
+
+/*
+ * Description: Verify the index metadata for an array->record->atomic index, and a composite two-field array index. 
+ */
+
+DROP DATAVERSE TestYelp IF EXISTS;
+CREATE DATAVERSE TestYelp;
+USE TestYelp;
+
+CREATE TYPE CheckinType AS {
+    checkin_id: uuid,
+    business_id: string,
+    checkin_times: [{
+        date: string,
+        time: string
+    }]
+};
+
+CREATE DATASET YelpCheckin(CheckinType) PRIMARY KEY checkin_id AUTOGENERATED;
+CREATE INDEX IdxYelpCheckinBusinessID ON YelpCheckin (business_id);
+CREATE INDEX IdxYelpCheckinDate ON YelpCheckin (UNNEST checkin_times SELECT date);
+CREATE INDEX IdxYelpCheckinDateTime ON YelpCheckin (UNNEST checkin_times SELECT date, time);
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/metadata/closed/use-case-3/use-case-3.2.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/metadata/closed/use-case-3/use-case-3.2.query.sqlpp
new file mode 100644
index 0000000..145f107
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/metadata/closed/use-case-3/use-case-3.2.query.sqlpp
@@ -0,0 +1,23 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+SELECT D.SearchKey, D.SearchKeyElements, D.SearchKeyType
+FROM Metadata.`Index` D
+WHERE D.IndexName = "IdxYelpCheckinDate" AND
+      D.DataverseName = "TestYelp";
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/metadata/closed/use-case-3/use-case-3.3.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/metadata/closed/use-case-3/use-case-3.3.query.sqlpp
new file mode 100644
index 0000000..20d1175
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/metadata/closed/use-case-3/use-case-3.3.query.sqlpp
@@ -0,0 +1,24 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+SELECT D.SearchKey, D.SearchKeyElements, D.SearchKeyType
+FROM Metadata.`Index` D
+WHERE D.IndexName = "IdxYelpCheckinDateTime" AND
+      D.DataverseName = "TestYelp";
+
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/metadata/closed/use-case-4/use-case-4.1.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/metadata/closed/use-case-4/use-case-4.1.ddl.sqlpp
new file mode 100644
index 0000000..205d0bb
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/metadata/closed/use-case-4/use-case-4.1.ddl.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.
+ */
+
+/*
+ * Description: Verify the index metadata for a array->record->array->atomic index. 
+ */
+
+DROP DATAVERSE TestYelp IF EXISTS;
+CREATE DATAVERSE TestYelp;
+USE TestYelp;
+
+CREATE TYPE CheckinType AS {
+    checkin_id: uuid,
+    business_id: string,
+    checkin_times: [{
+        dates: [string],
+        times: [string]
+    }]
+};
+
+CREATE DATASET YelpCheckin(CheckinType) PRIMARY KEY checkin_id AUTOGENERATED;
+CREATE INDEX IdxYelpCheckinBusinessID ON YelpCheckin (business_id);
+CREATE INDEX IdxYelpCheckinDates ON YelpCheckin (UNNEST checkin_times UNNEST dates);
+
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/metadata/closed/use-case-4/use-case-4.2.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/metadata/closed/use-case-4/use-case-4.2.query.sqlpp
new file mode 100644
index 0000000..822f970
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/metadata/closed/use-case-4/use-case-4.2.query.sqlpp
@@ -0,0 +1,23 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+SELECT D.SearchKey, D.SearchKeyElements, D.SearchKeyType
+FROM Metadata.`Index` D
+WHERE D.IndexName = "IdxYelpCheckinDates" AND D.DataverseName = "TestYelp";
+
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/metadata/closed/with-3-level-record-path/with-3-level-record-path.1.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/metadata/closed/with-3-level-record-path/with-3-level-record-path.1.ddl.sqlpp
new file mode 100644
index 0000000..22c7899
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/metadata/closed/with-3-level-record-path/with-3-level-record-path.1.ddl.sqlpp
@@ -0,0 +1,44 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+/*
+ * Description: Verify the index metadata for a record->record->record->array->atomic index. 
+ */
+
+DROP DATAVERSE TestYelp IF EXISTS;
+CREATE DATAVERSE TestYelp;
+USE TestYelp;
+
+CREATE TYPE CheckinType AS {
+    checkin_id: uuid,
+    business_id: string,
+    checkin_data: {
+        checkin_temporal: {
+            checkin_times: {
+                dates: [string],
+                times: [string]
+            }
+        }
+    }
+};
+
+CREATE DATASET YelpCheckin(CheckinType) PRIMARY KEY checkin_id AUTOGENERATED;
+CREATE INDEX IdxYelpCheckinBusinessID ON YelpCheckin (business_id);
+CREATE INDEX IdxYelpCheckinDates ON YelpCheckin (UNNEST checkin_data.checkin_temporal.checkin_times.dates);
+
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/metadata/closed/with-3-level-record-path/with-3-level-record-path.2.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/metadata/closed/with-3-level-record-path/with-3-level-record-path.2.query.sqlpp
new file mode 100644
index 0000000..822f970
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/metadata/closed/with-3-level-record-path/with-3-level-record-path.2.query.sqlpp
@@ -0,0 +1,23 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+SELECT D.SearchKey, D.SearchKeyElements, D.SearchKeyType
+FROM Metadata.`Index` D
+WHERE D.IndexName = "IdxYelpCheckinDates" AND D.DataverseName = "TestYelp";
+
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/metadata/closed/with-composite-array-different-indicators/with-composite-array-different-indicators.1.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/metadata/closed/with-composite-array-different-indicators/with-composite-array-different-indicators.1.ddl.sqlpp
new file mode 100644
index 0000000..27bc78a
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/metadata/closed/with-composite-array-different-indicators/with-composite-array-different-indicators.1.ddl.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.
+ */
+
+/*
+ * Description: Verify the index metadata for a composite array->record->atomic index on two array fields. 
+ */
+
+DROP DATAVERSE TestYelp IF EXISTS;
+CREATE DATAVERSE TestYelp;
+USE TestYelp;
+
+CREATE TYPE CheckinType AS {
+    checkin_id: uuid,
+    business_id: string,
+    checkin_times: [{
+        date: string,
+        t: { time: string }
+    }]
+};
+
+CREATE DATASET YelpCheckin(CheckinType) PRIMARY KEY checkin_id AUTOGENERATED;
+CREATE INDEX IdxYelpCheckinBusinessID ON YelpCheckin (business_id);
+CREATE INDEX IdxYelpCheckinDateTime ON YelpCheckin (UNNEST checkin_times SELECT date, t.time);
+
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/metadata/closed/with-composite-array-different-indicators/with-composite-array-different-indicators.2.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/metadata/closed/with-composite-array-different-indicators/with-composite-array-different-indicators.2.query.sqlpp
new file mode 100644
index 0000000..643dad3
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/metadata/closed/with-composite-array-different-indicators/with-composite-array-different-indicators.2.query.sqlpp
@@ -0,0 +1,23 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+SELECT D.SearchKey, D.SearchKeyElements, D.SearchKeyType
+FROM Metadata.`Index` D
+WHERE D.IndexName = "IdxYelpCheckinDateTime" AND D.DataverseName = "TestYelp";
+
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/metadata/closed/with-composite-sk/with-composite-sk.1.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/metadata/closed/with-composite-sk/with-composite-sk.1.ddl.sqlpp
new file mode 100644
index 0000000..da4ad89
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/metadata/closed/with-composite-sk/with-composite-sk.1.ddl.sqlpp
@@ -0,0 +1,37 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+/*
+ * Description: Verify the index metadata for both a composite atomic, array->atomic and array->atomic, atomic index. 
+ */
+
+DROP DATAVERSE TestYelp IF EXISTS;
+CREATE DATAVERSE TestYelp;
+USE TestYelp;
+
+CREATE TYPE CheckinType AS {
+    checkin_id: uuid,
+    business_id: string,
+    dates: [string]
+};
+
+CREATE DATASET YelpCheckin(CheckinType) PRIMARY KEY checkin_id AUTOGENERATED;
+CREATE INDEX IdxYelpCheckinBusinessID ON YelpCheckin (business_id);
+CREATE INDEX IdxYelpBusinessCheckinDates ON YelpCheckin (business_id, UNNEST dates);
+CREATE INDEX IdxYelpCheckinDatesBusiness ON YelpCheckin (UNNEST dates, business_id);
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/metadata/closed/with-composite-sk/with-composite-sk.2.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/metadata/closed/with-composite-sk/with-composite-sk.2.query.sqlpp
new file mode 100644
index 0000000..6a693ba
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/metadata/closed/with-composite-sk/with-composite-sk.2.query.sqlpp
@@ -0,0 +1,23 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+SELECT D.SearchKey, D.SearchKeyElements, D.SearchKeyType
+FROM Metadata.`Index` D
+WHERE D.IndexName = "IdxYelpBusinessCheckinDates" AND
+      D.DataverseName = "TestYelp";
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/metadata/closed/with-composite-sk/with-composite-sk.3.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/metadata/closed/with-composite-sk/with-composite-sk.3.query.sqlpp
new file mode 100644
index 0000000..688d2a1
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/metadata/closed/with-composite-sk/with-composite-sk.3.query.sqlpp
@@ -0,0 +1,23 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+SELECT D.SearchKey, D.SearchKeyElements, D.SearchKeyType
+FROM Metadata.`Index` D
+WHERE D.IndexName = "IdxYelpCheckinDatesBusiness" AND
+      D.DataverseName = "TestYelp";
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/metadata/open/use-case-1/use-case-1.1.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/metadata/open/use-case-1/use-case-1.1.ddl.sqlpp
new file mode 100644
index 0000000..b52f35a
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/metadata/open/use-case-1/use-case-1.1.ddl.sqlpp
@@ -0,0 +1,35 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+/*
+ * Description: Verify the index metadata for an array->atomic index. 
+ */
+
+DROP DATAVERSE TestYelp IF EXISTS;
+CREATE DATAVERSE TestYelp;
+USE TestYelp;
+
+CREATE TYPE CheckinType AS {
+    checkin_id: uuid
+};
+
+CREATE DATASET YelpCheckin(CheckinType) PRIMARY KEY checkin_id AUTOGENERATED;
+CREATE INDEX IdxYelpCheckinBusinessID ON YelpCheckin (business_id : string?) ;
+CREATE INDEX IdxYelpCheckinDates ON YelpCheckin (UNNEST dates: string?) ;
+
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/metadata/open/use-case-1/use-case-1.2.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/metadata/open/use-case-1/use-case-1.2.query.sqlpp
new file mode 100644
index 0000000..822f970
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/metadata/open/use-case-1/use-case-1.2.query.sqlpp
@@ -0,0 +1,23 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+SELECT D.SearchKey, D.SearchKeyElements, D.SearchKeyType
+FROM Metadata.`Index` D
+WHERE D.IndexName = "IdxYelpCheckinDates" AND D.DataverseName = "TestYelp";
+
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/metadata/open/use-case-2/use-case-2.1.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/metadata/open/use-case-2/use-case-2.1.ddl.sqlpp
new file mode 100644
index 0000000..8881b4d
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/metadata/open/use-case-2/use-case-2.1.ddl.sqlpp
@@ -0,0 +1,35 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+/*
+ * Description: Verify the index metadata for a record->array->atomic index. 
+ */
+
+DROP DATAVERSE TestYelp IF EXISTS;
+CREATE DATAVERSE TestYelp;
+USE TestYelp;
+
+CREATE TYPE CheckinType AS {
+    checkin_id: uuid
+};
+
+CREATE DATASET YelpCheckin(CheckinType) PRIMARY KEY checkin_id AUTOGENERATED;
+CREATE INDEX IdxYelpCheckinBusinessID ON YelpCheckin (business_id : string?) ;
+CREATE INDEX IdxYelpCheckinDates ON YelpCheckin (UNNEST checkin_times.dates : string ?) ;
+
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/metadata/open/use-case-2/use-case-2.2.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/metadata/open/use-case-2/use-case-2.2.query.sqlpp
new file mode 100644
index 0000000..822f970
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/metadata/open/use-case-2/use-case-2.2.query.sqlpp
@@ -0,0 +1,23 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+SELECT D.SearchKey, D.SearchKeyElements, D.SearchKeyType
+FROM Metadata.`Index` D
+WHERE D.IndexName = "IdxYelpCheckinDates" AND D.DataverseName = "TestYelp";
+
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/metadata/open/use-case-3/use-case-3.1.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/metadata/open/use-case-3/use-case-3.1.ddl.sqlpp
new file mode 100644
index 0000000..5839c9b
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/metadata/open/use-case-3/use-case-3.1.ddl.sqlpp
@@ -0,0 +1,35 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+/*
+ * Description: Verify the index metadata for an array->record->atomic index, and a composite two-field array index. 
+ */
+
+DROP DATAVERSE TestYelp IF EXISTS;
+CREATE DATAVERSE TestYelp;
+USE TestYelp;
+
+CREATE TYPE CheckinType AS {
+    checkin_id: uuid
+};
+
+CREATE DATASET YelpCheckin(CheckinType) PRIMARY KEY checkin_id AUTOGENERATED;
+CREATE INDEX IdxYelpCheckinBusinessID ON YelpCheckin (business_id : string ?) ;
+CREATE INDEX IdxYelpCheckinDate ON YelpCheckin (UNNEST checkin_times SELECT date : string ?) ;
+CREATE INDEX IdxYelpCheckinTime ON YelpCheckin (UNNEST checkin_times SELECT time : string ?) ;
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/metadata/open/use-case-3/use-case-3.2.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/metadata/open/use-case-3/use-case-3.2.query.sqlpp
new file mode 100644
index 0000000..f705a94
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/metadata/open/use-case-3/use-case-3.2.query.sqlpp
@@ -0,0 +1,24 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+SELECT D.SearchKey, D.SearchKeyElements, D.SearchKeyType
+FROM Metadata.`Index` D
+WHERE D.IndexName = "IdxYelpCheckinDate" AND
+      D.DataverseName = "TestYelp";
+
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/metadata/open/use-case-3/use-case-3.3.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/metadata/open/use-case-3/use-case-3.3.query.sqlpp
new file mode 100644
index 0000000..fc849a1
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/metadata/open/use-case-3/use-case-3.3.query.sqlpp
@@ -0,0 +1,24 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+SELECT D.SearchKey, D.SearchKeyElements, D.SearchKeyType
+FROM Metadata.`Index` D
+WHERE D.IndexName = "IdxYelpCheckinTime" AND
+      D.DataverseName = "TestYelp";
+
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/metadata/open/use-case-4/use-case-4.1.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/metadata/open/use-case-4/use-case-4.1.ddl.sqlpp
new file mode 100644
index 0000000..ad01847
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/metadata/open/use-case-4/use-case-4.1.ddl.sqlpp
@@ -0,0 +1,35 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+/*
+ * Description: Verify the index metadata for a array->record->array->atomic index. 
+ */
+
+DROP DATAVERSE TestYelp IF EXISTS;
+CREATE DATAVERSE TestYelp;
+USE TestYelp;
+
+CREATE TYPE CheckinType AS {
+    checkin_id: uuid
+};
+
+CREATE DATASET YelpCheckin(CheckinType) PRIMARY KEY checkin_id AUTOGENERATED;
+CREATE INDEX IdxYelpCheckinBusinessID ON YelpCheckin (business_id : string?) ;
+CREATE INDEX IdxYelpCheckinDates ON YelpCheckin (UNNEST checkin_times UNNEST dates : string ?) ;
+
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/metadata/open/use-case-4/use-case-4.2.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/metadata/open/use-case-4/use-case-4.2.query.sqlpp
new file mode 100644
index 0000000..822f970
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/metadata/open/use-case-4/use-case-4.2.query.sqlpp
@@ -0,0 +1,23 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+SELECT D.SearchKey, D.SearchKeyElements, D.SearchKeyType
+FROM Metadata.`Index` D
+WHERE D.IndexName = "IdxYelpCheckinDates" AND D.DataverseName = "TestYelp";
+
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/metadata/open/with-3-level-record-path/with-3-level-record-path.1.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/metadata/open/with-3-level-record-path/with-3-level-record-path.1.ddl.sqlpp
new file mode 100644
index 0000000..4a21b30
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/metadata/open/with-3-level-record-path/with-3-level-record-path.1.ddl.sqlpp
@@ -0,0 +1,34 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+/*
+ * Description: Verify the index metadata for a record->record->record->array->atomic index. 
+ */
+
+DROP DATAVERSE TestYelp IF EXISTS;
+CREATE DATAVERSE TestYelp;
+USE TestYelp;
+
+CREATE TYPE CheckinType AS {
+    checkin_id: uuid
+};
+
+CREATE DATASET YelpCheckin(CheckinType) PRIMARY KEY checkin_id AUTOGENERATED;
+CREATE INDEX IdxYelpCheckinDates ON YelpCheckin (UNNEST checkin_data.checkin_temporal.checkin_times.dates : string ?) ;
+
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/metadata/open/with-3-level-record-path/with-3-level-record-path.2.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/metadata/open/with-3-level-record-path/with-3-level-record-path.2.query.sqlpp
new file mode 100644
index 0000000..822f970
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/metadata/open/with-3-level-record-path/with-3-level-record-path.2.query.sqlpp
@@ -0,0 +1,23 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+SELECT D.SearchKey, D.SearchKeyElements, D.SearchKeyType
+FROM Metadata.`Index` D
+WHERE D.IndexName = "IdxYelpCheckinDates" AND D.DataverseName = "TestYelp";
+
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/metadata/open/with-composite-array-different-indicators/with-composite-array-different-indicators.1.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/metadata/open/with-composite-array-different-indicators/with-composite-array-different-indicators.1.ddl.sqlpp
new file mode 100644
index 0000000..c3a181a
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/metadata/open/with-composite-array-different-indicators/with-composite-array-different-indicators.1.ddl.sqlpp
@@ -0,0 +1,34 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+/*
+ * Description: Verify the index metadata for a composite array->record->atomic index on two array fields. 
+ */
+
+DROP DATAVERSE TestYelp IF EXISTS;
+CREATE DATAVERSE TestYelp;
+USE TestYelp;
+
+CREATE TYPE CheckinType AS {
+    checkin_id: uuid
+};
+
+CREATE DATASET YelpCheckin(CheckinType) PRIMARY KEY checkin_id AUTOGENERATED;
+CREATE INDEX IdxYelpCheckinDateTime ON YelpCheckin (UNNEST checkin_times SELECT date : string ?, t.time : string ?) ;
+
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/metadata/open/with-composite-array-different-indicators/with-composite-array-different-indicators.2.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/metadata/open/with-composite-array-different-indicators/with-composite-array-different-indicators.2.query.sqlpp
new file mode 100644
index 0000000..643dad3
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/metadata/open/with-composite-array-different-indicators/with-composite-array-different-indicators.2.query.sqlpp
@@ -0,0 +1,23 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+SELECT D.SearchKey, D.SearchKeyElements, D.SearchKeyType
+FROM Metadata.`Index` D
+WHERE D.IndexName = "IdxYelpCheckinDateTime" AND D.DataverseName = "TestYelp";
+
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/metadata/open/with-composite-sk/with-composite-sk.1.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/metadata/open/with-composite-sk/with-composite-sk.1.ddl.sqlpp
new file mode 100644
index 0000000..2a9497a
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/metadata/open/with-composite-sk/with-composite-sk.1.ddl.sqlpp
@@ -0,0 +1,33 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+/*
+ * Description: Verify the index metadata for both a composite atomic, array->atomic and array->atomic, atomic index. 
+ */
+
+DROP DATAVERSE TestYelp IF EXISTS;
+CREATE DATAVERSE TestYelp;
+USE TestYelp;
+
+CREATE TYPE CheckinType AS {
+    checkin_id: uuid
+};
+
+CREATE DATASET YelpCheckin(CheckinType) PRIMARY KEY checkin_id AUTOGENERATED;
+CREATE INDEX IdxYelpBusinessCheckinDates ON YelpCheckin (business_id : string ?, UNNEST dates : string ?) ;
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/metadata/open/with-composite-sk/with-composite-sk.2.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/metadata/open/with-composite-sk/with-composite-sk.2.query.sqlpp
new file mode 100644
index 0000000..bee279e
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/metadata/open/with-composite-sk/with-composite-sk.2.query.sqlpp
@@ -0,0 +1,22 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+SELECT D.SearchKey, D.SearchKeyElements, D.SearchKeyType
+FROM Metadata.`Index` D
+WHERE D.IndexName = "IdxYelpBusinessCheckinDates" AND D.DataverseName = "TestYelp";
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/select-quantified-queries/use-case-1/use-case-1.1.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/select-quantified-queries/use-case-1/use-case-1.1.ddl.sqlpp
new file mode 100644
index 0000000..19b7ff7
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/select-quantified-queries/use-case-1/use-case-1.1.ddl.sqlpp
@@ -0,0 +1,37 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+/*
+ * Description: Verify that we are able to correctly perform the following types of queries using an array->atomic index.
+ *     1) Implicit existential quantification w/ membership query.
+ *     2) Explicit existential quantification query, performing a range search.
+ *     3) Universal quantification query, perform a range search.
+ */
+
+DROP DATAVERSE TestYelp IF EXISTS;
+CREATE DATAVERSE TestYelp;
+USE TestYelp;
+
+CREATE TYPE CheckinType AS {
+    checkin_id: uuid,
+    business_id: string,
+    dates: [string]
+};
+
+CREATE DATASET YelpCheckin(CheckinType) PRIMARY KEY checkin_id AUTOGENERATED;
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/select-quantified-queries/use-case-1/use-case-1.2.update.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/select-quantified-queries/use-case-1/use-case-1.2.update.sqlpp
new file mode 100644
index 0000000..b75bf1b
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/select-quantified-queries/use-case-1/use-case-1.2.update.sqlpp
@@ -0,0 +1,276 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+USE TestYelp;
+
+INSERT INTO YelpCheckin (
+    [
+        {
+            "business_id": "--1UhMGODdWsrMastO9DZw",
+            "dates": [
+                "2016-04-26 19:49:16",
+                "2016-08-30 18:36:57",
+                "2016-10-15 02:45:18",
+                "2016-11-18 01:54:50",
+                "2017-04-20 18:39:06",
+                "2017-05-03 17:58:02",
+                "2019-03-19 22:04:48"
+            ]
+        },
+        {
+            "business_id": "--EF5N7P70J_UYBTPypYlA",
+            "dates": [
+                "2018-05-25 19:52:07",
+                "2018-09-18 16:09:44",
+                "2019-10-18 21:29:09"
+            ]
+        },
+        {
+            "business_id": "--Ni3oJ4VOqfOEu7Sj2Vzg",
+            "dates": [
+                "2019-06-07 17:54:58"
+            ]
+        },
+        {
+            "business_id": "--Y1Adl1YUWfYIRSd8vkmA",
+            "dates": [
+                "2011-05-03 20:54:05",
+                "2011-08-23 20:49:45",
+                "2014-12-04 06:13:01",
+                "2016-11-16 19:25:55"
+            ]
+        },
+        {
+            "business_id": "--YPwqIlRJrhHkJcjY3eiA",
+            "dates": [
+                "2016-06-18 21:35:45",
+                "2016-10-15 18:17:51"
+            ]
+        },
+        {
+            "business_id": "--e8PjCNhEz32pprnPhCwQ",
+            "dates": [
+                "2015-04-02 21:45:17"
+            ]
+        },
+        {
+            "business_id": "--kinfHwmtdjz03g8B8z8Q",
+            "dates": [
+                "2014-08-27 17:49:18",
+                "2015-12-19 21:30:31",
+                "2018-11-27 15:53:50"
+            ]
+        },
+        {
+            "business_id": "--q6datkI-f0EoVheXNEeQ",
+            "dates": [
+                "2014-01-28 20:56:04",
+                "2014-11-16 16:11:58",
+                "2015-11-15 19:21:53",
+                "2015-11-15 19:33:39"
+            ]
+        },
+        {
+            "business_id": "--qvQS4MigHPykD2GV0-zw",
+            "dates": [
+                "2019-04-11 18:30:12"
+            ]
+        },
+        {
+            "business_id": "--wIGbLEhlpl_UeAIyDmZQ",
+            "dates": [
+                "2015-06-06 20:01:06",
+                "2019-03-14 22:01:52"
+            ]
+        },
+        {
+            "business_id": "-0FA-Qdi3SPYIoJz9UQw-A",
+            "dates": [
+                "2018-09-29 18:55:17",
+                "2018-10-20 16:48:05",
+                "2018-10-20 22:20:24"
+            ]
+        },
+        {
+            "business_id": "-0Hj1hb_XW6ybWq2M7QhGA",
+            "dates": [
+                "2011-04-23 21:11:22",
+                "2014-05-04 19:42:48",
+                "2014-05-11 19:16:08",
+                "2014-06-04 19:14:18",
+                "2015-12-05 19:22:42",
+                "2017-05-15 23:19:00"
+            ]
+        },
+        {
+            "business_id": "-0KMvRFwDWdVBeTpT11iHw",
+            "dates": [
+                "2012-07-13 21:43:57",
+                "2016-12-24 02:27:31",
+                "2017-08-31 00:35:26"
+            ]
+        },
+        {
+            "business_id": "-0LPtgJC31FWMrMv317p0Q",
+            "dates": [
+                "2013-04-13 12:35:33",
+                "2013-08-19 23:35:49",
+                "2013-10-04 19:14:56"
+            ]
+        },
+        {
+            "business_id": "-0M3o2uWBnQZwd3hmfEwuw",
+            "dates": [
+                "2016-09-10 19:26:19",
+                "2018-09-08 14:15:37",
+                "2019-09-13 22:47:25"
+            ]
+        },
+        {
+            "business_id": "-0RRiWDtfnS16AKCtfvBZg",
+            "dates": [
+                "2017-05-19 14:30:16",
+                "2017-05-19 14:30:25",
+                "2017-08-28 15:49:37",
+                "2017-09-20 20:19:51",
+                "2017-10-01 16:31:05",
+                "2017-10-01 16:56:27",
+                "2017-12-27 23:33:20"
+            ]
+        },
+        {
+            "business_id": "-0Soj75v-XoRcf2ERr8Bmg",
+            "dates": [
+                "2019-06-05 18:22:49"
+            ]
+        },
+        {
+            "business_id": "-0ZumLlFjMh4ZW1z2nXGug",
+            "dates": [
+                "2011-09-24 21:37:32",
+                "2014-03-10 20:20:07",
+                "2015-05-27 00:40:24",
+                "2015-08-29 17:58:15",
+                "2018-03-16 15:03:26"
+            ]
+        },
+        {
+            "business_id": "-0aOudcaAyac0VJbMX-L1g",
+            "dates": [
+                "2015-03-16 23:51:16",
+                "2015-12-21 04:48:01",
+                "2016-10-28 20:22:42",
+                "2016-10-28 20:23:00"
+            ]
+        },
+        {
+            "business_id": "-0b86isaXMY0v4g-V8GZ9Q",
+            "dates": [
+                "2013-10-22 16:49:21",
+                "2014-11-21 17:39:24"
+            ]
+        },
+        {
+            "business_id": "-0d-BfFSU0bwLcnMaGRxYw",
+            "dates": [
+                "2014-08-07 18:30:48",
+                "2014-09-16 20:41:45",
+                "2014-10-12 23:22:27",
+                "2015-07-21 20:43:56",
+                "2015-07-21 20:45:07"
+            ]
+        },
+        {
+            "business_id": "-0jz6c3C6i7RG7Ag22K-Pg",
+            "dates": [
+                "2015-05-02 19:49:05",
+                "2015-05-06 03:52:18",
+                "2015-09-26 01:13:19"
+            ]
+        },
+        {
+            "business_id": "-0y3MZU2oYP8r1ruDP1bfQ",
+            "dates": [
+                "2015-04-11 13:14:14",
+                "2015-11-21 16:05:56",
+                "2016-05-06 14:10:04",
+                "2017-08-09 15:15:10",
+                "2017-10-21 15:12:56"
+            ]
+        },
+        {
+            "business_id": "-1BPe8UjF2_l3nVk-DFUjA",
+            "dates": [
+                "2015-12-03 18:44:00",
+                "2016-03-17 18:19:21",
+                "2016-11-02 15:58:38"
+            ]
+        },
+        {
+            "business_id": "-1E2CQu_38mkghvmZgCCRw",
+            "dates": [
+                "2019-04-04 22:02:37"
+            ]
+        },
+        {
+            "business_id": "-1wzk43IZ5D9Ysu6kzb5xA",
+            "dates": [
+                "2019-02-27 14:03:08"
+            ]
+        },
+        {
+            "business_id": "-23R9P2eG7VTc6DVLjFKzA",
+            "dates": [
+                "2011-12-21 19:02:51",
+                "2012-04-15 04:21:39",
+                "2012-04-15 14:23:56",
+                "2013-06-30 22:39:51",
+                "2013-10-04 20:34:13",
+                "2014-07-16 02:28:40"
+            ]
+        },
+        {
+            "business_id": "-26MGfikhJiTfCI-GqmzhQ",
+            "dates": [
+                "2018-06-13 20:16:07"
+            ]
+        },
+        {
+            "business_id": "-2bLuJsMZ0WhI9daurVQNQ",
+            "dates": [
+                "2015-05-29 16:46:17",
+                "2015-06-01 15:03:53"
+            ]
+        },
+        {
+            "business_id": "-2hDBMaza_ldqnZdiU06LQ",
+            "dates": [
+                "2011-10-08 12:02:23",
+                "2014-08-18 02:11:11",
+                "2016-01-07 05:27:51",
+                "2016-10-21 20:15:55",
+                "2016-12-01 03:57:10",
+                "2016-12-29 01:54:42",
+                "2018-07-22 19:55:31",
+                "2018-09-07 01:42:54",
+                "2019-03-08 03:41:06"
+            ]
+        }
+    ]
+);
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/select-quantified-queries/use-case-1/use-case-1.3.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/select-quantified-queries/use-case-1/use-case-1.3.ddl.sqlpp
new file mode 100644
index 0000000..a0fc1bf
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/select-quantified-queries/use-case-1/use-case-1.3.ddl.sqlpp
@@ -0,0 +1,22 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+USE TestYelp;
+
+CREATE INDEX IdxYelpCheckinDates ON YelpCheckin (UNNEST dates);
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/select-quantified-queries/use-case-1/use-case-1.4.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/select-quantified-queries/use-case-1/use-case-1.4.query.sqlpp
new file mode 100644
index 0000000..dc5fc32
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/select-quantified-queries/use-case-1/use-case-1.4.query.sqlpp
@@ -0,0 +1,25 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+SET `compiler.arrayindex` "true";
+
+USE TestYelp;
+
+SELECT C.business_id
+FROM YelpCheckin C 
+WHERE "2016-04-26 19:49:16" IN C.dates;
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/select-quantified-queries/use-case-1/use-case-1.5.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/select-quantified-queries/use-case-1/use-case-1.5.query.sqlpp
new file mode 100644
index 0000000..2921d18
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/select-quantified-queries/use-case-1/use-case-1.5.query.sqlpp
@@ -0,0 +1,26 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+SET `compiler.arrayindex` "true";
+
+USE TestYelp;
+
+SELECT COUNT(*)
+FROM YelpCheckin C
+WHERE SOME D IN C.dates
+SATISFIES D > "2016" AND D < "2017";
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/select-quantified-queries/use-case-1/use-case-1.6.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/select-quantified-queries/use-case-1/use-case-1.6.query.sqlpp
new file mode 100644
index 0000000..e64a93c
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/select-quantified-queries/use-case-1/use-case-1.6.query.sqlpp
@@ -0,0 +1,26 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+SET `compiler.arrayindex` "true";
+
+USE TestYelp;
+
+SELECT COUNT(*)
+FROM YelpCheckin C
+WHERE EVERY D IN C.dates
+SATISFIES D > "2016" AND D < "2017";
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/select-quantified-queries/use-case-2/use-case-2.1.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/select-quantified-queries/use-case-2/use-case-2.1.ddl.sqlpp
new file mode 100644
index 0000000..fd32cb9
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/select-quantified-queries/use-case-2/use-case-2.1.ddl.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.
+ */
+
+/*
+ * Description: Verify that we are able to correctly perform the following types of queries using a record->array->atomic index.
+ *     1) Implicit existential quantification w/ membership query.
+ *     2) Explicit existential quantification query, performing a range search.
+ *     3) Universal quantification query, performing a range search.
+ */
+
+DROP DATAVERSE TestYelp IF EXISTS;
+CREATE DATAVERSE TestYelp;
+USE TestYelp;
+
+CREATE TYPE CheckinType AS {
+    checkin_id: uuid,
+    business_id: string,
+    checkin_times: {
+        dates: [string],
+        times: [string]
+    }
+};
+
+CREATE DATASET YelpCheckin(CheckinType) PRIMARY KEY checkin_id AUTOGENERATED;
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/select-quantified-queries/use-case-2/use-case-2.2.update.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/select-quantified-queries/use-case-2/use-case-2.2.update.sqlpp
new file mode 100644
index 0000000..2549e8f
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/select-quantified-queries/use-case-2/use-case-2.2.update.sqlpp
@@ -0,0 +1,497 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+USE TestYelp;
+
+INSERT INTO YelpCheckin (
+    [
+        {
+            "business_id": "--1UhMGODdWsrMastO9DZw",
+            "checkin_times": {
+                "dates": [
+                    "2016-04-26",
+                    "2016-08-30",
+                    "2016-10-15",
+                    "2016-11-18",
+                    "2017-04-20",
+                    "2017-05-03",
+                    "2019-03-19"
+                ],
+                "times": [
+                    "19:49:16",
+                    "18:36:57",
+                    "02:45:18",
+                    "01:54:50",
+                    "18:39:06",
+                    "17:58:02",
+                    "22:04:48"
+                ]
+            }
+        },
+        {
+            "business_id": "--EF5N7P70J_UYBTPypYlA",
+            "checkin_times": {
+                "dates": [
+                    "2018-05-25",
+                    "2018-09-18",
+                    "2019-10-18"
+                ],
+                "times": [
+                    "19:52:07",
+                    "16:09:44",
+                    "21:29:09"
+                ]
+            }
+        },
+        {
+            "business_id": "--Ni3oJ4VOqfOEu7Sj2Vzg",
+            "checkin_times": {
+                "dates": [
+                    "2019-06-07"
+                ],
+                "times": [
+                    "17:54:58"
+                ]
+            }
+        },
+        {
+            "business_id": "--Y1Adl1YUWfYIRSd8vkmA",
+            "checkin_times": {
+                "dates": [
+                    "2011-05-03",
+                    "2011-08-23",
+                    "2014-12-04",
+                    "2016-11-16"
+                ],
+                "times": [
+                    "20:54:05",
+                    "20:49:45",
+                    "06:13:01",
+                    "19:25:55"
+                ]
+            }
+        },
+        {
+            "business_id": "--YPwqIlRJrhHkJcjY3eiA",
+            "checkin_times": {
+                "dates": [
+                    "2016-06-18",
+                    "2016-10-15"
+                ],
+                "times": [
+                    "21:35:45",
+                    "18:17:51"
+                ]
+            }
+        },
+        {
+            "business_id": "--e8PjCNhEz32pprnPhCwQ",
+            "checkin_times": {
+                "dates": [
+                    "2015-04-02"
+                ],
+                "times": [
+                    "21:45:17"
+                ]
+            }
+        },
+        {
+            "business_id": "--kinfHwmtdjz03g8B8z8Q",
+            "checkin_times": {
+                "dates": [
+                    "2014-08-27",
+                    "2015-12-19",
+                    "2018-11-27"
+                ],
+                "times": [
+                    "17:49:18",
+                    "21:30:31",
+                    "15:53:50"
+                ]
+            }
+        },
+        {
+            "business_id": "--q6datkI-f0EoVheXNEeQ",
+            "checkin_times": {
+                "dates": [
+                    "2014-01-28",
+                    "2014-11-16",
+                    "2015-11-15",
+                    "2015-11-15"
+                ],
+                "times": [
+                    "20:56:04",
+                    "16:11:58",
+                    "19:21:53",
+                    "19:33:39"
+                ]
+            }
+        },
+        {
+            "business_id": "--qvQS4MigHPykD2GV0-zw",
+            "checkin_times": {
+                "dates": [
+                    "2019-04-11"
+                ],
+                "times": [
+                    "18:30:12"
+                ]
+            }
+        },
+        {
+            "business_id": "--wIGbLEhlpl_UeAIyDmZQ",
+            "checkin_times": {
+                "dates": [
+                    "2015-06-06",
+                    "2019-03-14"
+                ],
+                "times": [
+                    "20:01:06",
+                    "22:01:52"
+                ]
+            }
+        },
+        {
+            "business_id": "-0FA-Qdi3SPYIoJz9UQw-A",
+            "checkin_times": {
+                "dates": [
+                    "2018-09-29",
+                    "2018-10-20",
+                    "2018-10-20"
+                ],
+                "times": [
+                    "18:55:17",
+                    "16:48:05",
+                    "22:20:24"
+                ]
+            }
+        },
+        {
+            "business_id": "-0Hj1hb_XW6ybWq2M7QhGA",
+            "checkin_times": {
+                "dates": [
+                    "2011-04-23",
+                    "2014-05-04",
+                    "2014-05-11",
+                    "2014-06-04",
+                    "2015-12-05",
+                    "2017-05-15"
+                ],
+                "times": [
+                    "21:11:22",
+                    "19:42:48",
+                    "19:16:08",
+                    "19:14:18",
+                    "19:22:42",
+                    "23:19:00"
+                ]
+            }
+        },
+        {
+            "business_id": "-0KMvRFwDWdVBeTpT11iHw",
+            "checkin_times": {
+                "dates": [
+                    "2012-07-13",
+                    "2016-12-24",
+                    "2017-08-31"
+                ],
+                "times": [
+                    "21:43:57",
+                    "02:27:31",
+                    "00:35:26"
+                ]
+            }
+        },
+        {
+            "business_id": "-0LPtgJC31FWMrMv317p0Q",
+            "checkin_times": {
+                "dates": [
+                    "2013-04-13",
+                    "2013-08-19",
+                    "2013-10-04"
+                ],
+                "times": [
+                    "12:35:33",
+                    "23:35:49",
+                    "19:14:56"
+                ]
+            }
+        },
+        {
+            "business_id": "-0M3o2uWBnQZwd3hmfEwuw",
+            "checkin_times": {
+                "dates": [
+                    "2016-09-10",
+                    "2018-09-08",
+                    "2019-09-13"
+                ],
+                "times": [
+                    "19:26:19",
+                    "14:15:37",
+                    "22:47:25"
+                ]
+            }
+        },
+        {
+            "business_id": "-0RRiWDtfnS16AKCtfvBZg",
+            "checkin_times": {
+                "dates": [
+                    "2017-05-19",
+                    "2017-05-19",
+                    "2017-08-28",
+                    "2017-09-20",
+                    "2017-10-01",
+                    "2017-10-01",
+                    "2017-12-27"
+                ],
+                "times": [
+                    "14:30:16",
+                    "14:30:25",
+                    "15:49:37",
+                    "20:19:51",
+                    "16:31:05",
+                    "16:56:27",
+                    "23:33:20"
+                ]
+            }
+        },
+        {
+            "business_id": "-0Soj75v-XoRcf2ERr8Bmg",
+            "checkin_times": {
+                "dates": [
+                    "2019-06-05"
+                ],
+                "times": [
+                    "18:22:49"
+                ]
+            }
+        },
+        {
+            "business_id": "-0ZumLlFjMh4ZW1z2nXGug",
+            "checkin_times": {
+                "dates": [
+                    "2011-09-24",
+                    "2014-03-10",
+                    "2015-05-27",
+                    "2015-08-29",
+                    "2018-03-16"
+                ],
+                "times": [
+                    "21:37:32",
+                    "20:20:07",
+                    "00:40:24",
+                    "17:58:15",
+                    "15:03:26"
+                ]
+            }
+        },
+        {
+            "business_id": "-0aOudcaAyac0VJbMX-L1g",
+            "checkin_times": {
+                "dates": [
+                    "2015-03-16",
+                    "2015-12-21",
+                    "2016-10-28",
+                    "2016-10-28"
+                ],
+                "times": [
+                    "23:51:16",
+                    "04:48:01",
+                    "20:22:42",
+                    "20:23:00"
+                ]
+            }
+        },
+        {
+            "business_id": "-0b86isaXMY0v4g-V8GZ9Q",
+            "checkin_times": {
+                "dates": [
+                    "2013-10-22",
+                    "2014-11-21"
+                ],
+                "times": [
+                    "16:49:21",
+                    "17:39:24"
+                ]
+            }
+        },
+        {
+            "business_id": "-0d-BfFSU0bwLcnMaGRxYw",
+            "checkin_times": {
+                "dates": [
+                    "2014-08-07",
+                    "2014-09-16",
+                    "2014-10-12",
+                    "2015-07-21",
+                    "2015-07-21"
+                ],
+                "times": [
+                    "18:30:48",
+                    "20:41:45",
+                    "23:22:27",
+                    "20:43:56",
+                    "20:45:07"
+                ]
+            }
+        },
+        {
+            "business_id": "-0jz6c3C6i7RG7Ag22K-Pg",
+            "checkin_times": {
+                "dates": [
+                    "2015-05-02",
+                    "2015-05-06",
+                    "2015-09-26"
+                ],
+                "times": [
+                    "19:49:05",
+                    "03:52:18",
+                    "01:13:19"
+                ]
+            }
+        },
+        {
+            "business_id": "-0y3MZU2oYP8r1ruDP1bfQ",
+            "checkin_times": {
+                "dates": [
+                    "2015-04-11",
+                    "2015-11-21",
+                    "2016-05-06",
+                    "2017-08-09",
+                    "2017-10-21"
+                ],
+                "times": [
+                    "13:14:14",
+                    "16:05:56",
+                    "14:10:04",
+                    "15:15:10",
+                    "15:12:56"
+                ]
+            }
+        },
+        {
+            "business_id": "-1BPe8UjF2_l3nVk-DFUjA",
+            "checkin_times": {
+                "dates": [
+                    "2015-12-03",
+                    "2016-03-17",
+                    "2016-11-02"
+                ],
+                "times": [
+                    "18:44:00",
+                    "18:19:21",
+                    "15:58:38"
+                ]
+            }
+        },
+        {
+            "business_id": "-1E2CQu_38mkghvmZgCCRw",
+            "checkin_times": {
+                "dates": [
+                    "2019-04-04"
+                ],
+                "times": [
+                    "22:02:37"
+                ]
+            }
+        },
+        {
+            "business_id": "-1wzk43IZ5D9Ysu6kzb5xA",
+            "checkin_times": {
+                "dates": [
+                    "2019-02-27"
+                ],
+                "times": [
+                    "14:03:08"
+                ]
+            }
+        },
+        {
+            "business_id": "-23R9P2eG7VTc6DVLjFKzA",
+            "checkin_times": {
+                "dates": [
+                    "2011-12-21",
+                    "2012-04-15",
+                    "2012-04-15",
+                    "2013-06-30",
+                    "2013-10-04",
+                    "2014-07-16"
+                ],
+                "times": [
+                    "19:02:51",
+                    "04:21:39",
+                    "14:23:56",
+                    "22:39:51",
+                    "20:34:13",
+                    "02:28:40"
+                ]
+            }
+        },
+        {
+            "business_id": "-26MGfikhJiTfCI-GqmzhQ",
+            "checkin_times": {
+                "dates": [
+                    "2018-06-13"
+                ],
+                "times": [
+                    "20:16:07"
+                ]
+            }
+        },
+        {
+            "business_id": "-2bLuJsMZ0WhI9daurVQNQ",
+            "checkin_times": {
+                "dates": [
+                    "2015-05-29",
+                    "2015-06-01"
+                ],
+                "times": [
+                    "16:46:17",
+                    "15:03:53"
+                ]
+            }
+        },
+        {
+            "business_id": "-2hDBMaza_ldqnZdiU06LQ",
+            "checkin_times": {
+                "dates": [
+                    "2011-10-08",
+                    "2014-08-18",
+                    "2016-01-07",
+                    "2016-10-21",
+                    "2016-12-01",
+                    "2016-12-29",
+                    "2018-07-22",
+                    "2018-09-07",
+                    "2019-03-08"
+                ],
+                "times": [
+                    "12:02:23",
+                    "02:11:11",
+                    "05:27:51",
+                    "20:15:55",
+                    "03:57:10",
+                    "01:54:42",
+                    "19:55:31",
+                    "01:42:54",
+                    "03:41:06"
+                ]
+            }
+        }
+    ]
+);
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/select-quantified-queries/use-case-2/use-case-2.3.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/select-quantified-queries/use-case-2/use-case-2.3.ddl.sqlpp
new file mode 100644
index 0000000..0ab72fe
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/select-quantified-queries/use-case-2/use-case-2.3.ddl.sqlpp
@@ -0,0 +1,22 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+USE TestYelp;
+
+CREATE INDEX IdxYelpCheckinDates ON YelpCheckin (UNNEST checkin_times.dates);
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/select-quantified-queries/use-case-2/use-case-2.4.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/select-quantified-queries/use-case-2/use-case-2.4.query.sqlpp
new file mode 100644
index 0000000..ff3b923
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/select-quantified-queries/use-case-2/use-case-2.4.query.sqlpp
@@ -0,0 +1,25 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+SET `compiler.arrayindex` "true";
+
+USE TestYelp;
+
+SELECT C.business_id
+FROM YelpCheckin C
+WHERE "2016-04-26" IN C.checkin_times.dates;
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/select-quantified-queries/use-case-2/use-case-2.5.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/select-quantified-queries/use-case-2/use-case-2.5.query.sqlpp
new file mode 100644
index 0000000..cea9d3e
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/select-quantified-queries/use-case-2/use-case-2.5.query.sqlpp
@@ -0,0 +1,26 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+SET `compiler.arrayindex` "true";
+
+USE TestYelp;
+
+SELECT COUNT(*)
+FROM YelpCheckin C
+WHERE SOME D IN C.checkin_times.dates 
+SATISFIES D > "2016" AND D < "2017";
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/select-quantified-queries/use-case-2/use-case-2.6.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/select-quantified-queries/use-case-2/use-case-2.6.query.sqlpp
new file mode 100644
index 0000000..c9652f1
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/select-quantified-queries/use-case-2/use-case-2.6.query.sqlpp
@@ -0,0 +1,26 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+SET `compiler.arrayindex` "true";
+
+USE TestYelp;
+
+SELECT COUNT(*)
+FROM YelpCheckin C
+WHERE EVERY D IN C.checkin_times.dates 
+SATISFIES D > "2016" AND D < "2017";
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/select-quantified-queries/use-case-3/use-case-3.1.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/select-quantified-queries/use-case-3/use-case-3.1.ddl.sqlpp
new file mode 100644
index 0000000..8b4c15b
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/select-quantified-queries/use-case-3/use-case-3.1.ddl.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.
+ */
+
+/*
+ * Description: Verify that we are able to correctly perform the following types of queries using a array->record->atomic index.
+ *     1) Explicit existential quantification query, performing a point search.
+ *     2) Explicit existential quantification query, performing a point search on two fields inside the array.
+ *     3) Universal quantification query, peforming a range search.
+ */
+
+DROP DATAVERSE TestYelp IF EXISTS;
+CREATE DATAVERSE TestYelp;
+USE TestYelp;
+
+CREATE TYPE CheckinType AS {
+    checkin_id: uuid,
+    business_id: string,
+    checkin_times: [{
+        date: string,
+        time: string
+    }]
+};
+
+CREATE DATASET YelpCheckin(CheckinType) PRIMARY KEY checkin_id AUTOGENERATED;
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/select-quantified-queries/use-case-3/use-case-3.2.update.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/select-quantified-queries/use-case-3/use-case-3.2.update.sqlpp
new file mode 100644
index 0000000..02ddf21
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/select-quantified-queries/use-case-3/use-case-3.2.update.sqlpp
@@ -0,0 +1,276 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+USE TestYelp;
+
+INSERT INTO YelpCheckin (
+    [
+        {
+            "business_id": "--1UhMGODdWsrMastO9DZw",
+            "checkin_times": [ 
+                {"date": "2016-04-26", "time": "19:49:16"},
+                {"date": "2016-08-30", "time": "18:36:57"},
+                {"date": "2016-10-15", "time": "02:45:18"},
+                {"date": "2016-11-18", "time": "01:54:50"},
+                {"date": "2017-04-20", "time": "18:39:06"},
+                {"date": "2017-05-03", "time": "17:58:02"},
+                {"date": "2019-03-19", "time": "22:04:48"}
+             ]
+        },
+        {
+            "business_id": "--EF5N7P70J_UYBTPypYlA",
+            "checkin_times": [ 
+                {"date": "2018-05-25", "time": "19:52:07"},
+                {"date": "2018-09-18", "time": "16:09:44"},
+                {"date": "2019-10-18", "time": "21:29:09"}
+             ]
+        },
+        {
+            "business_id": "--Ni3oJ4VOqfOEu7Sj2Vzg",
+            "checkin_times": [ 
+                {"date": "2019-06-07", "time": "17:54:58"}
+             ]
+        },
+        {
+            "business_id": "--Y1Adl1YUWfYIRSd8vkmA",
+            "checkin_times": [ 
+                {"date": "2011-05-03", "time": "20:54:05"},
+                {"date": "2011-08-23", "time": "20:49:45"},
+                {"date": "2014-12-04", "time": "06:13:01"},
+                {"date": "2016-11-16", "time": "19:25:55"}
+             ]
+        },
+        {
+            "business_id": "--YPwqIlRJrhHkJcjY3eiA",
+            "checkin_times": [ 
+                {"date": "2016-06-18", "time": "21:35:45"},
+                {"date": "2016-10-15", "time": "18:17:51"}
+             ]
+        },
+        {
+            "business_id": "--e8PjCNhEz32pprnPhCwQ",
+            "checkin_times": [ 
+                {"date": "2015-04-02", "time": "21:45:17"}
+             ]
+        },
+        {
+            "business_id": "--kinfHwmtdjz03g8B8z8Q",
+            "checkin_times": [ 
+                {"date": "2014-08-27", "time": "17:49:18"},
+                {"date": "2015-12-19", "time": "21:30:31"},
+                {"date": "2018-11-27", "time": "15:53:50"}
+             ]
+        },
+        {
+            "business_id": "--q6datkI-f0EoVheXNEeQ",
+            "checkin_times": [ 
+                {"date": "2014-01-28", "time": "20:56:04"},
+                {"date": "2014-11-16", "time": "16:11:58"},
+                {"date": "2015-11-15", "time": "19:21:53"},
+                {"date": "2015-11-15", "time": "19:33:39"}
+             ]
+        },
+        {
+            "business_id": "--qvQS4MigHPykD2GV0-zw",
+            "checkin_times": [ 
+                {"date": "2019-04-11", "time": "18:30:12"}
+             ]
+        },
+        {
+            "business_id": "--wIGbLEhlpl_UeAIyDmZQ",
+            "checkin_times": [ 
+                {"date": "2015-06-06", "time": "20:01:06"},
+                {"date": "2019-03-14", "time": "22:01:52"}
+             ]
+        },
+        {
+            "business_id": "-0FA-Qdi3SPYIoJz9UQw-A",
+            "checkin_times": [ 
+                {"date": "2018-09-29", "time": "18:55:17"},
+                {"date": "2018-10-20", "time": "16:48:05"},
+                {"date": "2018-10-20", "time": "22:20:24"}
+             ]
+        },
+        {
+            "business_id": "-0Hj1hb_XW6ybWq2M7QhGA",
+            "checkin_times": [ 
+                {"date": "2011-04-23", "time": "21:11:22"},
+                {"date": "2014-05-04", "time": "19:42:48"},
+                {"date": "2014-05-11", "time": "19:16:08"},
+                {"date": "2014-06-04", "time": "19:14:18"},
+                {"date": "2015-12-05", "time": "19:22:42"},
+                {"date": "2017-05-15", "time": "23:19:00"}
+             ]
+        },
+        {
+            "business_id": "-0KMvRFwDWdVBeTpT11iHw",
+            "checkin_times": [ 
+                {"date": "2012-07-13", "time": "21:43:57"},
+                {"date": "2016-12-24", "time": "02:27:31"},
+                {"date": "2017-08-31", "time": "00:35:26"}
+             ]
+        },
+        {
+            "business_id": "-0LPtgJC31FWMrMv317p0Q",
+            "checkin_times": [ 
+                {"date": "2013-04-13", "time": "12:35:33"},
+                {"date": "2013-08-19", "time": "23:35:49"},
+                {"date": "2013-10-04", "time": "19:14:56"}
+             ]
+        },
+        {
+            "business_id": "-0M3o2uWBnQZwd3hmfEwuw",
+            "checkin_times": [ 
+                {"date": "2016-09-10", "time": "19:26:19"},
+                {"date": "2018-09-08", "time": "14:15:37"},
+                {"date": "2019-09-13", "time": "22:47:25"}
+             ]
+        },
+        {
+            "business_id": "-0RRiWDtfnS16AKCtfvBZg",
+            "checkin_times": [ 
+                {"date": "2017-05-19", "time": "14:30:16"},
+                {"date": "2017-05-19", "time": "14:30:25"},
+                {"date": "2017-08-28", "time": "15:49:37"},
+                {"date": "2017-09-20", "time": "20:19:51"},
+                {"date": "2017-10-01", "time": "16:31:05"},
+                {"date": "2017-10-01", "time": "16:56:27"},
+                {"date": "2017-12-27", "time": "23:33:20"}
+             ]
+        },
+        {
+            "business_id": "-0Soj75v-XoRcf2ERr8Bmg",
+            "checkin_times": [ 
+                {"date": "2019-06-05", "time": "18:22:49"}
+             ]
+        },
+        {
+            "business_id": "-0ZumLlFjMh4ZW1z2nXGug",
+            "checkin_times": [ 
+                {"date": "2011-09-24", "time": "21:37:32"},
+                {"date": "2014-03-10", "time": "20:20:07"},
+                {"date": "2015-05-27", "time": "00:40:24"},
+                {"date": "2015-08-29", "time": "17:58:15"},
+                {"date": "2018-03-16", "time": "15:03:26"}
+             ]
+        },
+        {
+            "business_id": "-0aOudcaAyac0VJbMX-L1g",
+            "checkin_times": [ 
+                {"date": "2015-03-16", "time": "23:51:16"},
+                {"date": "2015-12-21", "time": "04:48:01"},
+                {"date": "2016-10-28", "time": "20:22:42"},
+                {"date": "2016-10-28", "time": "20:23:00"}
+             ]
+        },
+        {
+            "business_id": "-0b86isaXMY0v4g-V8GZ9Q",
+            "checkin_times": [ 
+                {"date": "2013-10-22", "time": "16:49:21"},
+                {"date": "2014-11-21", "time": "17:39:24"}
+             ]
+        },
+        {
+            "business_id": "-0d-BfFSU0bwLcnMaGRxYw",
+            "checkin_times": [ 
+                {"date": "2014-08-07", "time": "18:30:48"},
+                {"date": "2014-09-16", "time": "20:41:45"},
+                {"date": "2014-10-12", "time": "23:22:27"},
+                {"date": "2015-07-21", "time": "20:43:56"},
+                {"date": "2015-07-21", "time": "20:45:07"}
+             ]
+        },
+        {
+            "business_id": "-0jz6c3C6i7RG7Ag22K-Pg",
+            "checkin_times": [ 
+                {"date": "2015-05-02", "time": "19:49:05"},
+                {"date": "2015-05-06", "time": "03:52:18"},
+                {"date": "2015-09-26", "time": "01:13:19"}
+             ]
+        },
+        {
+            "business_id": "-0y3MZU2oYP8r1ruDP1bfQ",
+            "checkin_times": [ 
+                {"date": "2015-04-11", "time": "13:14:14"},
+                {"date": "2015-11-21", "time": "16:05:56"},
+                {"date": "2016-05-06", "time": "14:10:04"},
+                {"date": "2017-08-09", "time": "15:15:10"},
+                {"date": "2017-10-21", "time": "15:12:56"}
+             ]
+        },
+        {
+            "business_id": "-1BPe8UjF2_l3nVk-DFUjA",
+            "checkin_times": [ 
+                {"date": "2015-12-03", "time": "18:44:00"},
+                {"date": "2016-03-17", "time": "18:19:21"},
+                {"date": "2016-11-02", "time": "15:58:38"}
+             ]
+        },
+        {
+            "business_id": "-1E2CQu_38mkghvmZgCCRw",
+            "checkin_times": [ 
+                {"date": "2019-04-04", "time": "22:02:37"}
+             ]
+        },
+        {
+            "business_id": "-1wzk43IZ5D9Ysu6kzb5xA",
+            "checkin_times": [ 
+                {"date": "2019-02-27", "time": "14:03:08"}
+             ]
+        },
+        {
+            "business_id": "-23R9P2eG7VTc6DVLjFKzA",
+            "checkin_times": [ 
+                {"date": "2011-12-21", "time": "19:02:51"},
+                {"date": "2012-04-15", "time": "04:21:39"},
+                {"date": "2012-04-15", "time": "14:23:56"},
+                {"date": "2013-06-30", "time": "22:39:51"},
+                {"date": "2013-10-04", "time": "20:34:13"},
+                {"date": "2014-07-16", "time": "02:28:40"}
+             ]
+        },
+        {
+            "business_id": "-26MGfikhJiTfCI-GqmzhQ",
+            "checkin_times": [ 
+                {"date": "2018-06-13", "time": "20:16:07"}
+             ]
+        },
+        {
+            "business_id": "-2bLuJsMZ0WhI9daurVQNQ",
+            "checkin_times": [ 
+                {"date": "2015-05-29", "time": "16:46:17"},
+                {"date": "2015-06-01", "time": "15:03:53"}
+             ]
+        },
+        {
+            "business_id": "-2hDBMaza_ldqnZdiU06LQ",
+            "checkin_times": [ 
+                {"date": "2011-10-08", "time": "12:02:23"},
+                {"date": "2014-08-18", "time": "02:11:11"},
+                {"date": "2016-01-07", "time": "05:27:51"},
+                {"date": "2016-10-21", "time": "20:15:55"},
+                {"date": "2016-12-01", "time": "03:57:10"},
+                {"date": "2016-12-29", "time": "01:54:42"},
+                {"date": "2018-07-22", "time": "19:55:31"},
+                {"date": "2018-09-07", "time": "01:42:54"},
+                {"date": "2019-03-08", "time": "03:41:06"}
+             ]
+        }
+    ]
+);
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/select-quantified-queries/use-case-3/use-case-3.3.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/select-quantified-queries/use-case-3/use-case-3.3.ddl.sqlpp
new file mode 100644
index 0000000..ee911b4
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/select-quantified-queries/use-case-3/use-case-3.3.ddl.sqlpp
@@ -0,0 +1,22 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+USE TestYelp;
+
+CREATE INDEX IdxYelpCheckinDates ON YelpCheckin (UNNEST checkin_times SELECT date);
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/select-quantified-queries/use-case-3/use-case-3.4.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/select-quantified-queries/use-case-3/use-case-3.4.query.sqlpp
new file mode 100644
index 0000000..e21053a
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/select-quantified-queries/use-case-3/use-case-3.4.query.sqlpp
@@ -0,0 +1,26 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+SET `compiler.arrayindex` "true";
+
+USE TestYelp;
+
+SELECT C.business_id
+FROM YelpCheckin C
+WHERE SOME D IN C.checkin_times 
+SATISFIES "2016-04-26" = D.date;
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/select-quantified-queries/use-case-3/use-case-3.5.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/select-quantified-queries/use-case-3/use-case-3.5.query.sqlpp
new file mode 100644
index 0000000..30547411
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/select-quantified-queries/use-case-3/use-case-3.5.query.sqlpp
@@ -0,0 +1,26 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+SET `compiler.arrayindex` "true";
+
+USE TestYelp;
+
+SELECT C.business_id
+FROM YelpCheckin C
+WHERE SOME D IN C.checkin_times 
+SATISFIES D.date = "2016-04-26" AND D.time = "19:49:16";
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/select-quantified-queries/use-case-3/use-case-3.6.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/select-quantified-queries/use-case-3/use-case-3.6.query.sqlpp
new file mode 100644
index 0000000..6a73591
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/select-quantified-queries/use-case-3/use-case-3.6.query.sqlpp
@@ -0,0 +1,26 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+SET `compiler.arrayindex` "true";
+
+USE TestYelp;
+
+SELECT C.business_id
+FROM YelpCheckin C
+WHERE EVERY D IN C.checkin_times 
+SATISFIES D.date BETWEEN "2016" AND "2017";
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/select-quantified-queries/use-case-4/use-case-4.1.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/select-quantified-queries/use-case-4/use-case-4.1.ddl.sqlpp
new file mode 100644
index 0000000..65c16dd
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/select-quantified-queries/use-case-4/use-case-4.1.ddl.sqlpp
@@ -0,0 +1,41 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+/*
+ * Description: Verify that we are able to correctly perform the following types of queries using a array->record->array->atomic index.
+ *     1) Nested existential quantification (implicit using membership and explicit outer) query.
+ *     2) Nested existential quantification (explicit inner and outer) query.
+ *     3) Universal quantification containing explicit existential quantification query.
+ *     4) Nested universal quantification query.
+ */
+
+DROP DATAVERSE TestYelp IF EXISTS;
+CREATE DATAVERSE TestYelp;
+USE TestYelp;
+
+CREATE TYPE CheckinType AS {
+    checkin_id: uuid,
+    business_id: string,
+    checkin_times: [{
+        dates: [string],
+        times: [string]
+    }]
+};
+
+CREATE DATASET YelpCheckin(CheckinType) PRIMARY KEY checkin_id AUTOGENERATED;
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/select-quantified-queries/use-case-4/use-case-4.2.update.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/select-quantified-queries/use-case-4/use-case-4.2.update.sqlpp
new file mode 100644
index 0000000..7fcf9e4
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/select-quantified-queries/use-case-4/use-case-4.2.update.sqlpp
@@ -0,0 +1,425 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+USE TestYelp;
+
+INSERT INTO YelpCheckin (
+    [
+        {
+            "business_id": "--1UhMGODdWsrMastO9DZw",
+            "checkin_times": [ 
+                {
+                    "dates": ["2016-04-26", "2016-08-30", "2016-10-15", "2016-11-18"],
+                    "times": ["19:49:16", "18:36:57", "02:45:18", "01:54:50"]
+                },
+                {
+                    "dates": ["2017-04-20", "2017-05-03"],
+                    "times": ["18:39:06", "17:58:02"]
+                },
+                {
+                    "dates": ["2019-03-19"],
+                    "times": ["22:04:48"]
+                }
+             ]
+        },
+        {
+            "business_id": "--EF5N7P70J_UYBTPypYlA",
+            "checkin_times": [ 
+                {
+                    "dates": ["2018-05-25", "2018-09-18"],
+                    "times": ["19:52:07", "16:09:44"]
+                },
+                {
+                    "dates": ["2019-10-18"],
+                    "times": ["21:29:09"]
+                }
+             ]
+        },
+        {
+            "business_id": "--Ni3oJ4VOqfOEu7Sj2Vzg",
+            "checkin_times": [ 
+                {
+                    "dates": ["2019-06-07"],
+                    "times": ["17:54:58"]
+                }
+             ]
+        },
+        {
+            "business_id": "--Y1Adl1YUWfYIRSd8vkmA",
+            "checkin_times": [ 
+                {
+                    "dates": ["2011-05-03", "2011-08-23"],
+                    "times": ["20:54:05", "20:49:45"]
+                },
+                {
+                    "dates": ["2014-12-04"],
+                    "times": ["06:13:01"]
+                },
+                {
+                    "dates": ["2016-11-16"],
+                    "times": ["19:25:55"]
+                }
+             ]
+        },
+        {
+            "business_id": "--YPwqIlRJrhHkJcjY3eiA",
+            "checkin_times": [ 
+               {
+                    "dates": ["2016-06-18", "2016-10-15"],
+                    "times": ["21:35:45", "18:17:51"]
+                }
+             ]
+        },
+        {
+            "business_id": "--e8PjCNhEz32pprnPhCwQ",
+            "checkin_times": [ 
+                {
+                    "dates": ["2015-04-02"],
+                    "times": ["21:45:17"]
+                }
+             ]
+        },
+        {
+            "business_id": "--kinfHwmtdjz03g8B8z8Q",
+            "checkin_times": [ 
+                {
+                    "dates": ["2014-08-27"],
+                    "times": ["17:49:18"]
+                },
+                {
+                    "dates": ["2015-12-19"],
+                    "times": ["21:30:31"]
+                },
+                {
+                    "dates": ["2018-11-27"],
+                    "times": ["15:53:50"]
+                }
+             ]
+        },
+        {
+            "business_id": "--q6datkI-f0EoVheXNEeQ",
+            "checkin_times": [ 
+                {
+                    "dates": ["2014-01-28", "2014-11-16"],
+                    "times": ["20:56:04", "16:11:58"]
+                },
+                {
+                    "dates": ["2015-11-15", "2015-11-15"],
+                    "times": ["19:21:53", "19:33:39"]
+                }
+             ]
+        },
+        {
+            "business_id": "--qvQS4MigHPykD2GV0-zw",
+            "checkin_times": [ 
+                {
+                    "dates": ["2019-04-11"],
+                    "times": ["18:30:12"]
+                }
+             ]
+        },
+        {
+            "business_id": "--wIGbLEhlpl_UeAIyDmZQ",
+            "checkin_times": [ 
+                {
+                    "dates": ["2015-06-06"],
+                    "times": ["20:01:06"]
+                },
+                {
+                    "dates": ["2019-03-14"],
+                    "times": ["22:01:52"]
+                }
+             ]
+        },
+        {
+            "business_id": "-0FA-Qdi3SPYIoJz9UQw-A",
+            "checkin_times": [ 
+                {
+                    "dates": ["2018-09-29", "2018-10-20", "2018-10-20"],
+                    "times": ["18:55:17", "16:48:05", "22:20:24"]
+                }
+             ]
+        },
+        {
+            "business_id": "-0Hj1hb_XW6ybWq2M7QhGA",
+            "checkin_times": [ 
+                {
+                    "dates": ["2011-04-23"],
+                    "times": ["21:11:22"]
+                },
+                {
+                    "dates": ["2014-05-04", "2014-05-11", "2014-06-04"],
+                    "times": ["19:42:48", "19:16:08", "19:14:18"]
+                },
+                {
+                    "dates": ["2015-12-05"],
+                    "times": ["19:22:42"]
+                },
+                {
+                    "dates": ["2017-05-15"],
+                    "times": ["23:19:00"]
+                }
+             ]
+        },
+        {
+            "business_id": "-0KMvRFwDWdVBeTpT11iHw",
+            "checkin_times": [ 
+                {
+                    "dates": ["2012-07-13"],
+                    "times": ["21:43:57"]
+                },
+                {
+                    "dates": ["2016-12-24"],
+                    "times": ["02:27:31"]
+                },
+                {
+                    "dates": ["2017-08-31"],
+                    "times": ["00:35:26"]
+                }
+             ]
+        },
+        {
+            "business_id": "-0LPtgJC31FWMrMv317p0Q",
+            "checkin_times": [ 
+                {
+                    "dates": ["2013-04-13", "2013-08-19", "2013-10-04"],
+                    "times": ["12:35:33", "23:35:49", "19:14:56"]
+                }
+
+             ]
+        },
+        {
+            "business_id": "-0M3o2uWBnQZwd3hmfEwuw",
+            "checkin_times": [ 
+                {
+                    "dates": ["2016-09-10"],
+                    "times": ["19:26:19"]
+                },
+                {
+                    "dates": ["2018-09-08"],
+                    "times": ["14:15:37"]
+                },
+                {
+                    "dates": ["2019-09-13"],
+                    "times": ["22:47:25"]
+                }
+             ]
+        },
+        {
+            "business_id": "-0RRiWDtfnS16AKCtfvBZg",
+            "checkin_times": [ 
+                 {
+                    "dates": ["2017-05-19", "2017-05-19", "2017-08-28", "2017-09-20", "2017-10-01", "2017-10-01", "2017-12-27"],
+                    "times": ["14:30:16", "14:30:25", "15:49:37", "20:19:51", "16:31:05", "16:56:27", "23:33:20"]
+                }
+             ]
+        },
+        {
+            "business_id": "-0Soj75v-XoRcf2ERr8Bmg",
+            "checkin_times": [ 
+                {
+                    "dates": ["2019-06-05"],
+                    "times": ["18:22:49"]
+                }
+             ]
+        },
+        {
+            "business_id": "-0ZumLlFjMh4ZW1z2nXGug",
+            "checkin_times": [ 
+                {
+                    "dates": ["2011-09-24"],
+                    "times": ["21:37:32"]
+                },
+                {
+                    "dates": ["2014-03-10"],
+                    "times": ["20:20:07"]
+                },
+                {
+                    "dates": ["2015-05-27", "2015-08-29"],
+                    "times": ["00:40:24", "17:58:15"]
+                },
+                {
+                    "dates": ["2018-03-16"],
+                    "times": ["15:03:26"]
+                }
+             ]
+        },
+        {
+            "business_id": "-0aOudcaAyac0VJbMX-L1g",
+            "checkin_times": [ 
+                {
+                    "dates": ["2015-03-16", "2015-12-21"],
+                    "times": ["23:51:16", "04:48:01"]
+                },
+                {
+                    "dates": ["2016-10-28", "2016-10-28"],
+                    "times": ["20:22:42", "20:23:00"]
+                }
+             ]
+        },
+        {
+            "business_id": "-0b86isaXMY0v4g-V8GZ9Q",
+            "checkin_times": [ 
+                {
+                    "dates": ["2013-10-22"],
+                    "times": ["16:49:21"]
+                },
+                {
+                    "dates": ["2014-11-21"],
+                    "times": ["17:39:24"]
+                }
+             ]
+        },
+        {
+            "business_id": "-0d-BfFSU0bwLcnMaGRxYw",
+            "checkin_times": [ 
+                {
+                    "dates": ["2014-08-07", "2014-09-16", "2014-10-12"],
+                    "times": ["18:30:48", "20:41:45", "23:22:27"]
+                },
+                {
+                    "dates": ["2015-07-21", "2015-07-21"],
+                    "times": ["20:43:56", "20:45:07"]
+                }
+             ]
+        },
+        {
+            "business_id": "-0jz6c3C6i7RG7Ag22K-Pg",
+            "checkin_times": [ 
+                {
+                    "dates": ["2015-05-02", "2015-05-06", "2015-09-26"],
+                    "times": ["19:49:05", "03:52:18", "01:13:19"]
+                }
+
+             ]
+        },
+        {
+            "business_id": "-0y3MZU2oYP8r1ruDP1bfQ",
+            "checkin_times": [ 
+                {
+                    "dates": ["2015-04-11", "2015-11-21"],
+                    "times": ["13:14:14", "16:05:56"]
+                },
+                {
+                    "dates": ["2016-05-06"],
+                    "times": ["14:10:04"]
+                },
+                {
+                    "dates": ["2017-08-09", "2017-10-21"],
+                    "times": ["15:15:10", "15:12:56"]
+                }
+             ]
+        },
+        {
+            "business_id": "-1BPe8UjF2_l3nVk-DFUjA",
+            "checkin_times": [ 
+                {
+                    "dates": ["2015-12-03"],
+                    "times": ["18:44:00"]
+                },
+                {
+                    "dates": ["2016-03-17", "2016-11-02"],
+                    "times": ["18:19:21", "15:58:38"]
+                }
+             ]
+        },
+        {
+            "business_id": "-1E2CQu_38mkghvmZgCCRw",
+            "checkin_times": [ 
+                {
+                    "dates": ["2019-04-04"],
+                    "times": ["22:02:37"]
+                }
+             ]
+        },
+        {
+            "business_id": "-1wzk43IZ5D9Ysu6kzb5xA",
+            "checkin_times": [ 
+                {
+                    "dates": ["2019-02-27"],
+                    "times": ["14:03:08"]
+                }
+             ]
+        },
+        {
+            "business_id": "-23R9P2eG7VTc6DVLjFKzA",
+            "checkin_times": [ 
+                {
+                    "dates": ["2011-12-21"],
+                    "times": ["19:02:51"]
+                },
+                {
+                    "dates": ["2012-04-15", "2012-04-15"],
+                    "times": ["04:21:39", "14:23:56"]
+                },
+                {
+                    "dates": ["2013-06-30", "2013-10-04"],
+                    "times": ["22:39:51", "20:34:13"]
+                },
+                {
+                    "dates": ["2014-07-16"],
+                    "times": ["02:28:40"]
+                }
+             ]
+        },
+        {
+            "business_id": "-26MGfikhJiTfCI-GqmzhQ",
+            "checkin_times": [ 
+                {
+                    "dates": ["2018-06-13"],
+                    "times": ["20:16:07"]
+                }
+             ]
+        },
+        {
+            "business_id": "-2bLuJsMZ0WhI9daurVQNQ",
+            "checkin_times": [ 
+                {
+                    "dates": ["2015-05-29", "2015-06-01"],
+                    "times": ["16:46:17", "15:03:53"]
+                }
+             ]
+        },
+        {
+            "business_id": "-2hDBMaza_ldqnZdiU06LQ",
+            "checkin_times": [ 
+                {
+                    "dates": ["2011-10-08"],
+                    "times": ["12:02:23"]
+                },
+                {
+                    "dates": ["2014-08-18"],
+                    "times": ["02:11:11"]
+                },
+                {
+                    "dates": ["2016-01-07", "2016-10-21", "2016-12-01", "2016-12-29"],
+                    "times": ["05:27:51", "20:15:55", "03:57:10", "01:54:42"]
+                },
+                {
+                    "dates": ["2018-07-22", "2018-09-07"],
+                    "times": ["19:55:31", "01:42:54"]
+                },
+                {
+                    "dates": ["2019-03-08"],
+                    "times": ["03:41:06"]
+                }
+             ]
+        }
+    ]
+);
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/select-quantified-queries/use-case-4/use-case-4.3.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/select-quantified-queries/use-case-4/use-case-4.3.ddl.sqlpp
new file mode 100644
index 0000000..05a7c76
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/select-quantified-queries/use-case-4/use-case-4.3.ddl.sqlpp
@@ -0,0 +1,22 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+USE TestYelp;
+
+CREATE INDEX IdxYelpCheckinDates ON YelpCheckin (UNNEST checkin_times UNNEST dates);
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/select-quantified-queries/use-case-4/use-case-4.4.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/select-quantified-queries/use-case-4/use-case-4.4.query.sqlpp
new file mode 100644
index 0000000..1832378
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/select-quantified-queries/use-case-4/use-case-4.4.query.sqlpp
@@ -0,0 +1,26 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+SET `compiler.arrayindex` "true";
+
+USE TestYelp;
+
+SELECT C.business_id
+FROM YelpCheckin C
+WHERE SOME D IN C.checkin_times
+SATISFIES "2016-04-26" IN D.dates;
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/select-quantified-queries/use-case-4/use-case-4.5.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/select-quantified-queries/use-case-4/use-case-4.5.query.sqlpp
new file mode 100644
index 0000000..ea254b7
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/select-quantified-queries/use-case-4/use-case-4.5.query.sqlpp
@@ -0,0 +1,29 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+SET `compiler.arrayindex` "true";
+
+USE TestYelp;
+
+SELECT C.business_id
+FROM YelpCheckin C
+WHERE SOME CT IN C.checkin_times
+SATISFIES (
+    SOME D IN CT.dates
+    SATISFIES D = "2016-04-26"
+);
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/select-quantified-queries/use-case-4/use-case-4.6.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/select-quantified-queries/use-case-4/use-case-4.6.query.sqlpp
new file mode 100644
index 0000000..490dbdd
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/select-quantified-queries/use-case-4/use-case-4.6.query.sqlpp
@@ -0,0 +1,29 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+SET `compiler.arrayindex` "true";
+
+USE TestYelp;
+
+SELECT C.business_id
+FROM YelpCheckin C
+WHERE EVERY CT IN C.checkin_times
+SATISFIES (
+    SOME D IN CT.dates
+    SATISFIES "2019-06-07" = D
+);
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/select-quantified-queries/use-case-4/use-case-4.7.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/select-quantified-queries/use-case-4/use-case-4.7.query.sqlpp
new file mode 100644
index 0000000..4032e84
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/select-quantified-queries/use-case-4/use-case-4.7.query.sqlpp
@@ -0,0 +1,29 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+SET `compiler.arrayindex` "true";
+
+USE TestYelp;
+
+SELECT C.business_id
+FROM YelpCheckin C
+WHERE EVERY CT IN C.checkin_times
+SATISFIES (
+    EVERY D IN CT.dates
+    SATISFIES D > "2019-06-05"
+);
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/select-quantified-queries/with-composite-pk/with-composite-pk.1.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/select-quantified-queries/with-composite-pk/with-composite-pk.1.ddl.sqlpp
new file mode 100644
index 0000000..0b3bc97
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/select-quantified-queries/with-composite-pk/with-composite-pk.1.ddl.sqlpp
@@ -0,0 +1,38 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+/*
+ * Description: Verify that we are able to correctly perform the following types of queries using an array->atomic index 
+  *             on a dataset with a composite PK.
+ *     1) Implicit existential quantification w/ membership query.
+ *     2) Explicit existential quantification query, performing a range search.
+ *     3) Universal quantification query, perform a range search.
+ */
+
+DROP DATAVERSE TestYelp IF EXISTS;
+CREATE DATAVERSE TestYelp;
+USE TestYelp;
+
+CREATE TYPE CheckinType AS {
+    checkin_id: smallint,
+    business_id: string,
+    dates: [string]
+};
+
+CREATE DATASET YelpCheckin(CheckinType) PRIMARY KEY checkin_id, business_id;
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/select-quantified-queries/with-composite-pk/with-composite-pk.2.update.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/select-quantified-queries/with-composite-pk/with-composite-pk.2.update.sqlpp
new file mode 100644
index 0000000..6498c14
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/select-quantified-queries/with-composite-pk/with-composite-pk.2.update.sqlpp
@@ -0,0 +1,306 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+USE TestYelp;
+
+INSERT INTO YelpCheckin (
+    [
+        {
+            "checkin_id": 1,
+            "business_id": "--1UhMGODdWsrMastO9DZw",
+            "dates": [
+                "2016-04-26 19:49:16",
+                "2016-08-30 18:36:57",
+                "2016-10-15 02:45:18",
+                "2016-11-18 01:54:50",
+                "2017-04-20 18:39:06",
+                "2017-05-03 17:58:02",
+                "2019-03-19 22:04:48"
+            ]
+        },
+        {
+            "checkin_id": 2,
+            "business_id": "--EF5N7P70J_UYBTPypYlA",
+            "dates": [
+                "2018-05-25 19:52:07",
+                "2018-09-18 16:09:44",
+                "2019-10-18 21:29:09"
+            ]
+        },
+        {
+            "checkin_id": 3,
+            "business_id": "--Ni3oJ4VOqfOEu7Sj2Vzg",
+            "dates": [
+                "2019-06-07 17:54:58"
+            ]
+        },
+        {
+            "checkin_id": 4,
+            "business_id": "--Y1Adl1YUWfYIRSd8vkmA",
+            "dates": [
+                "2011-05-03 20:54:05",
+                "2011-08-23 20:49:45",
+                "2014-12-04 06:13:01",
+                "2016-11-16 19:25:55"
+            ]
+        },
+        {
+            "checkin_id": 5,
+            "business_id": "--YPwqIlRJrhHkJcjY3eiA",
+            "dates": [
+                "2016-06-18 21:35:45",
+                "2016-10-15 18:17:51"
+            ]
+        },
+        {
+            "checkin_id": 6,
+            "business_id": "--e8PjCNhEz32pprnPhCwQ",
+            "dates": [
+                "2015-04-02 21:45:17"
+            ]
+        },
+        {
+            "checkin_id": 7,
+            "business_id": "--kinfHwmtdjz03g8B8z8Q",
+            "dates": [
+                "2014-08-27 17:49:18",
+                "2015-12-19 21:30:31",
+                "2018-11-27 15:53:50"
+            ]
+        },
+        {
+            "checkin_id": 8,
+            "business_id": "--q6datkI-f0EoVheXNEeQ",
+            "dates": [
+                "2014-01-28 20:56:04",
+                "2014-11-16 16:11:58",
+                "2015-11-15 19:21:53",
+                "2015-11-15 19:33:39"
+            ]
+        },
+        {
+            "checkin_id": 9,
+            "business_id": "--qvQS4MigHPykD2GV0-zw",
+            "dates": [
+                "2019-04-11 18:30:12"
+            ]
+        },
+        {
+            "checkin_id": 10,
+            "business_id": "--wIGbLEhlpl_UeAIyDmZQ",
+            "dates": [
+                "2015-06-06 20:01:06",
+                "2019-03-14 22:01:52"
+            ]
+        },
+        {
+            "checkin_id": 11,
+            "business_id": "-0FA-Qdi3SPYIoJz9UQw-A",
+            "dates": [
+                "2018-09-29 18:55:17",
+                "2018-10-20 16:48:05",
+                "2018-10-20 22:20:24"
+            ]
+        },
+        {
+            "checkin_id": 12,
+            "business_id": "-0Hj1hb_XW6ybWq2M7QhGA",
+            "dates": [
+                "2011-04-23 21:11:22",
+                "2014-05-04 19:42:48",
+                "2014-05-11 19:16:08",
+                "2014-06-04 19:14:18",
+                "2015-12-05 19:22:42",
+                "2017-05-15 23:19:00"
+            ]
+        },
+        {
+            "checkin_id": 13,
+            "business_id": "-0KMvRFwDWdVBeTpT11iHw",
+            "dates": [
+                "2012-07-13 21:43:57",
+                "2016-12-24 02:27:31",
+                "2017-08-31 00:35:26"
+            ]
+        },
+        {
+            "checkin_id": 14,
+            "business_id": "-0LPtgJC31FWMrMv317p0Q",
+            "dates": [
+                "2013-04-13 12:35:33",
+                "2013-08-19 23:35:49",
+                "2013-10-04 19:14:56"
+            ]
+        },
+        {
+            "checkin_id": 15,
+            "business_id": "-0M3o2uWBnQZwd3hmfEwuw",
+            "dates": [
+                "2016-09-10 19:26:19",
+                "2018-09-08 14:15:37",
+                "2019-09-13 22:47:25"
+            ]
+        },
+        {
+            "checkin_id": 16,
+            "business_id": "-0RRiWDtfnS16AKCtfvBZg",
+            "dates": [
+                "2017-05-19 14:30:16",
+                "2017-05-19 14:30:25",
+                "2017-08-28 15:49:37",
+                "2017-09-20 20:19:51",
+                "2017-10-01 16:31:05",
+                "2017-10-01 16:56:27",
+                "2017-12-27 23:33:20"
+            ]
+        },
+        {
+            "checkin_id": 17,
+            "business_id": "-0Soj75v-XoRcf2ERr8Bmg",
+            "dates": [
+                "2019-06-05 18:22:49"
+            ]
+        },
+        {
+            "checkin_id": 18,
+            "business_id": "-0ZumLlFjMh4ZW1z2nXGug",
+            "dates": [
+                "2011-09-24 21:37:32",
+                "2014-03-10 20:20:07",
+                "2015-05-27 00:40:24",
+                "2015-08-29 17:58:15",
+                "2018-03-16 15:03:26"
+            ]
+        },
+        {
+            "checkin_id": 19,
+            "business_id": "-0aOudcaAyac0VJbMX-L1g",
+            "dates": [
+                "2015-03-16 23:51:16",
+                "2015-12-21 04:48:01",
+                "2016-10-28 20:22:42",
+                "2016-10-28 20:23:00"
+            ]
+        },
+        {
+            "checkin_id": 20,
+            "business_id": "-0b86isaXMY0v4g-V8GZ9Q",
+            "dates": [
+                "2013-10-22 16:49:21",
+                "2014-11-21 17:39:24"
+            ]
+        },
+        {
+            "checkin_id": 21,
+            "business_id": "-0d-BfFSU0bwLcnMaGRxYw",
+            "dates": [
+                "2014-08-07 18:30:48",
+                "2014-09-16 20:41:45",
+                "2014-10-12 23:22:27",
+                "2015-07-21 20:43:56",
+                "2015-07-21 20:45:07"
+            ]
+        },
+        {
+            "checkin_id": 22,
+            "business_id": "-0jz6c3C6i7RG7Ag22K-Pg",
+            "dates": [
+                "2015-05-02 19:49:05",
+                "2015-05-06 03:52:18",
+                "2015-09-26 01:13:19"
+            ]
+        },
+        {
+            "checkin_id": 23,
+            "business_id": "-0y3MZU2oYP8r1ruDP1bfQ",
+            "dates": [
+                "2015-04-11 13:14:14",
+                "2015-11-21 16:05:56",
+                "2016-05-06 14:10:04",
+                "2017-08-09 15:15:10",
+                "2017-10-21 15:12:56"
+            ]
+        },
+        {
+            "checkin_id": 24,
+            "business_id": "-1BPe8UjF2_l3nVk-DFUjA",
+            "dates": [
+                "2015-12-03 18:44:00",
+                "2016-03-17 18:19:21",
+                "2016-11-02 15:58:38"
+            ]
+        },
+        {
+            "checkin_id": 25,
+            "business_id": "-1E2CQu_38mkghvmZgCCRw",
+            "dates": [
+                "2019-04-04 22:02:37"
+            ]
+        },
+        {
+            "checkin_id": 26,
+            "business_id": "-1wzk43IZ5D9Ysu6kzb5xA",
+            "dates": [
+                "2019-02-27 14:03:08"
+            ]
+        },
+        {
+            "checkin_id": 27,
+            "business_id": "-23R9P2eG7VTc6DVLjFKzA",
+            "dates": [
+                "2011-12-21 19:02:51",
+                "2012-04-15 04:21:39",
+                "2012-04-15 14:23:56",
+                "2013-06-30 22:39:51",
+                "2013-10-04 20:34:13",
+                "2014-07-16 02:28:40"
+            ]
+        },
+        {
+            "checkin_id": 28,
+            "business_id": "-26MGfikhJiTfCI-GqmzhQ",
+            "dates": [
+                "2018-06-13 20:16:07"
+            ]
+        },
+        {
+            "checkin_id": 29,
+            "business_id": "-2bLuJsMZ0WhI9daurVQNQ",
+            "dates": [
+                "2015-05-29 16:46:17",
+                "2015-06-01 15:03:53"
+            ]
+        },
+        {
+            "checkin_id": 30,
+            "business_id": "-2hDBMaza_ldqnZdiU06LQ",
+            "dates": [
+                "2011-10-08 12:02:23",
+                "2014-08-18 02:11:11",
+                "2016-01-07 05:27:51",
+                "2016-10-21 20:15:55",
+                "2016-12-01 03:57:10",
+                "2016-12-29 01:54:42",
+                "2018-07-22 19:55:31",
+                "2018-09-07 01:42:54",
+                "2019-03-08 03:41:06"
+            ]
+        }
+    ]
+);
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/select-quantified-queries/with-composite-pk/with-composite-pk.3.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/select-quantified-queries/with-composite-pk/with-composite-pk.3.ddl.sqlpp
new file mode 100644
index 0000000..a0fc1bf
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/select-quantified-queries/with-composite-pk/with-composite-pk.3.ddl.sqlpp
@@ -0,0 +1,22 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+USE TestYelp;
+
+CREATE INDEX IdxYelpCheckinDates ON YelpCheckin (UNNEST dates);
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/select-quantified-queries/with-composite-pk/with-composite-pk.4.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/select-quantified-queries/with-composite-pk/with-composite-pk.4.query.sqlpp
new file mode 100644
index 0000000..dc5fc32
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/select-quantified-queries/with-composite-pk/with-composite-pk.4.query.sqlpp
@@ -0,0 +1,25 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+SET `compiler.arrayindex` "true";
+
+USE TestYelp;
+
+SELECT C.business_id
+FROM YelpCheckin C 
+WHERE "2016-04-26 19:49:16" IN C.dates;
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/select-quantified-queries/with-composite-pk/with-composite-pk.5.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/select-quantified-queries/with-composite-pk/with-composite-pk.5.query.sqlpp
new file mode 100644
index 0000000..2921d18
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/select-quantified-queries/with-composite-pk/with-composite-pk.5.query.sqlpp
@@ -0,0 +1,26 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+SET `compiler.arrayindex` "true";
+
+USE TestYelp;
+
+SELECT COUNT(*)
+FROM YelpCheckin C
+WHERE SOME D IN C.dates
+SATISFIES D > "2016" AND D < "2017";
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/select-quantified-queries/with-composite-pk/with-composite-pk.6.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/select-quantified-queries/with-composite-pk/with-composite-pk.6.query.sqlpp
new file mode 100644
index 0000000..e64a93c
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/select-quantified-queries/with-composite-pk/with-composite-pk.6.query.sqlpp
@@ -0,0 +1,26 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+SET `compiler.arrayindex` "true";
+
+USE TestYelp;
+
+SELECT COUNT(*)
+FROM YelpCheckin C
+WHERE EVERY D IN C.dates
+SATISFIES D > "2016" AND D < "2017";
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/select-quantified-queries/with-open-index/with-open-index.1.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/select-quantified-queries/with-open-index/with-open-index.1.ddl.sqlpp
new file mode 100644
index 0000000..4f8b1f1
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/select-quantified-queries/with-open-index/with-open-index.1.ddl.sqlpp
@@ -0,0 +1,35 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+/*
+ * Description: Verify that we are able to correctly perform the following types of queries using an open array->atomic index.
+ *     1) Implicit existential quantification w/ membership query.
+ *     2) Explicit existential quantification query, performing a range search.
+ *     3) Universal quantification query, perform a range search.
+ */
+
+DROP DATAVERSE TestYelp IF EXISTS;
+CREATE DATAVERSE TestYelp;
+USE TestYelp;
+
+CREATE TYPE CheckinType AS {
+    checkin_id: uuid
+};
+
+CREATE DATASET YelpCheckin(CheckinType) PRIMARY KEY checkin_id AUTOGENERATED;
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/select-quantified-queries/with-open-index/with-open-index.2.update.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/select-quantified-queries/with-open-index/with-open-index.2.update.sqlpp
new file mode 100644
index 0000000..b75bf1b
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/select-quantified-queries/with-open-index/with-open-index.2.update.sqlpp
@@ -0,0 +1,276 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+USE TestYelp;
+
+INSERT INTO YelpCheckin (
+    [
+        {
+            "business_id": "--1UhMGODdWsrMastO9DZw",
+            "dates": [
+                "2016-04-26 19:49:16",
+                "2016-08-30 18:36:57",
+                "2016-10-15 02:45:18",
+                "2016-11-18 01:54:50",
+                "2017-04-20 18:39:06",
+                "2017-05-03 17:58:02",
+                "2019-03-19 22:04:48"
+            ]
+        },
+        {
+            "business_id": "--EF5N7P70J_UYBTPypYlA",
+            "dates": [
+                "2018-05-25 19:52:07",
+                "2018-09-18 16:09:44",
+                "2019-10-18 21:29:09"
+            ]
+        },
+        {
+            "business_id": "--Ni3oJ4VOqfOEu7Sj2Vzg",
+            "dates": [
+                "2019-06-07 17:54:58"
+            ]
+        },
+        {
+            "business_id": "--Y1Adl1YUWfYIRSd8vkmA",
+            "dates": [
+                "2011-05-03 20:54:05",
+                "2011-08-23 20:49:45",
+                "2014-12-04 06:13:01",
+                "2016-11-16 19:25:55"
+            ]
+        },
+        {
+            "business_id": "--YPwqIlRJrhHkJcjY3eiA",
+            "dates": [
+                "2016-06-18 21:35:45",
+                "2016-10-15 18:17:51"
+            ]
+        },
+        {
+            "business_id": "--e8PjCNhEz32pprnPhCwQ",
+            "dates": [
+                "2015-04-02 21:45:17"
+            ]
+        },
+        {
+            "business_id": "--kinfHwmtdjz03g8B8z8Q",
+            "dates": [
+                "2014-08-27 17:49:18",
+                "2015-12-19 21:30:31",
+                "2018-11-27 15:53:50"
+            ]
+        },
+        {
+            "business_id": "--q6datkI-f0EoVheXNEeQ",
+            "dates": [
+                "2014-01-28 20:56:04",
+                "2014-11-16 16:11:58",
+                "2015-11-15 19:21:53",
+                "2015-11-15 19:33:39"
+            ]
+        },
+        {
+            "business_id": "--qvQS4MigHPykD2GV0-zw",
+            "dates": [
+                "2019-04-11 18:30:12"
+            ]
+        },
+        {
+            "business_id": "--wIGbLEhlpl_UeAIyDmZQ",
+            "dates": [
+                "2015-06-06 20:01:06",
+                "2019-03-14 22:01:52"
+            ]
+        },
+        {
+            "business_id": "-0FA-Qdi3SPYIoJz9UQw-A",
+            "dates": [
+                "2018-09-29 18:55:17",
+                "2018-10-20 16:48:05",
+                "2018-10-20 22:20:24"
+            ]
+        },
+        {
+            "business_id": "-0Hj1hb_XW6ybWq2M7QhGA",
+            "dates": [
+                "2011-04-23 21:11:22",
+                "2014-05-04 19:42:48",
+                "2014-05-11 19:16:08",
+                "2014-06-04 19:14:18",
+                "2015-12-05 19:22:42",
+                "2017-05-15 23:19:00"
+            ]
+        },
+        {
+            "business_id": "-0KMvRFwDWdVBeTpT11iHw",
+            "dates": [
+                "2012-07-13 21:43:57",
+                "2016-12-24 02:27:31",
+                "2017-08-31 00:35:26"
+            ]
+        },
+        {
+            "business_id": "-0LPtgJC31FWMrMv317p0Q",
+            "dates": [
+                "2013-04-13 12:35:33",
+                "2013-08-19 23:35:49",
+                "2013-10-04 19:14:56"
+            ]
+        },
+        {
+            "business_id": "-0M3o2uWBnQZwd3hmfEwuw",
+            "dates": [
+                "2016-09-10 19:26:19",
+                "2018-09-08 14:15:37",
+                "2019-09-13 22:47:25"
+            ]
+        },
+        {
+            "business_id": "-0RRiWDtfnS16AKCtfvBZg",
+            "dates": [
+                "2017-05-19 14:30:16",
+                "2017-05-19 14:30:25",
+                "2017-08-28 15:49:37",
+                "2017-09-20 20:19:51",
+                "2017-10-01 16:31:05",
+                "2017-10-01 16:56:27",
+                "2017-12-27 23:33:20"
+            ]
+        },
+        {
+            "business_id": "-0Soj75v-XoRcf2ERr8Bmg",
+            "dates": [
+                "2019-06-05 18:22:49"
+            ]
+        },
+        {
+            "business_id": "-0ZumLlFjMh4ZW1z2nXGug",
+            "dates": [
+                "2011-09-24 21:37:32",
+                "2014-03-10 20:20:07",
+                "2015-05-27 00:40:24",
+                "2015-08-29 17:58:15",
+                "2018-03-16 15:03:26"
+            ]
+        },
+        {
+            "business_id": "-0aOudcaAyac0VJbMX-L1g",
+            "dates": [
+                "2015-03-16 23:51:16",
+                "2015-12-21 04:48:01",
+                "2016-10-28 20:22:42",
+                "2016-10-28 20:23:00"
+            ]
+        },
+        {
+            "business_id": "-0b86isaXMY0v4g-V8GZ9Q",
+            "dates": [
+                "2013-10-22 16:49:21",
+                "2014-11-21 17:39:24"
+            ]
+        },
+        {
+            "business_id": "-0d-BfFSU0bwLcnMaGRxYw",
+            "dates": [
+                "2014-08-07 18:30:48",
+                "2014-09-16 20:41:45",
+                "2014-10-12 23:22:27",
+                "2015-07-21 20:43:56",
+                "2015-07-21 20:45:07"
+            ]
+        },
+        {
+            "business_id": "-0jz6c3C6i7RG7Ag22K-Pg",
+            "dates": [
+                "2015-05-02 19:49:05",
+                "2015-05-06 03:52:18",
+                "2015-09-26 01:13:19"
+            ]
+        },
+        {
+            "business_id": "-0y3MZU2oYP8r1ruDP1bfQ",
+            "dates": [
+                "2015-04-11 13:14:14",
+                "2015-11-21 16:05:56",
+                "2016-05-06 14:10:04",
+                "2017-08-09 15:15:10",
+                "2017-10-21 15:12:56"
+            ]
+        },
+        {
+            "business_id": "-1BPe8UjF2_l3nVk-DFUjA",
+            "dates": [
+                "2015-12-03 18:44:00",
+                "2016-03-17 18:19:21",
+                "2016-11-02 15:58:38"
+            ]
+        },
+        {
+            "business_id": "-1E2CQu_38mkghvmZgCCRw",
+            "dates": [
+                "2019-04-04 22:02:37"
+            ]
+        },
+        {
+            "business_id": "-1wzk43IZ5D9Ysu6kzb5xA",
+            "dates": [
+                "2019-02-27 14:03:08"
+            ]
+        },
+        {
+            "business_id": "-23R9P2eG7VTc6DVLjFKzA",
+            "dates": [
+                "2011-12-21 19:02:51",
+                "2012-04-15 04:21:39",
+                "2012-04-15 14:23:56",
+                "2013-06-30 22:39:51",
+                "2013-10-04 20:34:13",
+                "2014-07-16 02:28:40"
+            ]
+        },
+        {
+            "business_id": "-26MGfikhJiTfCI-GqmzhQ",
+            "dates": [
+                "2018-06-13 20:16:07"
+            ]
+        },
+        {
+            "business_id": "-2bLuJsMZ0WhI9daurVQNQ",
+            "dates": [
+                "2015-05-29 16:46:17",
+                "2015-06-01 15:03:53"
+            ]
+        },
+        {
+            "business_id": "-2hDBMaza_ldqnZdiU06LQ",
+            "dates": [
+                "2011-10-08 12:02:23",
+                "2014-08-18 02:11:11",
+                "2016-01-07 05:27:51",
+                "2016-10-21 20:15:55",
+                "2016-12-01 03:57:10",
+                "2016-12-29 01:54:42",
+                "2018-07-22 19:55:31",
+                "2018-09-07 01:42:54",
+                "2019-03-08 03:41:06"
+            ]
+        }
+    ]
+);
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/select-quantified-queries/with-open-index/with-open-index.3.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/select-quantified-queries/with-open-index/with-open-index.3.ddl.sqlpp
new file mode 100644
index 0000000..b933d8a
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/select-quantified-queries/with-open-index/with-open-index.3.ddl.sqlpp
@@ -0,0 +1,22 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+USE TestYelp;
+
+CREATE INDEX IdxYelpCheckinDates ON YelpCheckin (UNNEST dates : string ?) ;
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/select-quantified-queries/with-open-index/with-open-index.4.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/select-quantified-queries/with-open-index/with-open-index.4.query.sqlpp
new file mode 100644
index 0000000..dc5fc32
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/select-quantified-queries/with-open-index/with-open-index.4.query.sqlpp
@@ -0,0 +1,25 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+SET `compiler.arrayindex` "true";
+
+USE TestYelp;
+
+SELECT C.business_id
+FROM YelpCheckin C 
+WHERE "2016-04-26 19:49:16" IN C.dates;
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/select-quantified-queries/with-open-index/with-open-index.5.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/select-quantified-queries/with-open-index/with-open-index.5.query.sqlpp
new file mode 100644
index 0000000..2921d18
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/select-quantified-queries/with-open-index/with-open-index.5.query.sqlpp
@@ -0,0 +1,26 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+SET `compiler.arrayindex` "true";
+
+USE TestYelp;
+
+SELECT COUNT(*)
+FROM YelpCheckin C
+WHERE SOME D IN C.dates
+SATISFIES D > "2016" AND D < "2017";
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/select-quantified-queries/with-open-index/with-open-index.6.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/select-quantified-queries/with-open-index/with-open-index.6.query.sqlpp
new file mode 100644
index 0000000..e64a93c
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/select-quantified-queries/with-open-index/with-open-index.6.query.sqlpp
@@ -0,0 +1,26 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+SET `compiler.arrayindex` "true";
+
+USE TestYelp;
+
+SELECT COUNT(*)
+FROM YelpCheckin C
+WHERE EVERY D IN C.dates
+SATISFIES D > "2016" AND D < "2017";
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/select-unnest-queries/closed/use-case-1/use-case-1.1.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/select-unnest-queries/closed/use-case-1/use-case-1.1.ddl.sqlpp
new file mode 100644
index 0000000..ae09aa5
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/select-unnest-queries/closed/use-case-1/use-case-1.1.ddl.sqlpp
@@ -0,0 +1,36 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+/*
+ * Description: Verify that we are able to correctly perform the following types of queries using an array->atomic index.
+ *     1) UNNEST query, performing a point search.
+ *     2) UNNEST query, performing a range search.
+ */
+
+DROP DATAVERSE TestYelp IF EXISTS;
+CREATE DATAVERSE TestYelp;
+USE TestYelp;
+
+CREATE TYPE CheckinType AS {
+    checkin_id: uuid,
+    business_id: string,
+    dates: [string]
+};
+
+CREATE DATASET YelpCheckin(CheckinType) PRIMARY KEY checkin_id AUTOGENERATED;
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/select-unnest-queries/closed/use-case-1/use-case-1.2.update.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/select-unnest-queries/closed/use-case-1/use-case-1.2.update.sqlpp
new file mode 100644
index 0000000..b75bf1b
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/select-unnest-queries/closed/use-case-1/use-case-1.2.update.sqlpp
@@ -0,0 +1,276 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+USE TestYelp;
+
+INSERT INTO YelpCheckin (
+    [
+        {
+            "business_id": "--1UhMGODdWsrMastO9DZw",
+            "dates": [
+                "2016-04-26 19:49:16",
+                "2016-08-30 18:36:57",
+                "2016-10-15 02:45:18",
+                "2016-11-18 01:54:50",
+                "2017-04-20 18:39:06",
+                "2017-05-03 17:58:02",
+                "2019-03-19 22:04:48"
+            ]
+        },
+        {
+            "business_id": "--EF5N7P70J_UYBTPypYlA",
+            "dates": [
+                "2018-05-25 19:52:07",
+                "2018-09-18 16:09:44",
+                "2019-10-18 21:29:09"
+            ]
+        },
+        {
+            "business_id": "--Ni3oJ4VOqfOEu7Sj2Vzg",
+            "dates": [
+                "2019-06-07 17:54:58"
+            ]
+        },
+        {
+            "business_id": "--Y1Adl1YUWfYIRSd8vkmA",
+            "dates": [
+                "2011-05-03 20:54:05",
+                "2011-08-23 20:49:45",
+                "2014-12-04 06:13:01",
+                "2016-11-16 19:25:55"
+            ]
+        },
+        {
+            "business_id": "--YPwqIlRJrhHkJcjY3eiA",
+            "dates": [
+                "2016-06-18 21:35:45",
+                "2016-10-15 18:17:51"
+            ]
+        },
+        {
+            "business_id": "--e8PjCNhEz32pprnPhCwQ",
+            "dates": [
+                "2015-04-02 21:45:17"
+            ]
+        },
+        {
+            "business_id": "--kinfHwmtdjz03g8B8z8Q",
+            "dates": [
+                "2014-08-27 17:49:18",
+                "2015-12-19 21:30:31",
+                "2018-11-27 15:53:50"
+            ]
+        },
+        {
+            "business_id": "--q6datkI-f0EoVheXNEeQ",
+            "dates": [
+                "2014-01-28 20:56:04",
+                "2014-11-16 16:11:58",
+                "2015-11-15 19:21:53",
+                "2015-11-15 19:33:39"
+            ]
+        },
+        {
+            "business_id": "--qvQS4MigHPykD2GV0-zw",
+            "dates": [
+                "2019-04-11 18:30:12"
+            ]
+        },
+        {
+            "business_id": "--wIGbLEhlpl_UeAIyDmZQ",
+            "dates": [
+                "2015-06-06 20:01:06",
+                "2019-03-14 22:01:52"
+            ]
+        },
+        {
+            "business_id": "-0FA-Qdi3SPYIoJz9UQw-A",
+            "dates": [
+                "2018-09-29 18:55:17",
+                "2018-10-20 16:48:05",
+                "2018-10-20 22:20:24"
+            ]
+        },
+        {
+            "business_id": "-0Hj1hb_XW6ybWq2M7QhGA",
+            "dates": [
+                "2011-04-23 21:11:22",
+                "2014-05-04 19:42:48",
+                "2014-05-11 19:16:08",
+                "2014-06-04 19:14:18",
+                "2015-12-05 19:22:42",
+                "2017-05-15 23:19:00"
+            ]
+        },
+        {
+            "business_id": "-0KMvRFwDWdVBeTpT11iHw",
+            "dates": [
+                "2012-07-13 21:43:57",
+                "2016-12-24 02:27:31",
+                "2017-08-31 00:35:26"
+            ]
+        },
+        {
+            "business_id": "-0LPtgJC31FWMrMv317p0Q",
+            "dates": [
+                "2013-04-13 12:35:33",
+                "2013-08-19 23:35:49",
+                "2013-10-04 19:14:56"
+            ]
+        },
+        {
+            "business_id": "-0M3o2uWBnQZwd3hmfEwuw",
+            "dates": [
+                "2016-09-10 19:26:19",
+                "2018-09-08 14:15:37",
+                "2019-09-13 22:47:25"
+            ]
+        },
+        {
+            "business_id": "-0RRiWDtfnS16AKCtfvBZg",
+            "dates": [
+                "2017-05-19 14:30:16",
+                "2017-05-19 14:30:25",
+                "2017-08-28 15:49:37",
+                "2017-09-20 20:19:51",
+                "2017-10-01 16:31:05",
+                "2017-10-01 16:56:27",
+                "2017-12-27 23:33:20"
+            ]
+        },
+        {
+            "business_id": "-0Soj75v-XoRcf2ERr8Bmg",
+            "dates": [
+                "2019-06-05 18:22:49"
+            ]
+        },
+        {
+            "business_id": "-0ZumLlFjMh4ZW1z2nXGug",
+            "dates": [
+                "2011-09-24 21:37:32",
+                "2014-03-10 20:20:07",
+                "2015-05-27 00:40:24",
+                "2015-08-29 17:58:15",
+                "2018-03-16 15:03:26"
+            ]
+        },
+        {
+            "business_id": "-0aOudcaAyac0VJbMX-L1g",
+            "dates": [
+                "2015-03-16 23:51:16",
+                "2015-12-21 04:48:01",
+                "2016-10-28 20:22:42",
+                "2016-10-28 20:23:00"
+            ]
+        },
+        {
+            "business_id": "-0b86isaXMY0v4g-V8GZ9Q",
+            "dates": [
+                "2013-10-22 16:49:21",
+                "2014-11-21 17:39:24"
+            ]
+        },
+        {
+            "business_id": "-0d-BfFSU0bwLcnMaGRxYw",
+            "dates": [
+                "2014-08-07 18:30:48",
+                "2014-09-16 20:41:45",
+                "2014-10-12 23:22:27",
+                "2015-07-21 20:43:56",
+                "2015-07-21 20:45:07"
+            ]
+        },
+        {
+            "business_id": "-0jz6c3C6i7RG7Ag22K-Pg",
+            "dates": [
+                "2015-05-02 19:49:05",
+                "2015-05-06 03:52:18",
+                "2015-09-26 01:13:19"
+            ]
+        },
+        {
+            "business_id": "-0y3MZU2oYP8r1ruDP1bfQ",
+            "dates": [
+                "2015-04-11 13:14:14",
+                "2015-11-21 16:05:56",
+                "2016-05-06 14:10:04",
+                "2017-08-09 15:15:10",
+                "2017-10-21 15:12:56"
+            ]
+        },
+        {
+            "business_id": "-1BPe8UjF2_l3nVk-DFUjA",
+            "dates": [
+                "2015-12-03 18:44:00",
+                "2016-03-17 18:19:21",
+                "2016-11-02 15:58:38"
+            ]
+        },
+        {
+            "business_id": "-1E2CQu_38mkghvmZgCCRw",
+            "dates": [
+                "2019-04-04 22:02:37"
+            ]
+        },
+        {
+            "business_id": "-1wzk43IZ5D9Ysu6kzb5xA",
+            "dates": [
+                "2019-02-27 14:03:08"
+            ]
+        },
+        {
+            "business_id": "-23R9P2eG7VTc6DVLjFKzA",
+            "dates": [
+                "2011-12-21 19:02:51",
+                "2012-04-15 04:21:39",
+                "2012-04-15 14:23:56",
+                "2013-06-30 22:39:51",
+                "2013-10-04 20:34:13",
+                "2014-07-16 02:28:40"
+            ]
+        },
+        {
+            "business_id": "-26MGfikhJiTfCI-GqmzhQ",
+            "dates": [
+                "2018-06-13 20:16:07"
+            ]
+        },
+        {
+            "business_id": "-2bLuJsMZ0WhI9daurVQNQ",
+            "dates": [
+                "2015-05-29 16:46:17",
+                "2015-06-01 15:03:53"
+            ]
+        },
+        {
+            "business_id": "-2hDBMaza_ldqnZdiU06LQ",
+            "dates": [
+                "2011-10-08 12:02:23",
+                "2014-08-18 02:11:11",
+                "2016-01-07 05:27:51",
+                "2016-10-21 20:15:55",
+                "2016-12-01 03:57:10",
+                "2016-12-29 01:54:42",
+                "2018-07-22 19:55:31",
+                "2018-09-07 01:42:54",
+                "2019-03-08 03:41:06"
+            ]
+        }
+    ]
+);
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/select-unnest-queries/closed/use-case-1/use-case-1.3.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/select-unnest-queries/closed/use-case-1/use-case-1.3.ddl.sqlpp
new file mode 100644
index 0000000..a0fc1bf
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/select-unnest-queries/closed/use-case-1/use-case-1.3.ddl.sqlpp
@@ -0,0 +1,22 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+USE TestYelp;
+
+CREATE INDEX IdxYelpCheckinDates ON YelpCheckin (UNNEST dates);
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/select-unnest-queries/closed/use-case-1/use-case-1.4.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/select-unnest-queries/closed/use-case-1/use-case-1.4.query.sqlpp
new file mode 100644
index 0000000..ec79bc6
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/select-unnest-queries/closed/use-case-1/use-case-1.4.query.sqlpp
@@ -0,0 +1,25 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+SET `compiler.arrayindex` "true";
+
+USE TestYelp;
+
+SELECT C.business_id
+FROM YelpCheckin C, C.dates D
+WHERE "2016-04-26 19:49:16" = D;
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/select-unnest-queries/closed/use-case-1/use-case-1.5.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/select-unnest-queries/closed/use-case-1/use-case-1.5.query.sqlpp
new file mode 100644
index 0000000..cba5055
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/select-unnest-queries/closed/use-case-1/use-case-1.5.query.sqlpp
@@ -0,0 +1,25 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+SET `compiler.arrayindex` "true";
+
+USE TestYelp;
+
+SELECT COUNT(*)
+FROM YelpCheckin C, C.dates D
+WHERE D > "2016" AND D < "2017";
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/select-unnest-queries/closed/use-case-2/use-case-2.1.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/select-unnest-queries/closed/use-case-2/use-case-2.1.ddl.sqlpp
new file mode 100644
index 0000000..58dc001
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/select-unnest-queries/closed/use-case-2/use-case-2.1.ddl.sqlpp
@@ -0,0 +1,39 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+/*
+ * Description: Verify that we are able to correctly perform the following types of queries using a record->array->atomic index.
+ *     1) UNNEST query, performing a point search.
+ *     2) UNNEST query, performing a range search.
+ */
+
+DROP DATAVERSE TestYelp IF EXISTS;
+CREATE DATAVERSE TestYelp;
+USE TestYelp;
+
+CREATE TYPE CheckinType AS {
+    checkin_id: uuid,
+    business_id: string,
+    checkin_times: {
+        dates: [string],
+        times: [string]
+    }
+};
+
+CREATE DATASET YelpCheckin(CheckinType) PRIMARY KEY checkin_id AUTOGENERATED;
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/select-unnest-queries/closed/use-case-2/use-case-2.2.update.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/select-unnest-queries/closed/use-case-2/use-case-2.2.update.sqlpp
new file mode 100644
index 0000000..2549e8f
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/select-unnest-queries/closed/use-case-2/use-case-2.2.update.sqlpp
@@ -0,0 +1,497 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+USE TestYelp;
+
+INSERT INTO YelpCheckin (
+    [
+        {
+            "business_id": "--1UhMGODdWsrMastO9DZw",
+            "checkin_times": {
+                "dates": [
+                    "2016-04-26",
+                    "2016-08-30",
+                    "2016-10-15",
+                    "2016-11-18",
+                    "2017-04-20",
+                    "2017-05-03",
+                    "2019-03-19"
+                ],
+                "times": [
+                    "19:49:16",
+                    "18:36:57",
+                    "02:45:18",
+                    "01:54:50",
+                    "18:39:06",
+                    "17:58:02",
+                    "22:04:48"
+                ]
+            }
+        },
+        {
+            "business_id": "--EF5N7P70J_UYBTPypYlA",
+            "checkin_times": {
+                "dates": [
+                    "2018-05-25",
+                    "2018-09-18",
+                    "2019-10-18"
+                ],
+                "times": [
+                    "19:52:07",
+                    "16:09:44",
+                    "21:29:09"
+                ]
+            }
+        },
+        {
+            "business_id": "--Ni3oJ4VOqfOEu7Sj2Vzg",
+            "checkin_times": {
+                "dates": [
+                    "2019-06-07"
+                ],
+                "times": [
+                    "17:54:58"
+                ]
+            }
+        },
+        {
+            "business_id": "--Y1Adl1YUWfYIRSd8vkmA",
+            "checkin_times": {
+                "dates": [
+                    "2011-05-03",
+                    "2011-08-23",
+                    "2014-12-04",
+                    "2016-11-16"
+                ],
+                "times": [
+                    "20:54:05",
+                    "20:49:45",
+                    "06:13:01",
+                    "19:25:55"
+                ]
+            }
+        },
+        {
+            "business_id": "--YPwqIlRJrhHkJcjY3eiA",
+            "checkin_times": {
+                "dates": [
+                    "2016-06-18",
+                    "2016-10-15"
+                ],
+                "times": [
+                    "21:35:45",
+                    "18:17:51"
+                ]
+            }
+        },
+        {
+            "business_id": "--e8PjCNhEz32pprnPhCwQ",
+            "checkin_times": {
+                "dates": [
+                    "2015-04-02"
+                ],
+                "times": [
+                    "21:45:17"
+                ]
+            }
+        },
+        {
+            "business_id": "--kinfHwmtdjz03g8B8z8Q",
+            "checkin_times": {
+                "dates": [
+                    "2014-08-27",
+                    "2015-12-19",
+                    "2018-11-27"
+                ],
+                "times": [
+                    "17:49:18",
+                    "21:30:31",
+                    "15:53:50"
+                ]
+            }
+        },
+        {
+            "business_id": "--q6datkI-f0EoVheXNEeQ",
+            "checkin_times": {
+                "dates": [
+                    "2014-01-28",
+                    "2014-11-16",
+                    "2015-11-15",
+                    "2015-11-15"
+                ],
+                "times": [
+                    "20:56:04",
+                    "16:11:58",
+                    "19:21:53",
+                    "19:33:39"
+                ]
+            }
+        },
+        {
+            "business_id": "--qvQS4MigHPykD2GV0-zw",
+            "checkin_times": {
+                "dates": [
+                    "2019-04-11"
+                ],
+                "times": [
+                    "18:30:12"
+                ]
+            }
+        },
+        {
+            "business_id": "--wIGbLEhlpl_UeAIyDmZQ",
+            "checkin_times": {
+                "dates": [
+                    "2015-06-06",
+                    "2019-03-14"
+                ],
+                "times": [
+                    "20:01:06",
+                    "22:01:52"
+                ]
+            }
+        },
+        {
+            "business_id": "-0FA-Qdi3SPYIoJz9UQw-A",
+            "checkin_times": {
+                "dates": [
+                    "2018-09-29",
+                    "2018-10-20",
+                    "2018-10-20"
+                ],
+                "times": [
+                    "18:55:17",
+                    "16:48:05",
+                    "22:20:24"
+                ]
+            }
+        },
+        {
+            "business_id": "-0Hj1hb_XW6ybWq2M7QhGA",
+            "checkin_times": {
+                "dates": [
+                    "2011-04-23",
+                    "2014-05-04",
+                    "2014-05-11",
+                    "2014-06-04",
+                    "2015-12-05",
+                    "2017-05-15"
+                ],
+                "times": [
+                    "21:11:22",
+                    "19:42:48",
+                    "19:16:08",
+                    "19:14:18",
+                    "19:22:42",
+                    "23:19:00"
+                ]
+            }
+        },
+        {
+            "business_id": "-0KMvRFwDWdVBeTpT11iHw",
+            "checkin_times": {
+                "dates": [
+                    "2012-07-13",
+                    "2016-12-24",
+                    "2017-08-31"
+                ],
+                "times": [
+                    "21:43:57",
+                    "02:27:31",
+                    "00:35:26"
+                ]
+            }
+        },
+        {
+            "business_id": "-0LPtgJC31FWMrMv317p0Q",
+            "checkin_times": {
+                "dates": [
+                    "2013-04-13",
+                    "2013-08-19",
+                    "2013-10-04"
+                ],
+                "times": [
+                    "12:35:33",
+                    "23:35:49",
+                    "19:14:56"
+                ]
+            }
+        },
+        {
+            "business_id": "-0M3o2uWBnQZwd3hmfEwuw",
+            "checkin_times": {
+                "dates": [
+                    "2016-09-10",
+                    "2018-09-08",
+                    "2019-09-13"
+                ],
+                "times": [
+                    "19:26:19",
+                    "14:15:37",
+                    "22:47:25"
+                ]
+            }
+        },
+        {
+            "business_id": "-0RRiWDtfnS16AKCtfvBZg",
+            "checkin_times": {
+                "dates": [
+                    "2017-05-19",
+                    "2017-05-19",
+                    "2017-08-28",
+                    "2017-09-20",
+                    "2017-10-01",
+                    "2017-10-01",
+                    "2017-12-27"
+                ],
+                "times": [
+                    "14:30:16",
+                    "14:30:25",
+                    "15:49:37",
+                    "20:19:51",
+                    "16:31:05",
+                    "16:56:27",
+                    "23:33:20"
+                ]
+            }
+        },
+        {
+            "business_id": "-0Soj75v-XoRcf2ERr8Bmg",
+            "checkin_times": {
+                "dates": [
+                    "2019-06-05"
+                ],
+                "times": [
+                    "18:22:49"
+                ]
+            }
+        },
+        {
+            "business_id": "-0ZumLlFjMh4ZW1z2nXGug",
+            "checkin_times": {
+                "dates": [
+                    "2011-09-24",
+                    "2014-03-10",
+                    "2015-05-27",
+                    "2015-08-29",
+                    "2018-03-16"
+                ],
+                "times": [
+                    "21:37:32",
+                    "20:20:07",
+                    "00:40:24",
+                    "17:58:15",
+                    "15:03:26"
+                ]
+            }
+        },
+        {
+            "business_id": "-0aOudcaAyac0VJbMX-L1g",
+            "checkin_times": {
+                "dates": [
+                    "2015-03-16",
+                    "2015-12-21",
+                    "2016-10-28",
+                    "2016-10-28"
+                ],
+                "times": [
+                    "23:51:16",
+                    "04:48:01",
+                    "20:22:42",
+                    "20:23:00"
+                ]
+            }
+        },
+        {
+            "business_id": "-0b86isaXMY0v4g-V8GZ9Q",
+            "checkin_times": {
+                "dates": [
+                    "2013-10-22",
+                    "2014-11-21"
+                ],
+                "times": [
+                    "16:49:21",
+                    "17:39:24"
+                ]
+            }
+        },
+        {
+            "business_id": "-0d-BfFSU0bwLcnMaGRxYw",
+            "checkin_times": {
+                "dates": [
+                    "2014-08-07",
+                    "2014-09-16",
+                    "2014-10-12",
+                    "2015-07-21",
+                    "2015-07-21"
+                ],
+                "times": [
+                    "18:30:48",
+                    "20:41:45",
+                    "23:22:27",
+                    "20:43:56",
+                    "20:45:07"
+                ]
+            }
+        },
+        {
+            "business_id": "-0jz6c3C6i7RG7Ag22K-Pg",
+            "checkin_times": {
+                "dates": [
+                    "2015-05-02",
+                    "2015-05-06",
+                    "2015-09-26"
+                ],
+                "times": [
+                    "19:49:05",
+                    "03:52:18",
+                    "01:13:19"
+                ]
+            }
+        },
+        {
+            "business_id": "-0y3MZU2oYP8r1ruDP1bfQ",
+            "checkin_times": {
+                "dates": [
+                    "2015-04-11",
+                    "2015-11-21",
+                    "2016-05-06",
+                    "2017-08-09",
+                    "2017-10-21"
+                ],
+                "times": [
+                    "13:14:14",
+                    "16:05:56",
+                    "14:10:04",
+                    "15:15:10",
+                    "15:12:56"
+                ]
+            }
+        },
+        {
+            "business_id": "-1BPe8UjF2_l3nVk-DFUjA",
+            "checkin_times": {
+                "dates": [
+                    "2015-12-03",
+                    "2016-03-17",
+                    "2016-11-02"
+                ],
+                "times": [
+                    "18:44:00",
+                    "18:19:21",
+                    "15:58:38"
+                ]
+            }
+        },
+        {
+            "business_id": "-1E2CQu_38mkghvmZgCCRw",
+            "checkin_times": {
+                "dates": [
+                    "2019-04-04"
+                ],
+                "times": [
+                    "22:02:37"
+                ]
+            }
+        },
+        {
+            "business_id": "-1wzk43IZ5D9Ysu6kzb5xA",
+            "checkin_times": {
+                "dates": [
+                    "2019-02-27"
+                ],
+                "times": [
+                    "14:03:08"
+                ]
+            }
+        },
+        {
+            "business_id": "-23R9P2eG7VTc6DVLjFKzA",
+            "checkin_times": {
+                "dates": [
+                    "2011-12-21",
+                    "2012-04-15",
+                    "2012-04-15",
+                    "2013-06-30",
+                    "2013-10-04",
+                    "2014-07-16"
+                ],
+                "times": [
+                    "19:02:51",
+                    "04:21:39",
+                    "14:23:56",
+                    "22:39:51",
+                    "20:34:13",
+                    "02:28:40"
+                ]
+            }
+        },
+        {
+            "business_id": "-26MGfikhJiTfCI-GqmzhQ",
+            "checkin_times": {
+                "dates": [
+                    "2018-06-13"
+                ],
+                "times": [
+                    "20:16:07"
+                ]
+            }
+        },
+        {
+            "business_id": "-2bLuJsMZ0WhI9daurVQNQ",
+            "checkin_times": {
+                "dates": [
+                    "2015-05-29",
+                    "2015-06-01"
+                ],
+                "times": [
+                    "16:46:17",
+                    "15:03:53"
+                ]
+            }
+        },
+        {
+            "business_id": "-2hDBMaza_ldqnZdiU06LQ",
+            "checkin_times": {
+                "dates": [
+                    "2011-10-08",
+                    "2014-08-18",
+                    "2016-01-07",
+                    "2016-10-21",
+                    "2016-12-01",
+                    "2016-12-29",
+                    "2018-07-22",
+                    "2018-09-07",
+                    "2019-03-08"
+                ],
+                "times": [
+                    "12:02:23",
+                    "02:11:11",
+                    "05:27:51",
+                    "20:15:55",
+                    "03:57:10",
+                    "01:54:42",
+                    "19:55:31",
+                    "01:42:54",
+                    "03:41:06"
+                ]
+            }
+        }
+    ]
+);
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/select-unnest-queries/closed/use-case-2/use-case-2.3.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/select-unnest-queries/closed/use-case-2/use-case-2.3.ddl.sqlpp
new file mode 100644
index 0000000..0ab72fe
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/select-unnest-queries/closed/use-case-2/use-case-2.3.ddl.sqlpp
@@ -0,0 +1,22 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+USE TestYelp;
+
+CREATE INDEX IdxYelpCheckinDates ON YelpCheckin (UNNEST checkin_times.dates);
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/select-unnest-queries/closed/use-case-2/use-case-2.4.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/select-unnest-queries/closed/use-case-2/use-case-2.4.query.sqlpp
new file mode 100644
index 0000000..0f23cd4
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/select-unnest-queries/closed/use-case-2/use-case-2.4.query.sqlpp
@@ -0,0 +1,25 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+SET `compiler.arrayindex` "true";
+
+USE TestYelp;
+
+SELECT C.business_id
+FROM YelpCheckin C, C.checkin_times.dates D
+WHERE "2016-04-26" = D;
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/select-unnest-queries/closed/use-case-2/use-case-2.5.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/select-unnest-queries/closed/use-case-2/use-case-2.5.query.sqlpp
new file mode 100644
index 0000000..9887068
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/select-unnest-queries/closed/use-case-2/use-case-2.5.query.sqlpp
@@ -0,0 +1,25 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+SET `compiler.arrayindex` "true";
+
+USE TestYelp;
+
+SELECT COUNT(*)
+FROM YelpCheckin C, C.checkin_times.dates D
+WHERE D > "2016" AND D < "2017";
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/select-unnest-queries/closed/use-case-3/use-case-3.1.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/select-unnest-queries/closed/use-case-3/use-case-3.1.ddl.sqlpp
new file mode 100644
index 0000000..d20997e
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/select-unnest-queries/closed/use-case-3/use-case-3.1.ddl.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.
+ */
+
+/*
+ * Description: Verify that we are able to correctly perform the following types of queries using a array->record->atomic index.
+ *     1) UNNEST query, performing a point search.
+ *     2) UNNEST query, performing a range search.
+ *     3) UNNEST query, performing a point search with an additional predicate after the UNNEST.
+ */
+
+DROP DATAVERSE TestYelp IF EXISTS;
+CREATE DATAVERSE TestYelp;
+USE TestYelp;
+
+CREATE TYPE CheckinType AS {
+    checkin_id: uuid,
+    business_id: string,
+    checkin_times: [{
+        date: string,
+        time: string
+    }]
+};
+
+CREATE DATASET YelpCheckin(CheckinType) PRIMARY KEY checkin_id AUTOGENERATED;
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/select-unnest-queries/closed/use-case-3/use-case-3.2.update.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/select-unnest-queries/closed/use-case-3/use-case-3.2.update.sqlpp
new file mode 100644
index 0000000..02ddf21
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/select-unnest-queries/closed/use-case-3/use-case-3.2.update.sqlpp
@@ -0,0 +1,276 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+USE TestYelp;
+
+INSERT INTO YelpCheckin (
+    [
+        {
+            "business_id": "--1UhMGODdWsrMastO9DZw",
+            "checkin_times": [ 
+                {"date": "2016-04-26", "time": "19:49:16"},
+                {"date": "2016-08-30", "time": "18:36:57"},
+                {"date": "2016-10-15", "time": "02:45:18"},
+                {"date": "2016-11-18", "time": "01:54:50"},
+                {"date": "2017-04-20", "time": "18:39:06"},
+                {"date": "2017-05-03", "time": "17:58:02"},
+                {"date": "2019-03-19", "time": "22:04:48"}
+             ]
+        },
+        {
+            "business_id": "--EF5N7P70J_UYBTPypYlA",
+            "checkin_times": [ 
+                {"date": "2018-05-25", "time": "19:52:07"},
+                {"date": "2018-09-18", "time": "16:09:44"},
+                {"date": "2019-10-18", "time": "21:29:09"}
+             ]
+        },
+        {
+            "business_id": "--Ni3oJ4VOqfOEu7Sj2Vzg",
+            "checkin_times": [ 
+                {"date": "2019-06-07", "time": "17:54:58"}
+             ]
+        },
+        {
+            "business_id": "--Y1Adl1YUWfYIRSd8vkmA",
+            "checkin_times": [ 
+                {"date": "2011-05-03", "time": "20:54:05"},
+                {"date": "2011-08-23", "time": "20:49:45"},
+                {"date": "2014-12-04", "time": "06:13:01"},
+                {"date": "2016-11-16", "time": "19:25:55"}
+             ]
+        },
+        {
+            "business_id": "--YPwqIlRJrhHkJcjY3eiA",
+            "checkin_times": [ 
+                {"date": "2016-06-18", "time": "21:35:45"},
+                {"date": "2016-10-15", "time": "18:17:51"}
+             ]
+        },
+        {
+            "business_id": "--e8PjCNhEz32pprnPhCwQ",
+            "checkin_times": [ 
+                {"date": "2015-04-02", "time": "21:45:17"}
+             ]
+        },
+        {
+            "business_id": "--kinfHwmtdjz03g8B8z8Q",
+            "checkin_times": [ 
+                {"date": "2014-08-27", "time": "17:49:18"},
+                {"date": "2015-12-19", "time": "21:30:31"},
+                {"date": "2018-11-27", "time": "15:53:50"}
+             ]
+        },
+        {
+            "business_id": "--q6datkI-f0EoVheXNEeQ",
+            "checkin_times": [ 
+                {"date": "2014-01-28", "time": "20:56:04"},
+                {"date": "2014-11-16", "time": "16:11:58"},
+                {"date": "2015-11-15", "time": "19:21:53"},
+                {"date": "2015-11-15", "time": "19:33:39"}
+             ]
+        },
+        {
+            "business_id": "--qvQS4MigHPykD2GV0-zw",
+            "checkin_times": [ 
+                {"date": "2019-04-11", "time": "18:30:12"}
+             ]
+        },
+        {
+            "business_id": "--wIGbLEhlpl_UeAIyDmZQ",
+            "checkin_times": [ 
+                {"date": "2015-06-06", "time": "20:01:06"},
+                {"date": "2019-03-14", "time": "22:01:52"}
+             ]
+        },
+        {
+            "business_id": "-0FA-Qdi3SPYIoJz9UQw-A",
+            "checkin_times": [ 
+                {"date": "2018-09-29", "time": "18:55:17"},
+                {"date": "2018-10-20", "time": "16:48:05"},
+                {"date": "2018-10-20", "time": "22:20:24"}
+             ]
+        },
+        {
+            "business_id": "-0Hj1hb_XW6ybWq2M7QhGA",
+            "checkin_times": [ 
+                {"date": "2011-04-23", "time": "21:11:22"},
+                {"date": "2014-05-04", "time": "19:42:48"},
+                {"date": "2014-05-11", "time": "19:16:08"},
+                {"date": "2014-06-04", "time": "19:14:18"},
+                {"date": "2015-12-05", "time": "19:22:42"},
+                {"date": "2017-05-15", "time": "23:19:00"}
+             ]
+        },
+        {
+            "business_id": "-0KMvRFwDWdVBeTpT11iHw",
+            "checkin_times": [ 
+                {"date": "2012-07-13", "time": "21:43:57"},
+                {"date": "2016-12-24", "time": "02:27:31"},
+                {"date": "2017-08-31", "time": "00:35:26"}
+             ]
+        },
+        {
+            "business_id": "-0LPtgJC31FWMrMv317p0Q",
+            "checkin_times": [ 
+                {"date": "2013-04-13", "time": "12:35:33"},
+                {"date": "2013-08-19", "time": "23:35:49"},
+                {"date": "2013-10-04", "time": "19:14:56"}
+             ]
+        },
+        {
+            "business_id": "-0M3o2uWBnQZwd3hmfEwuw",
+            "checkin_times": [ 
+                {"date": "2016-09-10", "time": "19:26:19"},
+                {"date": "2018-09-08", "time": "14:15:37"},
+                {"date": "2019-09-13", "time": "22:47:25"}
+             ]
+        },
+        {
+            "business_id": "-0RRiWDtfnS16AKCtfvBZg",
+            "checkin_times": [ 
+                {"date": "2017-05-19", "time": "14:30:16"},
+                {"date": "2017-05-19", "time": "14:30:25"},
+                {"date": "2017-08-28", "time": "15:49:37"},
+                {"date": "2017-09-20", "time": "20:19:51"},
+                {"date": "2017-10-01", "time": "16:31:05"},
+                {"date": "2017-10-01", "time": "16:56:27"},
+                {"date": "2017-12-27", "time": "23:33:20"}
+             ]
+        },
+        {
+            "business_id": "-0Soj75v-XoRcf2ERr8Bmg",
+            "checkin_times": [ 
+                {"date": "2019-06-05", "time": "18:22:49"}
+             ]
+        },
+        {
+            "business_id": "-0ZumLlFjMh4ZW1z2nXGug",
+            "checkin_times": [ 
+                {"date": "2011-09-24", "time": "21:37:32"},
+                {"date": "2014-03-10", "time": "20:20:07"},
+                {"date": "2015-05-27", "time": "00:40:24"},
+                {"date": "2015-08-29", "time": "17:58:15"},
+                {"date": "2018-03-16", "time": "15:03:26"}
+             ]
+        },
+        {
+            "business_id": "-0aOudcaAyac0VJbMX-L1g",
+            "checkin_times": [ 
+                {"date": "2015-03-16", "time": "23:51:16"},
+                {"date": "2015-12-21", "time": "04:48:01"},
+                {"date": "2016-10-28", "time": "20:22:42"},
+                {"date": "2016-10-28", "time": "20:23:00"}
+             ]
+        },
+        {
+            "business_id": "-0b86isaXMY0v4g-V8GZ9Q",
+            "checkin_times": [ 
+                {"date": "2013-10-22", "time": "16:49:21"},
+                {"date": "2014-11-21", "time": "17:39:24"}
+             ]
+        },
+        {
+            "business_id": "-0d-BfFSU0bwLcnMaGRxYw",
+            "checkin_times": [ 
+                {"date": "2014-08-07", "time": "18:30:48"},
+                {"date": "2014-09-16", "time": "20:41:45"},
+                {"date": "2014-10-12", "time": "23:22:27"},
+                {"date": "2015-07-21", "time": "20:43:56"},
+                {"date": "2015-07-21", "time": "20:45:07"}
+             ]
+        },
+        {
+            "business_id": "-0jz6c3C6i7RG7Ag22K-Pg",
+            "checkin_times": [ 
+                {"date": "2015-05-02", "time": "19:49:05"},
+                {"date": "2015-05-06", "time": "03:52:18"},
+                {"date": "2015-09-26", "time": "01:13:19"}
+             ]
+        },
+        {
+            "business_id": "-0y3MZU2oYP8r1ruDP1bfQ",
+            "checkin_times": [ 
+                {"date": "2015-04-11", "time": "13:14:14"},
+                {"date": "2015-11-21", "time": "16:05:56"},
+                {"date": "2016-05-06", "time": "14:10:04"},
+                {"date": "2017-08-09", "time": "15:15:10"},
+                {"date": "2017-10-21", "time": "15:12:56"}
+             ]
+        },
+        {
+            "business_id": "-1BPe8UjF2_l3nVk-DFUjA",
+            "checkin_times": [ 
+                {"date": "2015-12-03", "time": "18:44:00"},
+                {"date": "2016-03-17", "time": "18:19:21"},
+                {"date": "2016-11-02", "time": "15:58:38"}
+             ]
+        },
+        {
+            "business_id": "-1E2CQu_38mkghvmZgCCRw",
+            "checkin_times": [ 
+                {"date": "2019-04-04", "time": "22:02:37"}
+             ]
+        },
+        {
+            "business_id": "-1wzk43IZ5D9Ysu6kzb5xA",
+            "checkin_times": [ 
+                {"date": "2019-02-27", "time": "14:03:08"}
+             ]
+        },
+        {
+            "business_id": "-23R9P2eG7VTc6DVLjFKzA",
+            "checkin_times": [ 
+                {"date": "2011-12-21", "time": "19:02:51"},
+                {"date": "2012-04-15", "time": "04:21:39"},
+                {"date": "2012-04-15", "time": "14:23:56"},
+                {"date": "2013-06-30", "time": "22:39:51"},
+                {"date": "2013-10-04", "time": "20:34:13"},
+                {"date": "2014-07-16", "time": "02:28:40"}
+             ]
+        },
+        {
+            "business_id": "-26MGfikhJiTfCI-GqmzhQ",
+            "checkin_times": [ 
+                {"date": "2018-06-13", "time": "20:16:07"}
+             ]
+        },
+        {
+            "business_id": "-2bLuJsMZ0WhI9daurVQNQ",
+            "checkin_times": [ 
+                {"date": "2015-05-29", "time": "16:46:17"},
+                {"date": "2015-06-01", "time": "15:03:53"}
+             ]
+        },
+        {
+            "business_id": "-2hDBMaza_ldqnZdiU06LQ",
+            "checkin_times": [ 
+                {"date": "2011-10-08", "time": "12:02:23"},
+                {"date": "2014-08-18", "time": "02:11:11"},
+                {"date": "2016-01-07", "time": "05:27:51"},
+                {"date": "2016-10-21", "time": "20:15:55"},
+                {"date": "2016-12-01", "time": "03:57:10"},
+                {"date": "2016-12-29", "time": "01:54:42"},
+                {"date": "2018-07-22", "time": "19:55:31"},
+                {"date": "2018-09-07", "time": "01:42:54"},
+                {"date": "2019-03-08", "time": "03:41:06"}
+             ]
+        }
+    ]
+);
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/select-unnest-queries/closed/use-case-3/use-case-3.3.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/select-unnest-queries/closed/use-case-3/use-case-3.3.ddl.sqlpp
new file mode 100644
index 0000000..ee911b4
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/select-unnest-queries/closed/use-case-3/use-case-3.3.ddl.sqlpp
@@ -0,0 +1,22 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+USE TestYelp;
+
+CREATE INDEX IdxYelpCheckinDates ON YelpCheckin (UNNEST checkin_times SELECT date);
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/select-unnest-queries/closed/use-case-3/use-case-3.4.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/select-unnest-queries/closed/use-case-3/use-case-3.4.query.sqlpp
new file mode 100644
index 0000000..912cfcb
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/select-unnest-queries/closed/use-case-3/use-case-3.4.query.sqlpp
@@ -0,0 +1,25 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+SET `compiler.arrayindex` "true";
+
+USE TestYelp;
+
+SELECT C.business_id
+FROM YelpCheckin C, C.checkin_times D
+WHERE "2016-04-26" = D.date;
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/select-unnest-queries/closed/use-case-3/use-case-3.5.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/select-unnest-queries/closed/use-case-3/use-case-3.5.query.sqlpp
new file mode 100644
index 0000000..244ae62
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/select-unnest-queries/closed/use-case-3/use-case-3.5.query.sqlpp
@@ -0,0 +1,25 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+SET `compiler.arrayindex` "true";
+
+USE TestYelp;
+
+SELECT COUNT(*)
+FROM YelpCheckin C, C.checkin_times D
+WHERE D.date > "2016" AND D.date < "2017";
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/select-unnest-queries/closed/use-case-3/use-case-3.6.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/select-unnest-queries/closed/use-case-3/use-case-3.6.query.sqlpp
new file mode 100644
index 0000000..4ab2742
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/select-unnest-queries/closed/use-case-3/use-case-3.6.query.sqlpp
@@ -0,0 +1,26 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+SET `compiler.arrayindex` "true";
+
+USE TestYelp;
+
+SELECT C.business_id
+FROM YelpCheckin C, C.checkin_times D
+WHERE D.date = "2016-04-26" AND 
+      D.time = "19:49:16";
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/select-unnest-queries/closed/use-case-4/use-case-4.1.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/select-unnest-queries/closed/use-case-4/use-case-4.1.ddl.sqlpp
new file mode 100644
index 0000000..809e207
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/select-unnest-queries/closed/use-case-4/use-case-4.1.ddl.sqlpp
@@ -0,0 +1,39 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+/*
+ * Description: Verify that we are able to correctly perform the following types of queries using a array->record->array->atomic index.
+ *     1) UNNEST query, performing a point search.
+ *     2) UNNEST query, performing a range search.
+ */
+
+DROP DATAVERSE TestYelp IF EXISTS;
+CREATE DATAVERSE TestYelp;
+USE TestYelp;
+
+CREATE TYPE CheckinType AS {
+    checkin_id: uuid,
+    business_id: string,
+    checkin_times: [{
+        dates: [string],
+        times: [string]
+    }]
+};
+
+CREATE DATASET YelpCheckin(CheckinType) PRIMARY KEY checkin_id AUTOGENERATED;
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/select-unnest-queries/closed/use-case-4/use-case-4.2.update.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/select-unnest-queries/closed/use-case-4/use-case-4.2.update.sqlpp
new file mode 100644
index 0000000..7fcf9e4
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/select-unnest-queries/closed/use-case-4/use-case-4.2.update.sqlpp
@@ -0,0 +1,425 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+USE TestYelp;
+
+INSERT INTO YelpCheckin (
+    [
+        {
+            "business_id": "--1UhMGODdWsrMastO9DZw",
+            "checkin_times": [ 
+                {
+                    "dates": ["2016-04-26", "2016-08-30", "2016-10-15", "2016-11-18"],
+                    "times": ["19:49:16", "18:36:57", "02:45:18", "01:54:50"]
+                },
+                {
+                    "dates": ["2017-04-20", "2017-05-03"],
+                    "times": ["18:39:06", "17:58:02"]
+                },
+                {
+                    "dates": ["2019-03-19"],
+                    "times": ["22:04:48"]
+                }
+             ]
+        },
+        {
+            "business_id": "--EF5N7P70J_UYBTPypYlA",
+            "checkin_times": [ 
+                {
+                    "dates": ["2018-05-25", "2018-09-18"],
+                    "times": ["19:52:07", "16:09:44"]
+                },
+                {
+                    "dates": ["2019-10-18"],
+                    "times": ["21:29:09"]
+                }
+             ]
+        },
+        {
+            "business_id": "--Ni3oJ4VOqfOEu7Sj2Vzg",
+            "checkin_times": [ 
+                {
+                    "dates": ["2019-06-07"],
+                    "times": ["17:54:58"]
+                }
+             ]
+        },
+        {
+            "business_id": "--Y1Adl1YUWfYIRSd8vkmA",
+            "checkin_times": [ 
+                {
+                    "dates": ["2011-05-03", "2011-08-23"],
+                    "times": ["20:54:05", "20:49:45"]
+                },
+                {
+                    "dates": ["2014-12-04"],
+                    "times": ["06:13:01"]
+                },
+                {
+                    "dates": ["2016-11-16"],
+                    "times": ["19:25:55"]
+                }
+             ]
+        },
+        {
+            "business_id": "--YPwqIlRJrhHkJcjY3eiA",
+            "checkin_times": [ 
+               {
+                    "dates": ["2016-06-18", "2016-10-15"],
+                    "times": ["21:35:45", "18:17:51"]
+                }
+             ]
+        },
+        {
+            "business_id": "--e8PjCNhEz32pprnPhCwQ",
+            "checkin_times": [ 
+                {
+                    "dates": ["2015-04-02"],
+                    "times": ["21:45:17"]
+                }
+             ]
+        },
+        {
+            "business_id": "--kinfHwmtdjz03g8B8z8Q",
+            "checkin_times": [ 
+                {
+                    "dates": ["2014-08-27"],
+                    "times": ["17:49:18"]
+                },
+                {
+                    "dates": ["2015-12-19"],
+                    "times": ["21:30:31"]
+                },
+                {
+                    "dates": ["2018-11-27"],
+                    "times": ["15:53:50"]
+                }
+             ]
+        },
+        {
+            "business_id": "--q6datkI-f0EoVheXNEeQ",
+            "checkin_times": [ 
+                {
+                    "dates": ["2014-01-28", "2014-11-16"],
+                    "times": ["20:56:04", "16:11:58"]
+                },
+                {
+                    "dates": ["2015-11-15", "2015-11-15"],
+                    "times": ["19:21:53", "19:33:39"]
+                }
+             ]
+        },
+        {
+            "business_id": "--qvQS4MigHPykD2GV0-zw",
+            "checkin_times": [ 
+                {
+                    "dates": ["2019-04-11"],
+                    "times": ["18:30:12"]
+                }
+             ]
+        },
+        {
+            "business_id": "--wIGbLEhlpl_UeAIyDmZQ",
+            "checkin_times": [ 
+                {
+                    "dates": ["2015-06-06"],
+                    "times": ["20:01:06"]
+                },
+                {
+                    "dates": ["2019-03-14"],
+                    "times": ["22:01:52"]
+                }
+             ]
+        },
+        {
+            "business_id": "-0FA-Qdi3SPYIoJz9UQw-A",
+            "checkin_times": [ 
+                {
+                    "dates": ["2018-09-29", "2018-10-20", "2018-10-20"],
+                    "times": ["18:55:17", "16:48:05", "22:20:24"]
+                }
+             ]
+        },
+        {
+            "business_id": "-0Hj1hb_XW6ybWq2M7QhGA",
+            "checkin_times": [ 
+                {
+                    "dates": ["2011-04-23"],
+                    "times": ["21:11:22"]
+                },
+                {
+                    "dates": ["2014-05-04", "2014-05-11", "2014-06-04"],
+                    "times": ["19:42:48", "19:16:08", "19:14:18"]
+                },
+                {
+                    "dates": ["2015-12-05"],
+                    "times": ["19:22:42"]
+                },
+                {
+                    "dates": ["2017-05-15"],
+                    "times": ["23:19:00"]
+                }
+             ]
+        },
+        {
+            "business_id": "-0KMvRFwDWdVBeTpT11iHw",
+            "checkin_times": [ 
+                {
+                    "dates": ["2012-07-13"],
+                    "times": ["21:43:57"]
+                },
+                {
+                    "dates": ["2016-12-24"],
+                    "times": ["02:27:31"]
+                },
+                {
+                    "dates": ["2017-08-31"],
+                    "times": ["00:35:26"]
+                }
+             ]
+        },
+        {
+            "business_id": "-0LPtgJC31FWMrMv317p0Q",
+            "checkin_times": [ 
+                {
+                    "dates": ["2013-04-13", "2013-08-19", "2013-10-04"],
+                    "times": ["12:35:33", "23:35:49", "19:14:56"]
+                }
+
+             ]
+        },
+        {
+            "business_id": "-0M3o2uWBnQZwd3hmfEwuw",
+            "checkin_times": [ 
+                {
+                    "dates": ["2016-09-10"],
+                    "times": ["19:26:19"]
+                },
+                {
+                    "dates": ["2018-09-08"],
+                    "times": ["14:15:37"]
+                },
+                {
+                    "dates": ["2019-09-13"],
+                    "times": ["22:47:25"]
+                }
+             ]
+        },
+        {
+            "business_id": "-0RRiWDtfnS16AKCtfvBZg",
+            "checkin_times": [ 
+                 {
+                    "dates": ["2017-05-19", "2017-05-19", "2017-08-28", "2017-09-20", "2017-10-01", "2017-10-01", "2017-12-27"],
+                    "times": ["14:30:16", "14:30:25", "15:49:37", "20:19:51", "16:31:05", "16:56:27", "23:33:20"]
+                }
+             ]
+        },
+        {
+            "business_id": "-0Soj75v-XoRcf2ERr8Bmg",
+            "checkin_times": [ 
+                {
+                    "dates": ["2019-06-05"],
+                    "times": ["18:22:49"]
+                }
+             ]
+        },
+        {
+            "business_id": "-0ZumLlFjMh4ZW1z2nXGug",
+            "checkin_times": [ 
+                {
+                    "dates": ["2011-09-24"],
+                    "times": ["21:37:32"]
+                },
+                {
+                    "dates": ["2014-03-10"],
+                    "times": ["20:20:07"]
+                },
+                {
+                    "dates": ["2015-05-27", "2015-08-29"],
+                    "times": ["00:40:24", "17:58:15"]
+                },
+                {
+                    "dates": ["2018-03-16"],
+                    "times": ["15:03:26"]
+                }
+             ]
+        },
+        {
+            "business_id": "-0aOudcaAyac0VJbMX-L1g",
+            "checkin_times": [ 
+                {
+                    "dates": ["2015-03-16", "2015-12-21"],
+                    "times": ["23:51:16", "04:48:01"]
+                },
+                {
+                    "dates": ["2016-10-28", "2016-10-28"],
+                    "times": ["20:22:42", "20:23:00"]
+                }
+             ]
+        },
+        {
+            "business_id": "-0b86isaXMY0v4g-V8GZ9Q",
+            "checkin_times": [ 
+                {
+                    "dates": ["2013-10-22"],
+                    "times": ["16:49:21"]
+                },
+                {
+                    "dates": ["2014-11-21"],
+                    "times": ["17:39:24"]
+                }
+             ]
+        },
+        {
+            "business_id": "-0d-BfFSU0bwLcnMaGRxYw",
+            "checkin_times": [ 
+                {
+                    "dates": ["2014-08-07", "2014-09-16", "2014-10-12"],
+                    "times": ["18:30:48", "20:41:45", "23:22:27"]
+                },
+                {
+                    "dates": ["2015-07-21", "2015-07-21"],
+                    "times": ["20:43:56", "20:45:07"]
+                }
+             ]
+        },
+        {
+            "business_id": "-0jz6c3C6i7RG7Ag22K-Pg",
+            "checkin_times": [ 
+                {
+                    "dates": ["2015-05-02", "2015-05-06", "2015-09-26"],
+                    "times": ["19:49:05", "03:52:18", "01:13:19"]
+                }
+
+             ]
+        },
+        {
+            "business_id": "-0y3MZU2oYP8r1ruDP1bfQ",
+            "checkin_times": [ 
+                {
+                    "dates": ["2015-04-11", "2015-11-21"],
+                    "times": ["13:14:14", "16:05:56"]
+                },
+                {
+                    "dates": ["2016-05-06"],
+                    "times": ["14:10:04"]
+                },
+                {
+                    "dates": ["2017-08-09", "2017-10-21"],
+                    "times": ["15:15:10", "15:12:56"]
+                }
+             ]
+        },
+        {
+            "business_id": "-1BPe8UjF2_l3nVk-DFUjA",
+            "checkin_times": [ 
+                {
+                    "dates": ["2015-12-03"],
+                    "times": ["18:44:00"]
+                },
+                {
+                    "dates": ["2016-03-17", "2016-11-02"],
+                    "times": ["18:19:21", "15:58:38"]
+                }
+             ]
+        },
+        {
+            "business_id": "-1E2CQu_38mkghvmZgCCRw",
+            "checkin_times": [ 
+                {
+                    "dates": ["2019-04-04"],
+                    "times": ["22:02:37"]
+                }
+             ]
+        },
+        {
+            "business_id": "-1wzk43IZ5D9Ysu6kzb5xA",
+            "checkin_times": [ 
+                {
+                    "dates": ["2019-02-27"],
+                    "times": ["14:03:08"]
+                }
+             ]
+        },
+        {
+            "business_id": "-23R9P2eG7VTc6DVLjFKzA",
+            "checkin_times": [ 
+                {
+                    "dates": ["2011-12-21"],
+                    "times": ["19:02:51"]
+                },
+                {
+                    "dates": ["2012-04-15", "2012-04-15"],
+                    "times": ["04:21:39", "14:23:56"]
+                },
+                {
+                    "dates": ["2013-06-30", "2013-10-04"],
+                    "times": ["22:39:51", "20:34:13"]
+                },
+                {
+                    "dates": ["2014-07-16"],
+                    "times": ["02:28:40"]
+                }
+             ]
+        },
+        {
+            "business_id": "-26MGfikhJiTfCI-GqmzhQ",
+            "checkin_times": [ 
+                {
+                    "dates": ["2018-06-13"],
+                    "times": ["20:16:07"]
+                }
+             ]
+        },
+        {
+            "business_id": "-2bLuJsMZ0WhI9daurVQNQ",
+            "checkin_times": [ 
+                {
+                    "dates": ["2015-05-29", "2015-06-01"],
+                    "times": ["16:46:17", "15:03:53"]
+                }
+             ]
+        },
+        {
+            "business_id": "-2hDBMaza_ldqnZdiU06LQ",
+            "checkin_times": [ 
+                {
+                    "dates": ["2011-10-08"],
+                    "times": ["12:02:23"]
+                },
+                {
+                    "dates": ["2014-08-18"],
+                    "times": ["02:11:11"]
+                },
+                {
+                    "dates": ["2016-01-07", "2016-10-21", "2016-12-01", "2016-12-29"],
+                    "times": ["05:27:51", "20:15:55", "03:57:10", "01:54:42"]
+                },
+                {
+                    "dates": ["2018-07-22", "2018-09-07"],
+                    "times": ["19:55:31", "01:42:54"]
+                },
+                {
+                    "dates": ["2019-03-08"],
+                    "times": ["03:41:06"]
+                }
+             ]
+        }
+    ]
+);
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/select-unnest-queries/closed/use-case-4/use-case-4.3.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/select-unnest-queries/closed/use-case-4/use-case-4.3.ddl.sqlpp
new file mode 100644
index 0000000..05a7c76
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/select-unnest-queries/closed/use-case-4/use-case-4.3.ddl.sqlpp
@@ -0,0 +1,22 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+USE TestYelp;
+
+CREATE INDEX IdxYelpCheckinDates ON YelpCheckin (UNNEST checkin_times UNNEST dates);
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/select-unnest-queries/closed/use-case-4/use-case-4.4.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/select-unnest-queries/closed/use-case-4/use-case-4.4.query.sqlpp
new file mode 100644
index 0000000..cd5b2c8
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/select-unnest-queries/closed/use-case-4/use-case-4.4.query.sqlpp
@@ -0,0 +1,26 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+SET `compiler.arrayindex` "true";
+
+USE TestYelp;
+
+SELECT C.business_id
+FROM YelpCheckin C, C.checkin_times CT, CT.dates D
+WHERE "2016-04-26" = D;
+
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/select-unnest-queries/closed/use-case-4/use-case-4.5.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/select-unnest-queries/closed/use-case-4/use-case-4.5.query.sqlpp
new file mode 100644
index 0000000..2f858dc
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/select-unnest-queries/closed/use-case-4/use-case-4.5.query.sqlpp
@@ -0,0 +1,25 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+SET `compiler.arrayindex` "true";
+
+USE TestYelp;
+
+SELECT COUNT(*)
+FROM YelpCheckin C, C.checkin_times CT, CT.dates D
+WHERE D > "2016" AND D < "2017";
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/select-unnest-queries/closed/with-3-level-record-path/with-3-level-record-path.1.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/select-unnest-queries/closed/with-3-level-record-path/with-3-level-record-path.1.ddl.sqlpp
new file mode 100644
index 0000000..5caafd9
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/select-unnest-queries/closed/with-3-level-record-path/with-3-level-record-path.1.ddl.sqlpp
@@ -0,0 +1,44 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+/*
+ * Description: Verify that we are able to correctly perform the following types of queries using a record->record->record->array->atomic index.
+ *     1) UNNEST query, performing a point search.
+ *     2) UNNEST query, performing a range search.
+ */
+
+
+DROP DATAVERSE TestYelp IF EXISTS;
+CREATE DATAVERSE TestYelp;
+USE TestYelp;
+
+CREATE TYPE CheckinType AS {
+    checkin_id: uuid,
+    business_id: string,
+    checkin_data: {
+        checkin_temporal: {
+            checkin_times: {
+                dates: [string],
+                times: [string]
+            }
+        }
+    }
+};
+
+CREATE DATASET YelpCheckin(CheckinType) PRIMARY KEY checkin_id AUTOGENERATED;
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/select-unnest-queries/closed/with-3-level-record-path/with-3-level-record-path.2.update.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/select-unnest-queries/closed/with-3-level-record-path/with-3-level-record-path.2.update.sqlpp
new file mode 100644
index 0000000..884c224
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/select-unnest-queries/closed/with-3-level-record-path/with-3-level-record-path.2.update.sqlpp
@@ -0,0 +1,617 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+USE TestYelp;
+
+INSERT INTO YelpCheckin (
+    [
+        {
+            "business_id": "--1UhMGODdWsrMastO9DZw",
+            "checkin_data": {
+                "checkin_temporal": {
+                    "checkin_times": {
+                        "dates": [
+                            "2016-04-26",
+                            "2016-08-30",
+                            "2016-10-15",
+                            "2016-11-18",
+                            "2017-04-20",
+                            "2017-05-03",
+                            "2019-03-19"
+                        ],
+                        "times": [
+                            "19:49:16",
+                            "18:36:57",
+                            "02:45:18",
+                            "01:54:50",
+                            "18:39:06",
+                            "17:58:02",
+                            "22:04:48"
+                        ]
+                    }
+                }
+            }
+        },
+        {
+            "business_id": "--EF5N7P70J_UYBTPypYlA",
+            "checkin_data": {
+                "checkin_temporal": {
+                    "checkin_times": {
+                        "dates": [
+                            "2018-05-25",
+                            "2018-09-18",
+                            "2019-10-18"
+                        ],
+                        "times": [
+                            "19:52:07",
+                            "16:09:44",
+                            "21:29:09"
+                        ]
+                    }
+                }
+            }
+        },
+        {
+            "business_id": "--Ni3oJ4VOqfOEu7Sj2Vzg",
+            "checkin_data": {
+                "checkin_temporal": {
+                    "checkin_times": {
+                        "dates": [
+                            "2019-06-07"
+                        ],
+                        "times": [
+                            "17:54:58"
+                        ]
+                    }
+                }
+            }
+        },
+        {
+            "business_id": "--Y1Adl1YUWfYIRSd8vkmA",
+            "checkin_data": {
+                "checkin_temporal": {
+                    "checkin_times": {
+                        "dates": [
+                            "2011-05-03",
+                            "2011-08-23",
+                            "2014-12-04",
+                            "2016-11-16"
+                        ],
+                        "times": [
+                            "20:54:05",
+                            "20:49:45",
+                            "06:13:01",
+                            "19:25:55"
+                        ]
+                    }
+                }
+            }
+        },
+        {
+            "business_id": "--YPwqIlRJrhHkJcjY3eiA",
+            "checkin_data": {
+                "checkin_temporal": {
+                    "checkin_times": {
+                        "dates": [
+                            "2016-06-18",
+                            "2016-10-15"
+                        ],
+                        "times": [
+                            "21:35:45",
+                            "18:17:51"
+                        ]
+                    }
+                }
+            }
+        },
+        {
+            "business_id": "--e8PjCNhEz32pprnPhCwQ",
+            "checkin_data": {
+                "checkin_temporal": {
+                    "checkin_times": {
+                        "dates": [
+                            "2015-04-02"
+                        ],
+                        "times": [
+                            "21:45:17"
+                        ]
+                    }
+                }
+            }
+        },
+        {
+            "business_id": "--kinfHwmtdjz03g8B8z8Q",
+            "checkin_data": {
+                "checkin_temporal": {
+                    "checkin_times": {
+                        "dates": [
+                            "2014-08-27",
+                            "2015-12-19",
+                            "2018-11-27"
+                        ],
+                        "times": [
+                            "17:49:18",
+                            "21:30:31",
+                            "15:53:50"
+                        ]
+                    }
+                }
+            }
+        },
+        {
+            "business_id": "--q6datkI-f0EoVheXNEeQ",
+            "checkin_data": {
+                "checkin_temporal": {
+                    "checkin_times": {
+                        "dates": [
+                            "2014-01-28",
+                            "2014-11-16",
+                            "2015-11-15",
+                            "2015-11-15"
+                        ],
+                        "times": [
+                            "20:56:04",
+                            "16:11:58",
+                            "19:21:53",
+                            "19:33:39"
+                        ]
+                    }
+                }
+            }
+        },
+        {
+            "business_id": "--qvQS4MigHPykD2GV0-zw",
+            "checkin_data": {
+                "checkin_temporal": {
+                    "checkin_times": {
+                        "dates": [
+                            "2019-04-11"
+                        ],
+                        "times": [
+                            "18:30:12"
+                        ]
+                    }
+                }
+            }
+        },
+        {
+            "business_id": "--wIGbLEhlpl_UeAIyDmZQ",
+            "checkin_data": {
+                "checkin_temporal": {
+                    "checkin_times": {
+                        "dates": [
+                            "2015-06-06",
+                            "2019-03-14"
+                        ],
+                        "times": [
+                            "20:01:06",
+                            "22:01:52"
+                        ]
+                    }
+                }
+            }
+        },
+        {
+            "business_id": "-0FA-Qdi3SPYIoJz9UQw-A",
+            "checkin_data": {
+                "checkin_temporal": {
+                    "checkin_times": {
+                        "dates": [
+                            "2018-09-29",
+                            "2018-10-20",
+                            "2018-10-20"
+                        ],
+                        "times": [
+                            "18:55:17",
+                            "16:48:05",
+                            "22:20:24"
+                        ]
+                    }
+                }
+            }
+        },
+        {
+            "business_id": "-0Hj1hb_XW6ybWq2M7QhGA",
+            "checkin_data": {
+                "checkin_temporal": {
+                    "checkin_times": {
+                        "dates": [
+                            "2011-04-23",
+                            "2014-05-04",
+                            "2014-05-11",
+                            "2014-06-04",
+                            "2015-12-05",
+                            "2017-05-15"
+                        ],
+                        "times": [
+                            "21:11:22",
+                            "19:42:48",
+                            "19:16:08",
+                            "19:14:18",
+                            "19:22:42",
+                            "23:19:00"
+                        ]
+                    }
+                }
+            }
+        },
+        {
+            "business_id": "-0KMvRFwDWdVBeTpT11iHw",
+            "checkin_data": {
+                "checkin_temporal": {
+                    "checkin_times": {
+                        "dates": [
+                            "2012-07-13",
+                            "2016-12-24",
+                            "2017-08-31"
+                        ],
+                        "times": [
+                            "21:43:57",
+                            "02:27:31",
+                            "00:35:26"
+                        ]
+                    }
+                }
+            }
+        },
+        {
+            "business_id": "-0LPtgJC31FWMrMv317p0Q",
+            "checkin_data": {
+                "checkin_temporal": {
+                    "checkin_times": {
+                        "dates": [
+                            "2013-04-13",
+                            "2013-08-19",
+                            "2013-10-04"
+                        ],
+                        "times": [
+                            "12:35:33",
+                            "23:35:49",
+                            "19:14:56"
+                        ]
+                    }
+                }
+            }
+        },
+        {
+            "business_id": "-0M3o2uWBnQZwd3hmfEwuw",
+            "checkin_data": {
+                "checkin_temporal": {
+                    "checkin_times": {
+                        "dates": [
+                            "2016-09-10",
+                            "2018-09-08",
+                            "2019-09-13"
+                        ],
+                        "times": [
+                            "19:26:19",
+                            "14:15:37",
+                            "22:47:25"
+                        ]
+                    }
+                }
+            }
+        },
+        {
+            "business_id": "-0RRiWDtfnS16AKCtfvBZg",
+            "checkin_data": {
+                "checkin_temporal": {
+                    "checkin_times": {
+                        "dates": [
+                            "2017-05-19",
+                            "2017-05-19",
+                            "2017-08-28",
+                            "2017-09-20",
+                            "2017-10-01",
+                            "2017-10-01",
+                            "2017-12-27"
+                        ],
+                        "times": [
+                            "14:30:16",
+                            "14:30:25",
+                            "15:49:37",
+                            "20:19:51",
+                            "16:31:05",
+                            "16:56:27",
+                            "23:33:20"
+                        ]
+                    }
+                }
+            }
+        },
+        {
+            "business_id": "-0Soj75v-XoRcf2ERr8Bmg",
+            "checkin_data": {
+                "checkin_temporal": {
+                    "checkin_times": {
+                        "dates": [
+                            "2019-06-05"
+                        ],
+                        "times": [
+                            "18:22:49"
+                        ]
+                    }
+                }
+            }
+        },
+        {
+            "business_id": "-0ZumLlFjMh4ZW1z2nXGug",
+            "checkin_data": {
+                "checkin_temporal": {
+                    "checkin_times": {
+                        "dates": [
+                            "2011-09-24",
+                            "2014-03-10",
+                            "2015-05-27",
+                            "2015-08-29",
+                            "2018-03-16"
+                        ],
+                        "times": [
+                            "21:37:32",
+                            "20:20:07",
+                            "00:40:24",
+                            "17:58:15",
+                            "15:03:26"
+                        ]
+                    }
+                }
+            }
+        },
+        {
+            "business_id": "-0aOudcaAyac0VJbMX-L1g",
+            "checkin_data": {
+                "checkin_temporal": {
+                    "checkin_times": {
+                        "dates": [
+                            "2015-03-16",
+                            "2015-12-21",
+                            "2016-10-28",
+                            "2016-10-28"
+                        ],
+                        "times": [
+                            "23:51:16",
+                            "04:48:01",
+                            "20:22:42",
+                            "20:23:00"
+                        ]
+                    }
+                }
+            }
+        },
+        {
+            "business_id": "-0b86isaXMY0v4g-V8GZ9Q",
+            "checkin_data": {
+                "checkin_temporal": {
+                    "checkin_times": {
+                        "dates": [
+                            "2013-10-22",
+                            "2014-11-21"
+                        ],
+                        "times": [
+                            "16:49:21",
+                            "17:39:24"
+                        ]
+                    }
+                }
+            }
+        },
+        {
+            "business_id": "-0d-BfFSU0bwLcnMaGRxYw",
+            "checkin_data": {
+                "checkin_temporal": {
+                    "checkin_times": {
+                        "dates": [
+                            "2014-08-07",
+                            "2014-09-16",
+                            "2014-10-12",
+                            "2015-07-21",
+                            "2015-07-21"
+                        ],
+                        "times": [
+                            "18:30:48",
+                            "20:41:45",
+                            "23:22:27",
+                            "20:43:56",
+                            "20:45:07"
+                        ]
+                    }
+                }
+            }
+        },
+        {
+            "business_id": "-0jz6c3C6i7RG7Ag22K-Pg",
+            "checkin_data": {
+                "checkin_temporal": {
+                    "checkin_times": {
+                        "dates": [
+                            "2015-05-02",
+                            "2015-05-06",
+                            "2015-09-26"
+                        ],
+                        "times": [
+                            "19:49:05",
+                            "03:52:18",
+                            "01:13:19"
+                        ]
+                    }
+                }
+            }
+        },
+        {
+            "business_id": "-0y3MZU2oYP8r1ruDP1bfQ",
+            "checkin_data": {
+                "checkin_temporal": {
+                    "checkin_times": {
+                        "dates": [
+                            "2015-04-11",
+                            "2015-11-21",
+                            "2016-05-06",
+                            "2017-08-09",
+                            "2017-10-21"
+                        ],
+                        "times": [
+                            "13:14:14",
+                            "16:05:56",
+                            "14:10:04",
+                            "15:15:10",
+                            "15:12:56"
+                        ]
+                    }
+                }
+            }
+        },
+        {
+            "business_id": "-1BPe8UjF2_l3nVk-DFUjA",
+            "checkin_data": {
+                "checkin_temporal": {
+                    "checkin_times": {
+                        "dates": [
+                            "2015-12-03",
+                            "2016-03-17",
+                            "2016-11-02"
+                        ],
+                        "times": [
+                            "18:44:00",
+                            "18:19:21",
+                            "15:58:38"
+                        ]
+                    }
+                }
+            }
+        },
+        {
+            "business_id": "-1E2CQu_38mkghvmZgCCRw",
+            "checkin_data": {
+                "checkin_temporal": {
+                    "checkin_times": {
+                        "dates": [
+                            "2019-04-04"
+                        ],
+                        "times": [
+                            "22:02:37"
+                        ]
+                    }
+                }
+            }
+        },
+        {
+            "business_id": "-1wzk43IZ5D9Ysu6kzb5xA",
+            "checkin_data": {
+                "checkin_temporal": {
+                    "checkin_times": {
+                        "dates": [
+                            "2019-02-27"
+                        ],
+                        "times": [
+                            "14:03:08"
+                        ]
+                    }
+                }
+            }
+        },
+        {
+            "business_id": "-23R9P2eG7VTc6DVLjFKzA",
+            "checkin_data": {
+                "checkin_temporal": {
+                    "checkin_times": {
+                        "dates": [
+                            "2011-12-21",
+                            "2012-04-15",
+                            "2012-04-15",
+                            "2013-06-30",
+                            "2013-10-04",
+                            "2014-07-16"
+                        ],
+                        "times": [
+                            "19:02:51",
+                            "04:21:39",
+                            "14:23:56",
+                            "22:39:51",
+                            "20:34:13",
+                            "02:28:40"
+                        ]
+                    }
+                }
+            }
+        },
+        {
+            "business_id": "-26MGfikhJiTfCI-GqmzhQ",
+            "checkin_data": {
+                "checkin_temporal": {
+                    "checkin_times": {
+                        "dates": [
+                            "2018-06-13"
+                        ],
+                        "times": [
+                            "20:16:07"
+                        ]
+                    }
+                }
+            }
+        },
+        {
+            "business_id": "-2bLuJsMZ0WhI9daurVQNQ",
+            "checkin_data": {
+                "checkin_temporal": {
+                    "checkin_times": {
+                        "dates": [
+                            "2015-05-29",
+                            "2015-06-01"
+                        ],
+                        "times": [
+                            "16:46:17",
+                            "15:03:53"
+                        ]
+                    }
+                }
+            }
+        },
+        {
+            "business_id": "-2hDBMaza_ldqnZdiU06LQ",
+            "checkin_data": {
+                "checkin_temporal": {
+                    "checkin_times": {
+                        "dates": [
+                            "2011-10-08",
+                            "2014-08-18",
+                            "2016-01-07",
+                            "2016-10-21",
+                            "2016-12-01",
+                            "2016-12-29",
+                            "2018-07-22",
+                            "2018-09-07",
+                            "2019-03-08"
+                        ],
+                        "times": [
+                            "12:02:23",
+                            "02:11:11",
+                            "05:27:51",
+                            "20:15:55",
+                            "03:57:10",
+                            "01:54:42",
+                            "19:55:31",
+                            "01:42:54",
+                            "03:41:06"
+                        ]
+                    }
+                }
+            }
+        }
+    ]
+);
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/select-unnest-queries/closed/with-3-level-record-path/with-3-level-record-path.3.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/select-unnest-queries/closed/with-3-level-record-path/with-3-level-record-path.3.ddl.sqlpp
new file mode 100644
index 0000000..5ef37bc
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/select-unnest-queries/closed/with-3-level-record-path/with-3-level-record-path.3.ddl.sqlpp
@@ -0,0 +1,22 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+USE TestYelp;
+
+CREATE INDEX IdxYelpCheckinDates ON YelpCheckin (UNNEST checkin_data.checkin_temporal.checkin_times.dates);
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/select-unnest-queries/closed/with-3-level-record-path/with-3-level-record-path.4.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/select-unnest-queries/closed/with-3-level-record-path/with-3-level-record-path.4.query.sqlpp
new file mode 100644
index 0000000..cee1887
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/select-unnest-queries/closed/with-3-level-record-path/with-3-level-record-path.4.query.sqlpp
@@ -0,0 +1,25 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+SET `compiler.arrayindex` "true";
+
+USE TestYelp;
+
+SELECT C.business_id
+FROM YelpCheckin C, C.checkin_data.checkin_temporal.checkin_times.dates D
+WHERE "2016-04-26" = D;
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/select-unnest-queries/closed/with-3-level-record-path/with-3-level-record-path.5.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/select-unnest-queries/closed/with-3-level-record-path/with-3-level-record-path.5.query.sqlpp
new file mode 100644
index 0000000..847e73f
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/select-unnest-queries/closed/with-3-level-record-path/with-3-level-record-path.5.query.sqlpp
@@ -0,0 +1,25 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+SET `compiler.arrayindex` "true";
+
+USE TestYelp;
+
+SELECT COUNT(*)
+FROM YelpCheckin C, C.checkin_data.checkin_temporal.checkin_times.dates D
+WHERE D > "2016" AND D < "2017";
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/select-unnest-queries/closed/with-composite-pk/with-composite-pk.1.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/select-unnest-queries/closed/with-composite-pk/with-composite-pk.1.ddl.sqlpp
new file mode 100644
index 0000000..b375231
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/select-unnest-queries/closed/with-composite-pk/with-composite-pk.1.ddl.sqlpp
@@ -0,0 +1,36 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+/*
+ * Description: Verify that we are able to correctly perform the following types of queries using an array->atomic index with a composite PK.
+ *     1) UNNEST query, performing a point search.
+ *     2) UNNEST query, performing a range search.
+ */
+
+DROP DATAVERSE TestYelp IF EXISTS;
+CREATE DATAVERSE TestYelp;
+USE TestYelp;
+
+CREATE TYPE CheckinType AS {
+    checkin_id: smallint,
+    business_id: string,
+    dates: [string]
+};
+
+CREATE DATASET YelpCheckin(CheckinType) PRIMARY KEY checkin_id, business_id;
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/select-unnest-queries/closed/with-composite-pk/with-composite-pk.2.update.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/select-unnest-queries/closed/with-composite-pk/with-composite-pk.2.update.sqlpp
new file mode 100644
index 0000000..6498c14
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/select-unnest-queries/closed/with-composite-pk/with-composite-pk.2.update.sqlpp
@@ -0,0 +1,306 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+USE TestYelp;
+
+INSERT INTO YelpCheckin (
+    [
+        {
+            "checkin_id": 1,
+            "business_id": "--1UhMGODdWsrMastO9DZw",
+            "dates": [
+                "2016-04-26 19:49:16",
+                "2016-08-30 18:36:57",
+                "2016-10-15 02:45:18",
+                "2016-11-18 01:54:50",
+                "2017-04-20 18:39:06",
+                "2017-05-03 17:58:02",
+                "2019-03-19 22:04:48"
+            ]
+        },
+        {
+            "checkin_id": 2,
+            "business_id": "--EF5N7P70J_UYBTPypYlA",
+            "dates": [
+                "2018-05-25 19:52:07",
+                "2018-09-18 16:09:44",
+                "2019-10-18 21:29:09"
+            ]
+        },
+        {
+            "checkin_id": 3,
+            "business_id": "--Ni3oJ4VOqfOEu7Sj2Vzg",
+            "dates": [
+                "2019-06-07 17:54:58"
+            ]
+        },
+        {
+            "checkin_id": 4,
+            "business_id": "--Y1Adl1YUWfYIRSd8vkmA",
+            "dates": [
+                "2011-05-03 20:54:05",
+                "2011-08-23 20:49:45",
+                "2014-12-04 06:13:01",
+                "2016-11-16 19:25:55"
+            ]
+        },
+        {
+            "checkin_id": 5,
+            "business_id": "--YPwqIlRJrhHkJcjY3eiA",
+            "dates": [
+                "2016-06-18 21:35:45",
+                "2016-10-15 18:17:51"
+            ]
+        },
+        {
+            "checkin_id": 6,
+            "business_id": "--e8PjCNhEz32pprnPhCwQ",
+            "dates": [
+                "2015-04-02 21:45:17"
+            ]
+        },
+        {
+            "checkin_id": 7,
+            "business_id": "--kinfHwmtdjz03g8B8z8Q",
+            "dates": [
+                "2014-08-27 17:49:18",
+                "2015-12-19 21:30:31",
+                "2018-11-27 15:53:50"
+            ]
+        },
+        {
+            "checkin_id": 8,
+            "business_id": "--q6datkI-f0EoVheXNEeQ",
+            "dates": [
+                "2014-01-28 20:56:04",
+                "2014-11-16 16:11:58",
+                "2015-11-15 19:21:53",
+                "2015-11-15 19:33:39"
+            ]
+        },
+        {
+            "checkin_id": 9,
+            "business_id": "--qvQS4MigHPykD2GV0-zw",
+            "dates": [
+                "2019-04-11 18:30:12"
+            ]
+        },
+        {
+            "checkin_id": 10,
+            "business_id": "--wIGbLEhlpl_UeAIyDmZQ",
+            "dates": [
+                "2015-06-06 20:01:06",
+                "2019-03-14 22:01:52"
+            ]
+        },
+        {
+            "checkin_id": 11,
+            "business_id": "-0FA-Qdi3SPYIoJz9UQw-A",
+            "dates": [
+                "2018-09-29 18:55:17",
+                "2018-10-20 16:48:05",
+                "2018-10-20 22:20:24"
+            ]
+        },
+        {
+            "checkin_id": 12,
+            "business_id": "-0Hj1hb_XW6ybWq2M7QhGA",
+            "dates": [
+                "2011-04-23 21:11:22",
+                "2014-05-04 19:42:48",
+                "2014-05-11 19:16:08",
+                "2014-06-04 19:14:18",
+                "2015-12-05 19:22:42",
+                "2017-05-15 23:19:00"
+            ]
+        },
+        {
+            "checkin_id": 13,
+            "business_id": "-0KMvRFwDWdVBeTpT11iHw",
+            "dates": [
+                "2012-07-13 21:43:57",
+                "2016-12-24 02:27:31",
+                "2017-08-31 00:35:26"
+            ]
+        },
+        {
+            "checkin_id": 14,
+            "business_id": "-0LPtgJC31FWMrMv317p0Q",
+            "dates": [
+                "2013-04-13 12:35:33",
+                "2013-08-19 23:35:49",
+                "2013-10-04 19:14:56"
+            ]
+        },
+        {
+            "checkin_id": 15,
+            "business_id": "-0M3o2uWBnQZwd3hmfEwuw",
+            "dates": [
+                "2016-09-10 19:26:19",
+                "2018-09-08 14:15:37",
+                "2019-09-13 22:47:25"
+            ]
+        },
+        {
+            "checkin_id": 16,
+            "business_id": "-0RRiWDtfnS16AKCtfvBZg",
+            "dates": [
+                "2017-05-19 14:30:16",
+                "2017-05-19 14:30:25",
+                "2017-08-28 15:49:37",
+                "2017-09-20 20:19:51",
+                "2017-10-01 16:31:05",
+                "2017-10-01 16:56:27",
+                "2017-12-27 23:33:20"
+            ]
+        },
+        {
+            "checkin_id": 17,
+            "business_id": "-0Soj75v-XoRcf2ERr8Bmg",
+            "dates": [
+                "2019-06-05 18:22:49"
+            ]
+        },
+        {
+            "checkin_id": 18,
+            "business_id": "-0ZumLlFjMh4ZW1z2nXGug",
+            "dates": [
+                "2011-09-24 21:37:32",
+                "2014-03-10 20:20:07",
+                "2015-05-27 00:40:24",
+                "2015-08-29 17:58:15",
+                "2018-03-16 15:03:26"
+            ]
+        },
+        {
+            "checkin_id": 19,
+            "business_id": "-0aOudcaAyac0VJbMX-L1g",
+            "dates": [
+                "2015-03-16 23:51:16",
+                "2015-12-21 04:48:01",
+                "2016-10-28 20:22:42",
+                "2016-10-28 20:23:00"
+            ]
+        },
+        {
+            "checkin_id": 20,
+            "business_id": "-0b86isaXMY0v4g-V8GZ9Q",
+            "dates": [
+                "2013-10-22 16:49:21",
+                "2014-11-21 17:39:24"
+            ]
+        },
+        {
+            "checkin_id": 21,
+            "business_id": "-0d-BfFSU0bwLcnMaGRxYw",
+            "dates": [
+                "2014-08-07 18:30:48",
+                "2014-09-16 20:41:45",
+                "2014-10-12 23:22:27",
+                "2015-07-21 20:43:56",
+                "2015-07-21 20:45:07"
+            ]
+        },
+        {
+            "checkin_id": 22,
+            "business_id": "-0jz6c3C6i7RG7Ag22K-Pg",
+            "dates": [
+                "2015-05-02 19:49:05",
+                "2015-05-06 03:52:18",
+                "2015-09-26 01:13:19"
+            ]
+        },
+        {
+            "checkin_id": 23,
+            "business_id": "-0y3MZU2oYP8r1ruDP1bfQ",
+            "dates": [
+                "2015-04-11 13:14:14",
+                "2015-11-21 16:05:56",
+                "2016-05-06 14:10:04",
+                "2017-08-09 15:15:10",
+                "2017-10-21 15:12:56"
+            ]
+        },
+        {
+            "checkin_id": 24,
+            "business_id": "-1BPe8UjF2_l3nVk-DFUjA",
+            "dates": [
+                "2015-12-03 18:44:00",
+                "2016-03-17 18:19:21",
+                "2016-11-02 15:58:38"
+            ]
+        },
+        {
+            "checkin_id": 25,
+            "business_id": "-1E2CQu_38mkghvmZgCCRw",
+            "dates": [
+                "2019-04-04 22:02:37"
+            ]
+        },
+        {
+            "checkin_id": 26,
+            "business_id": "-1wzk43IZ5D9Ysu6kzb5xA",
+            "dates": [
+                "2019-02-27 14:03:08"
+            ]
+        },
+        {
+            "checkin_id": 27,
+            "business_id": "-23R9P2eG7VTc6DVLjFKzA",
+            "dates": [
+                "2011-12-21 19:02:51",
+                "2012-04-15 04:21:39",
+                "2012-04-15 14:23:56",
+                "2013-06-30 22:39:51",
+                "2013-10-04 20:34:13",
+                "2014-07-16 02:28:40"
+            ]
+        },
+        {
+            "checkin_id": 28,
+            "business_id": "-26MGfikhJiTfCI-GqmzhQ",
+            "dates": [
+                "2018-06-13 20:16:07"
+            ]
+        },
+        {
+            "checkin_id": 29,
+            "business_id": "-2bLuJsMZ0WhI9daurVQNQ",
+            "dates": [
+                "2015-05-29 16:46:17",
+                "2015-06-01 15:03:53"
+            ]
+        },
+        {
+            "checkin_id": 30,
+            "business_id": "-2hDBMaza_ldqnZdiU06LQ",
+            "dates": [
+                "2011-10-08 12:02:23",
+                "2014-08-18 02:11:11",
+                "2016-01-07 05:27:51",
+                "2016-10-21 20:15:55",
+                "2016-12-01 03:57:10",
+                "2016-12-29 01:54:42",
+                "2018-07-22 19:55:31",
+                "2018-09-07 01:42:54",
+                "2019-03-08 03:41:06"
+            ]
+        }
+    ]
+);
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/select-unnest-queries/closed/with-composite-pk/with-composite-pk.3.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/select-unnest-queries/closed/with-composite-pk/with-composite-pk.3.ddl.sqlpp
new file mode 100644
index 0000000..a0fc1bf
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/select-unnest-queries/closed/with-composite-pk/with-composite-pk.3.ddl.sqlpp
@@ -0,0 +1,22 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+USE TestYelp;
+
+CREATE INDEX IdxYelpCheckinDates ON YelpCheckin (UNNEST dates);
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/select-unnest-queries/closed/with-composite-pk/with-composite-pk.4.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/select-unnest-queries/closed/with-composite-pk/with-composite-pk.4.query.sqlpp
new file mode 100644
index 0000000..ec79bc6
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/select-unnest-queries/closed/with-composite-pk/with-composite-pk.4.query.sqlpp
@@ -0,0 +1,25 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+SET `compiler.arrayindex` "true";
+
+USE TestYelp;
+
+SELECT C.business_id
+FROM YelpCheckin C, C.dates D
+WHERE "2016-04-26 19:49:16" = D;
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/select-unnest-queries/closed/with-composite-pk/with-composite-pk.5.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/select-unnest-queries/closed/with-composite-pk/with-composite-pk.5.query.sqlpp
new file mode 100644
index 0000000..cba5055
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/select-unnest-queries/closed/with-composite-pk/with-composite-pk.5.query.sqlpp
@@ -0,0 +1,25 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+SET `compiler.arrayindex` "true";
+
+USE TestYelp;
+
+SELECT COUNT(*)
+FROM YelpCheckin C, C.dates D
+WHERE D > "2016" AND D < "2017";
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/select-unnest-queries/closed/with-composite-sk/with-composite-sk.1.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/select-unnest-queries/closed/with-composite-sk/with-composite-sk.1.ddl.sqlpp
new file mode 100644
index 0000000..fc115e3
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/select-unnest-queries/closed/with-composite-sk/with-composite-sk.1.ddl.sqlpp
@@ -0,0 +1,39 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+/*
+ * Description: Verify that we are able to correctly perform the following types of queries using a array->record->atomic index on both array fields + an atomic field.
+ *     1) UNNEST query, performing a composite point search.
+ *     2) UNNEST query, performing a range search.
+ */
+
+DROP DATAVERSE TestYelp IF EXISTS;
+CREATE DATAVERSE TestYelp;
+USE TestYelp;
+
+CREATE TYPE CheckinType AS {
+    checkin_id: uuid,
+    business_id: string,
+    checkin_times: [{
+        date: string,
+        time: string
+    }]
+};
+
+CREATE DATASET YelpCheckin(CheckinType) PRIMARY KEY checkin_id AUTOGENERATED;
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/select-unnest-queries/closed/with-composite-sk/with-composite-sk.2.update.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/select-unnest-queries/closed/with-composite-sk/with-composite-sk.2.update.sqlpp
new file mode 100644
index 0000000..02ddf21
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/select-unnest-queries/closed/with-composite-sk/with-composite-sk.2.update.sqlpp
@@ -0,0 +1,276 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+USE TestYelp;
+
+INSERT INTO YelpCheckin (
+    [
+        {
+            "business_id": "--1UhMGODdWsrMastO9DZw",
+            "checkin_times": [ 
+                {"date": "2016-04-26", "time": "19:49:16"},
+                {"date": "2016-08-30", "time": "18:36:57"},
+                {"date": "2016-10-15", "time": "02:45:18"},
+                {"date": "2016-11-18", "time": "01:54:50"},
+                {"date": "2017-04-20", "time": "18:39:06"},
+                {"date": "2017-05-03", "time": "17:58:02"},
+                {"date": "2019-03-19", "time": "22:04:48"}
+             ]
+        },
+        {
+            "business_id": "--EF5N7P70J_UYBTPypYlA",
+            "checkin_times": [ 
+                {"date": "2018-05-25", "time": "19:52:07"},
+                {"date": "2018-09-18", "time": "16:09:44"},
+                {"date": "2019-10-18", "time": "21:29:09"}
+             ]
+        },
+        {
+            "business_id": "--Ni3oJ4VOqfOEu7Sj2Vzg",
+            "checkin_times": [ 
+                {"date": "2019-06-07", "time": "17:54:58"}
+             ]
+        },
+        {
+            "business_id": "--Y1Adl1YUWfYIRSd8vkmA",
+            "checkin_times": [ 
+                {"date": "2011-05-03", "time": "20:54:05"},
+                {"date": "2011-08-23", "time": "20:49:45"},
+                {"date": "2014-12-04", "time": "06:13:01"},
+                {"date": "2016-11-16", "time": "19:25:55"}
+             ]
+        },
+        {
+            "business_id": "--YPwqIlRJrhHkJcjY3eiA",
+            "checkin_times": [ 
+                {"date": "2016-06-18", "time": "21:35:45"},
+                {"date": "2016-10-15", "time": "18:17:51"}
+             ]
+        },
+        {
+            "business_id": "--e8PjCNhEz32pprnPhCwQ",
+            "checkin_times": [ 
+                {"date": "2015-04-02", "time": "21:45:17"}
+             ]
+        },
+        {
+            "business_id": "--kinfHwmtdjz03g8B8z8Q",
+            "checkin_times": [ 
+                {"date": "2014-08-27", "time": "17:49:18"},
+                {"date": "2015-12-19", "time": "21:30:31"},
+                {"date": "2018-11-27", "time": "15:53:50"}
+             ]
+        },
+        {
+            "business_id": "--q6datkI-f0EoVheXNEeQ",
+            "checkin_times": [ 
+                {"date": "2014-01-28", "time": "20:56:04"},
+                {"date": "2014-11-16", "time": "16:11:58"},
+                {"date": "2015-11-15", "time": "19:21:53"},
+                {"date": "2015-11-15", "time": "19:33:39"}
+             ]
+        },
+        {
+            "business_id": "--qvQS4MigHPykD2GV0-zw",
+            "checkin_times": [ 
+                {"date": "2019-04-11", "time": "18:30:12"}
+             ]
+        },
+        {
+            "business_id": "--wIGbLEhlpl_UeAIyDmZQ",
+            "checkin_times": [ 
+                {"date": "2015-06-06", "time": "20:01:06"},
+                {"date": "2019-03-14", "time": "22:01:52"}
+             ]
+        },
+        {
+            "business_id": "-0FA-Qdi3SPYIoJz9UQw-A",
+            "checkin_times": [ 
+                {"date": "2018-09-29", "time": "18:55:17"},
+                {"date": "2018-10-20", "time": "16:48:05"},
+                {"date": "2018-10-20", "time": "22:20:24"}
+             ]
+        },
+        {
+            "business_id": "-0Hj1hb_XW6ybWq2M7QhGA",
+            "checkin_times": [ 
+                {"date": "2011-04-23", "time": "21:11:22"},
+                {"date": "2014-05-04", "time": "19:42:48"},
+                {"date": "2014-05-11", "time": "19:16:08"},
+                {"date": "2014-06-04", "time": "19:14:18"},
+                {"date": "2015-12-05", "time": "19:22:42"},
+                {"date": "2017-05-15", "time": "23:19:00"}
+             ]
+        },
+        {
+            "business_id": "-0KMvRFwDWdVBeTpT11iHw",
+            "checkin_times": [ 
+                {"date": "2012-07-13", "time": "21:43:57"},
+                {"date": "2016-12-24", "time": "02:27:31"},
+                {"date": "2017-08-31", "time": "00:35:26"}
+             ]
+        },
+        {
+            "business_id": "-0LPtgJC31FWMrMv317p0Q",
+            "checkin_times": [ 
+                {"date": "2013-04-13", "time": "12:35:33"},
+                {"date": "2013-08-19", "time": "23:35:49"},
+                {"date": "2013-10-04", "time": "19:14:56"}
+             ]
+        },
+        {
+            "business_id": "-0M3o2uWBnQZwd3hmfEwuw",
+            "checkin_times": [ 
+                {"date": "2016-09-10", "time": "19:26:19"},
+                {"date": "2018-09-08", "time": "14:15:37"},
+                {"date": "2019-09-13", "time": "22:47:25"}
+             ]
+        },
+        {
+            "business_id": "-0RRiWDtfnS16AKCtfvBZg",
+            "checkin_times": [ 
+                {"date": "2017-05-19", "time": "14:30:16"},
+                {"date": "2017-05-19", "time": "14:30:25"},
+                {"date": "2017-08-28", "time": "15:49:37"},
+                {"date": "2017-09-20", "time": "20:19:51"},
+                {"date": "2017-10-01", "time": "16:31:05"},
+                {"date": "2017-10-01", "time": "16:56:27"},
+                {"date": "2017-12-27", "time": "23:33:20"}
+             ]
+        },
+        {
+            "business_id": "-0Soj75v-XoRcf2ERr8Bmg",
+            "checkin_times": [ 
+                {"date": "2019-06-05", "time": "18:22:49"}
+             ]
+        },
+        {
+            "business_id": "-0ZumLlFjMh4ZW1z2nXGug",
+            "checkin_times": [ 
+                {"date": "2011-09-24", "time": "21:37:32"},
+                {"date": "2014-03-10", "time": "20:20:07"},
+                {"date": "2015-05-27", "time": "00:40:24"},
+                {"date": "2015-08-29", "time": "17:58:15"},
+                {"date": "2018-03-16", "time": "15:03:26"}
+             ]
+        },
+        {
+            "business_id": "-0aOudcaAyac0VJbMX-L1g",
+            "checkin_times": [ 
+                {"date": "2015-03-16", "time": "23:51:16"},
+                {"date": "2015-12-21", "time": "04:48:01"},
+                {"date": "2016-10-28", "time": "20:22:42"},
+                {"date": "2016-10-28", "time": "20:23:00"}
+             ]
+        },
+        {
+            "business_id": "-0b86isaXMY0v4g-V8GZ9Q",
+            "checkin_times": [ 
+                {"date": "2013-10-22", "time": "16:49:21"},
+                {"date": "2014-11-21", "time": "17:39:24"}
+             ]
+        },
+        {
+            "business_id": "-0d-BfFSU0bwLcnMaGRxYw",
+            "checkin_times": [ 
+                {"date": "2014-08-07", "time": "18:30:48"},
+                {"date": "2014-09-16", "time": "20:41:45"},
+                {"date": "2014-10-12", "time": "23:22:27"},
+                {"date": "2015-07-21", "time": "20:43:56"},
+                {"date": "2015-07-21", "time": "20:45:07"}
+             ]
+        },
+        {
+            "business_id": "-0jz6c3C6i7RG7Ag22K-Pg",
+            "checkin_times": [ 
+                {"date": "2015-05-02", "time": "19:49:05"},
+                {"date": "2015-05-06", "time": "03:52:18"},
+                {"date": "2015-09-26", "time": "01:13:19"}
+             ]
+        },
+        {
+            "business_id": "-0y3MZU2oYP8r1ruDP1bfQ",
+            "checkin_times": [ 
+                {"date": "2015-04-11", "time": "13:14:14"},
+                {"date": "2015-11-21", "time": "16:05:56"},
+                {"date": "2016-05-06", "time": "14:10:04"},
+                {"date": "2017-08-09", "time": "15:15:10"},
+                {"date": "2017-10-21", "time": "15:12:56"}
+             ]
+        },
+        {
+            "business_id": "-1BPe8UjF2_l3nVk-DFUjA",
+            "checkin_times": [ 
+                {"date": "2015-12-03", "time": "18:44:00"},
+                {"date": "2016-03-17", "time": "18:19:21"},
+                {"date": "2016-11-02", "time": "15:58:38"}
+             ]
+        },
+        {
+            "business_id": "-1E2CQu_38mkghvmZgCCRw",
+            "checkin_times": [ 
+                {"date": "2019-04-04", "time": "22:02:37"}
+             ]
+        },
+        {
+            "business_id": "-1wzk43IZ5D9Ysu6kzb5xA",
+            "checkin_times": [ 
+                {"date": "2019-02-27", "time": "14:03:08"}
+             ]
+        },
+        {
+            "business_id": "-23R9P2eG7VTc6DVLjFKzA",
+            "checkin_times": [ 
+                {"date": "2011-12-21", "time": "19:02:51"},
+                {"date": "2012-04-15", "time": "04:21:39"},
+                {"date": "2012-04-15", "time": "14:23:56"},
+                {"date": "2013-06-30", "time": "22:39:51"},
+                {"date": "2013-10-04", "time": "20:34:13"},
+                {"date": "2014-07-16", "time": "02:28:40"}
+             ]
+        },
+        {
+            "business_id": "-26MGfikhJiTfCI-GqmzhQ",
+            "checkin_times": [ 
+                {"date": "2018-06-13", "time": "20:16:07"}
+             ]
+        },
+        {
+            "business_id": "-2bLuJsMZ0WhI9daurVQNQ",
+            "checkin_times": [ 
+                {"date": "2015-05-29", "time": "16:46:17"},
+                {"date": "2015-06-01", "time": "15:03:53"}
+             ]
+        },
+        {
+            "business_id": "-2hDBMaza_ldqnZdiU06LQ",
+            "checkin_times": [ 
+                {"date": "2011-10-08", "time": "12:02:23"},
+                {"date": "2014-08-18", "time": "02:11:11"},
+                {"date": "2016-01-07", "time": "05:27:51"},
+                {"date": "2016-10-21", "time": "20:15:55"},
+                {"date": "2016-12-01", "time": "03:57:10"},
+                {"date": "2016-12-29", "time": "01:54:42"},
+                {"date": "2018-07-22", "time": "19:55:31"},
+                {"date": "2018-09-07", "time": "01:42:54"},
+                {"date": "2019-03-08", "time": "03:41:06"}
+             ]
+        }
+    ]
+);
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/select-unnest-queries/closed/with-composite-sk/with-composite-sk.3.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/select-unnest-queries/closed/with-composite-sk/with-composite-sk.3.ddl.sqlpp
new file mode 100644
index 0000000..20de338
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/select-unnest-queries/closed/with-composite-sk/with-composite-sk.3.ddl.sqlpp
@@ -0,0 +1,22 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+USE TestYelp;
+
+CREATE INDEX IdxYelpCheckinDatesTimesBusiness ON YelpCheckin (UNNEST checkin_times SELECT date, time);
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/select-unnest-queries/closed/with-composite-sk/with-composite-sk.4.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/select-unnest-queries/closed/with-composite-sk/with-composite-sk.4.query.sqlpp
new file mode 100644
index 0000000..3023de1
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/select-unnest-queries/closed/with-composite-sk/with-composite-sk.4.query.sqlpp
@@ -0,0 +1,27 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+SET `compiler.arrayindex` "true";
+
+USE TestYelp;
+
+SELECT C.business_id
+FROM YelpCheckin C, C.checkin_times D
+WHERE "2016-04-26" = D.date AND 
+      "19:49:16" = D.time AND
+      "--1UhMGODdWsrMastO9DZw" = C.business_id;
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/select-unnest-queries/closed/with-composite-sk/with-composite-sk.5.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/select-unnest-queries/closed/with-composite-sk/with-composite-sk.5.query.sqlpp
new file mode 100644
index 0000000..244ae62
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/select-unnest-queries/closed/with-composite-sk/with-composite-sk.5.query.sqlpp
@@ -0,0 +1,25 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+SET `compiler.arrayindex` "true";
+
+USE TestYelp;
+
+SELECT COUNT(*)
+FROM YelpCheckin C, C.checkin_times D
+WHERE D.date > "2016" AND D.date < "2017";
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/select-unnest-queries/closed/with-filter-fields/with-filter-fields.1.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/select-unnest-queries/closed/with-filter-fields/with-filter-fields.1.ddl.sqlpp
new file mode 100644
index 0000000..2d7d649
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/select-unnest-queries/closed/with-filter-fields/with-filter-fields.1.ddl.sqlpp
@@ -0,0 +1,36 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+/*
+ * Description: Verify that we are able to correctly perform the following types of queries using an array->atomic index with a dataset filter field.
+ *     1) UNNEST query, performing a point search with the inclusion of the filter field.
+ *     2) UNNEST query, performing a range search without the inclusion of the filter field.
+ */
+
+DROP DATAVERSE TestYelp IF EXISTS;
+CREATE DATAVERSE TestYelp;
+USE TestYelp;
+
+CREATE TYPE CheckinType AS {
+    checkin_id: uuid,
+    business_id: string,
+    dates: [string]
+};
+
+CREATE DATASET YelpCheckin(CheckinType) PRIMARY KEY checkin_id AUTOGENERATED WITH FILTER ON business_id;
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/select-unnest-queries/closed/with-filter-fields/with-filter-fields.2.update.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/select-unnest-queries/closed/with-filter-fields/with-filter-fields.2.update.sqlpp
new file mode 100644
index 0000000..b75bf1b
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/select-unnest-queries/closed/with-filter-fields/with-filter-fields.2.update.sqlpp
@@ -0,0 +1,276 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+USE TestYelp;
+
+INSERT INTO YelpCheckin (
+    [
+        {
+            "business_id": "--1UhMGODdWsrMastO9DZw",
+            "dates": [
+                "2016-04-26 19:49:16",
+                "2016-08-30 18:36:57",
+                "2016-10-15 02:45:18",
+                "2016-11-18 01:54:50",
+                "2017-04-20 18:39:06",
+                "2017-05-03 17:58:02",
+                "2019-03-19 22:04:48"
+            ]
+        },
+        {
+            "business_id": "--EF5N7P70J_UYBTPypYlA",
+            "dates": [
+                "2018-05-25 19:52:07",
+                "2018-09-18 16:09:44",
+                "2019-10-18 21:29:09"
+            ]
+        },
+        {
+            "business_id": "--Ni3oJ4VOqfOEu7Sj2Vzg",
+            "dates": [
+                "2019-06-07 17:54:58"
+            ]
+        },
+        {
+            "business_id": "--Y1Adl1YUWfYIRSd8vkmA",
+            "dates": [
+                "2011-05-03 20:54:05",
+                "2011-08-23 20:49:45",
+                "2014-12-04 06:13:01",
+                "2016-11-16 19:25:55"
+            ]
+        },
+        {
+            "business_id": "--YPwqIlRJrhHkJcjY3eiA",
+            "dates": [
+                "2016-06-18 21:35:45",
+                "2016-10-15 18:17:51"
+            ]
+        },
+        {
+            "business_id": "--e8PjCNhEz32pprnPhCwQ",
+            "dates": [
+                "2015-04-02 21:45:17"
+            ]
+        },
+        {
+            "business_id": "--kinfHwmtdjz03g8B8z8Q",
+            "dates": [
+                "2014-08-27 17:49:18",
+                "2015-12-19 21:30:31",
+                "2018-11-27 15:53:50"
+            ]
+        },
+        {
+            "business_id": "--q6datkI-f0EoVheXNEeQ",
+            "dates": [
+                "2014-01-28 20:56:04",
+                "2014-11-16 16:11:58",
+                "2015-11-15 19:21:53",
+                "2015-11-15 19:33:39"
+            ]
+        },
+        {
+            "business_id": "--qvQS4MigHPykD2GV0-zw",
+            "dates": [
+                "2019-04-11 18:30:12"
+            ]
+        },
+        {
+            "business_id": "--wIGbLEhlpl_UeAIyDmZQ",
+            "dates": [
+                "2015-06-06 20:01:06",
+                "2019-03-14 22:01:52"
+            ]
+        },
+        {
+            "business_id": "-0FA-Qdi3SPYIoJz9UQw-A",
+            "dates": [
+                "2018-09-29 18:55:17",
+                "2018-10-20 16:48:05",
+                "2018-10-20 22:20:24"
+            ]
+        },
+        {
+            "business_id": "-0Hj1hb_XW6ybWq2M7QhGA",
+            "dates": [
+                "2011-04-23 21:11:22",
+                "2014-05-04 19:42:48",
+                "2014-05-11 19:16:08",
+                "2014-06-04 19:14:18",
+                "2015-12-05 19:22:42",
+                "2017-05-15 23:19:00"
+            ]
+        },
+        {
+            "business_id": "-0KMvRFwDWdVBeTpT11iHw",
+            "dates": [
+                "2012-07-13 21:43:57",
+                "2016-12-24 02:27:31",
+                "2017-08-31 00:35:26"
+            ]
+        },
+        {
+            "business_id": "-0LPtgJC31FWMrMv317p0Q",
+            "dates": [
+                "2013-04-13 12:35:33",
+                "2013-08-19 23:35:49",
+                "2013-10-04 19:14:56"
+            ]
+        },
+        {
+            "business_id": "-0M3o2uWBnQZwd3hmfEwuw",
+            "dates": [
+                "2016-09-10 19:26:19",
+                "2018-09-08 14:15:37",
+                "2019-09-13 22:47:25"
+            ]
+        },
+        {
+            "business_id": "-0RRiWDtfnS16AKCtfvBZg",
+            "dates": [
+                "2017-05-19 14:30:16",
+                "2017-05-19 14:30:25",
+                "2017-08-28 15:49:37",
+                "2017-09-20 20:19:51",
+                "2017-10-01 16:31:05",
+                "2017-10-01 16:56:27",
+                "2017-12-27 23:33:20"
+            ]
+        },
+        {
+            "business_id": "-0Soj75v-XoRcf2ERr8Bmg",
+            "dates": [
+                "2019-06-05 18:22:49"
+            ]
+        },
+        {
+            "business_id": "-0ZumLlFjMh4ZW1z2nXGug",
+            "dates": [
+                "2011-09-24 21:37:32",
+                "2014-03-10 20:20:07",
+                "2015-05-27 00:40:24",
+                "2015-08-29 17:58:15",
+                "2018-03-16 15:03:26"
+            ]
+        },
+        {
+            "business_id": "-0aOudcaAyac0VJbMX-L1g",
+            "dates": [
+                "2015-03-16 23:51:16",
+                "2015-12-21 04:48:01",
+                "2016-10-28 20:22:42",
+                "2016-10-28 20:23:00"
+            ]
+        },
+        {
+            "business_id": "-0b86isaXMY0v4g-V8GZ9Q",
+            "dates": [
+                "2013-10-22 16:49:21",
+                "2014-11-21 17:39:24"
+            ]
+        },
+        {
+            "business_id": "-0d-BfFSU0bwLcnMaGRxYw",
+            "dates": [
+                "2014-08-07 18:30:48",
+                "2014-09-16 20:41:45",
+                "2014-10-12 23:22:27",
+                "2015-07-21 20:43:56",
+                "2015-07-21 20:45:07"
+            ]
+        },
+        {
+            "business_id": "-0jz6c3C6i7RG7Ag22K-Pg",
+            "dates": [
+                "2015-05-02 19:49:05",
+                "2015-05-06 03:52:18",
+                "2015-09-26 01:13:19"
+            ]
+        },
+        {
+            "business_id": "-0y3MZU2oYP8r1ruDP1bfQ",
+            "dates": [
+                "2015-04-11 13:14:14",
+                "2015-11-21 16:05:56",
+                "2016-05-06 14:10:04",
+                "2017-08-09 15:15:10",
+                "2017-10-21 15:12:56"
+            ]
+        },
+        {
+            "business_id": "-1BPe8UjF2_l3nVk-DFUjA",
+            "dates": [
+                "2015-12-03 18:44:00",
+                "2016-03-17 18:19:21",
+                "2016-11-02 15:58:38"
+            ]
+        },
+        {
+            "business_id": "-1E2CQu_38mkghvmZgCCRw",
+            "dates": [
+                "2019-04-04 22:02:37"
+            ]
+        },
+        {
+            "business_id": "-1wzk43IZ5D9Ysu6kzb5xA",
+            "dates": [
+                "2019-02-27 14:03:08"
+            ]
+        },
+        {
+            "business_id": "-23R9P2eG7VTc6DVLjFKzA",
+            "dates": [
+                "2011-12-21 19:02:51",
+                "2012-04-15 04:21:39",
+                "2012-04-15 14:23:56",
+                "2013-06-30 22:39:51",
+                "2013-10-04 20:34:13",
+                "2014-07-16 02:28:40"
+            ]
+        },
+        {
+            "business_id": "-26MGfikhJiTfCI-GqmzhQ",
+            "dates": [
+                "2018-06-13 20:16:07"
+            ]
+        },
+        {
+            "business_id": "-2bLuJsMZ0WhI9daurVQNQ",
+            "dates": [
+                "2015-05-29 16:46:17",
+                "2015-06-01 15:03:53"
+            ]
+        },
+        {
+            "business_id": "-2hDBMaza_ldqnZdiU06LQ",
+            "dates": [
+                "2011-10-08 12:02:23",
+                "2014-08-18 02:11:11",
+                "2016-01-07 05:27:51",
+                "2016-10-21 20:15:55",
+                "2016-12-01 03:57:10",
+                "2016-12-29 01:54:42",
+                "2018-07-22 19:55:31",
+                "2018-09-07 01:42:54",
+                "2019-03-08 03:41:06"
+            ]
+        }
+    ]
+);
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/select-unnest-queries/closed/with-filter-fields/with-filter-fields.3.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/select-unnest-queries/closed/with-filter-fields/with-filter-fields.3.ddl.sqlpp
new file mode 100644
index 0000000..a0fc1bf
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/select-unnest-queries/closed/with-filter-fields/with-filter-fields.3.ddl.sqlpp
@@ -0,0 +1,22 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+USE TestYelp;
+
+CREATE INDEX IdxYelpCheckinDates ON YelpCheckin (UNNEST dates);
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/select-unnest-queries/closed/with-filter-fields/with-filter-fields.4.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/select-unnest-queries/closed/with-filter-fields/with-filter-fields.4.query.sqlpp
new file mode 100644
index 0000000..bd0f395
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/select-unnest-queries/closed/with-filter-fields/with-filter-fields.4.query.sqlpp
@@ -0,0 +1,26 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+SET `compiler.arrayindex` "true";
+
+USE TestYelp;
+
+SELECT C.business_id
+FROM YelpCheckin C, C.dates D
+WHERE "2016-04-26 19:49:16" = D AND 
+      C.business_id = "--1UhMGODdWsrMastO9DZw";
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/select-unnest-queries/closed/with-filter-fields/with-filter-fields.5.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/select-unnest-queries/closed/with-filter-fields/with-filter-fields.5.query.sqlpp
new file mode 100644
index 0000000..cba5055
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/select-unnest-queries/closed/with-filter-fields/with-filter-fields.5.query.sqlpp
@@ -0,0 +1,25 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+SET `compiler.arrayindex` "true";
+
+USE TestYelp;
+
+SELECT COUNT(*)
+FROM YelpCheckin C, C.dates D
+WHERE D > "2016" AND D < "2017";
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/select-unnest-queries/open/use-case-1/use-case-1.1.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/select-unnest-queries/open/use-case-1/use-case-1.1.ddl.sqlpp
new file mode 100644
index 0000000..c87f4ca
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/select-unnest-queries/open/use-case-1/use-case-1.1.ddl.sqlpp
@@ -0,0 +1,34 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+/*
+ * Description: Verify that we are able to correctly perform the following types of queries using an array->atomic index.
+ *     1) UNNEST query, performing a point search.
+ *     2) UNNEST query, performing a range search.
+ */
+
+DROP DATAVERSE TestYelp IF EXISTS;
+CREATE DATAVERSE TestYelp;
+USE TestYelp;
+
+CREATE TYPE CheckinType AS {
+    checkin_id: uuid
+};
+
+CREATE DATASET YelpCheckin(CheckinType) PRIMARY KEY checkin_id AUTOGENERATED;
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/select-unnest-queries/open/use-case-1/use-case-1.2.update.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/select-unnest-queries/open/use-case-1/use-case-1.2.update.sqlpp
new file mode 100644
index 0000000..b75bf1b
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/select-unnest-queries/open/use-case-1/use-case-1.2.update.sqlpp
@@ -0,0 +1,276 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+USE TestYelp;
+
+INSERT INTO YelpCheckin (
+    [
+        {
+            "business_id": "--1UhMGODdWsrMastO9DZw",
+            "dates": [
+                "2016-04-26 19:49:16",
+                "2016-08-30 18:36:57",
+                "2016-10-15 02:45:18",
+                "2016-11-18 01:54:50",
+                "2017-04-20 18:39:06",
+                "2017-05-03 17:58:02",
+                "2019-03-19 22:04:48"
+            ]
+        },
+        {
+            "business_id": "--EF5N7P70J_UYBTPypYlA",
+            "dates": [
+                "2018-05-25 19:52:07",
+                "2018-09-18 16:09:44",
+                "2019-10-18 21:29:09"
+            ]
+        },
+        {
+            "business_id": "--Ni3oJ4VOqfOEu7Sj2Vzg",
+            "dates": [
+                "2019-06-07 17:54:58"
+            ]
+        },
+        {
+            "business_id": "--Y1Adl1YUWfYIRSd8vkmA",
+            "dates": [
+                "2011-05-03 20:54:05",
+                "2011-08-23 20:49:45",
+                "2014-12-04 06:13:01",
+                "2016-11-16 19:25:55"
+            ]
+        },
+        {
+            "business_id": "--YPwqIlRJrhHkJcjY3eiA",
+            "dates": [
+                "2016-06-18 21:35:45",
+                "2016-10-15 18:17:51"
+            ]
+        },
+        {
+            "business_id": "--e8PjCNhEz32pprnPhCwQ",
+            "dates": [
+                "2015-04-02 21:45:17"
+            ]
+        },
+        {
+            "business_id": "--kinfHwmtdjz03g8B8z8Q",
+            "dates": [
+                "2014-08-27 17:49:18",
+                "2015-12-19 21:30:31",
+                "2018-11-27 15:53:50"
+            ]
+        },
+        {
+            "business_id": "--q6datkI-f0EoVheXNEeQ",
+            "dates": [
+                "2014-01-28 20:56:04",
+                "2014-11-16 16:11:58",
+                "2015-11-15 19:21:53",
+                "2015-11-15 19:33:39"
+            ]
+        },
+        {
+            "business_id": "--qvQS4MigHPykD2GV0-zw",
+            "dates": [
+                "2019-04-11 18:30:12"
+            ]
+        },
+        {
+            "business_id": "--wIGbLEhlpl_UeAIyDmZQ",
+            "dates": [
+                "2015-06-06 20:01:06",
+                "2019-03-14 22:01:52"
+            ]
+        },
+        {
+            "business_id": "-0FA-Qdi3SPYIoJz9UQw-A",
+            "dates": [
+                "2018-09-29 18:55:17",
+                "2018-10-20 16:48:05",
+                "2018-10-20 22:20:24"
+            ]
+        },
+        {
+            "business_id": "-0Hj1hb_XW6ybWq2M7QhGA",
+            "dates": [
+                "2011-04-23 21:11:22",
+                "2014-05-04 19:42:48",
+                "2014-05-11 19:16:08",
+                "2014-06-04 19:14:18",
+                "2015-12-05 19:22:42",
+                "2017-05-15 23:19:00"
+            ]
+        },
+        {
+            "business_id": "-0KMvRFwDWdVBeTpT11iHw",
+            "dates": [
+                "2012-07-13 21:43:57",
+                "2016-12-24 02:27:31",
+                "2017-08-31 00:35:26"
+            ]
+        },
+        {
+            "business_id": "-0LPtgJC31FWMrMv317p0Q",
+            "dates": [
+                "2013-04-13 12:35:33",
+                "2013-08-19 23:35:49",
+                "2013-10-04 19:14:56"
+            ]
+        },
+        {
+            "business_id": "-0M3o2uWBnQZwd3hmfEwuw",
+            "dates": [
+                "2016-09-10 19:26:19",
+                "2018-09-08 14:15:37",
+                "2019-09-13 22:47:25"
+            ]
+        },
+        {
+            "business_id": "-0RRiWDtfnS16AKCtfvBZg",
+            "dates": [
+                "2017-05-19 14:30:16",
+                "2017-05-19 14:30:25",
+                "2017-08-28 15:49:37",
+                "2017-09-20 20:19:51",
+                "2017-10-01 16:31:05",
+                "2017-10-01 16:56:27",
+                "2017-12-27 23:33:20"
+            ]
+        },
+        {
+            "business_id": "-0Soj75v-XoRcf2ERr8Bmg",
+            "dates": [
+                "2019-06-05 18:22:49"
+            ]
+        },
+        {
+            "business_id": "-0ZumLlFjMh4ZW1z2nXGug",
+            "dates": [
+                "2011-09-24 21:37:32",
+                "2014-03-10 20:20:07",
+                "2015-05-27 00:40:24",
+                "2015-08-29 17:58:15",
+                "2018-03-16 15:03:26"
+            ]
+        },
+        {
+            "business_id": "-0aOudcaAyac0VJbMX-L1g",
+            "dates": [
+                "2015-03-16 23:51:16",
+                "2015-12-21 04:48:01",
+                "2016-10-28 20:22:42",
+                "2016-10-28 20:23:00"
+            ]
+        },
+        {
+            "business_id": "-0b86isaXMY0v4g-V8GZ9Q",
+            "dates": [
+                "2013-10-22 16:49:21",
+                "2014-11-21 17:39:24"
+            ]
+        },
+        {
+            "business_id": "-0d-BfFSU0bwLcnMaGRxYw",
+            "dates": [
+                "2014-08-07 18:30:48",
+                "2014-09-16 20:41:45",
+                "2014-10-12 23:22:27",
+                "2015-07-21 20:43:56",
+                "2015-07-21 20:45:07"
+            ]
+        },
+        {
+            "business_id": "-0jz6c3C6i7RG7Ag22K-Pg",
+            "dates": [
+                "2015-05-02 19:49:05",
+                "2015-05-06 03:52:18",
+                "2015-09-26 01:13:19"
+            ]
+        },
+        {
+            "business_id": "-0y3MZU2oYP8r1ruDP1bfQ",
+            "dates": [
+                "2015-04-11 13:14:14",
+                "2015-11-21 16:05:56",
+                "2016-05-06 14:10:04",
+                "2017-08-09 15:15:10",
+                "2017-10-21 15:12:56"
+            ]
+        },
+        {
+            "business_id": "-1BPe8UjF2_l3nVk-DFUjA",
+            "dates": [
+                "2015-12-03 18:44:00",
+                "2016-03-17 18:19:21",
+                "2016-11-02 15:58:38"
+            ]
+        },
+        {
+            "business_id": "-1E2CQu_38mkghvmZgCCRw",
+            "dates": [
+                "2019-04-04 22:02:37"
+            ]
+        },
+        {
+            "business_id": "-1wzk43IZ5D9Ysu6kzb5xA",
+            "dates": [
+                "2019-02-27 14:03:08"
+            ]
+        },
+        {
+            "business_id": "-23R9P2eG7VTc6DVLjFKzA",
+            "dates": [
+                "2011-12-21 19:02:51",
+                "2012-04-15 04:21:39",
+                "2012-04-15 14:23:56",
+                "2013-06-30 22:39:51",
+                "2013-10-04 20:34:13",
+                "2014-07-16 02:28:40"
+            ]
+        },
+        {
+            "business_id": "-26MGfikhJiTfCI-GqmzhQ",
+            "dates": [
+                "2018-06-13 20:16:07"
+            ]
+        },
+        {
+            "business_id": "-2bLuJsMZ0WhI9daurVQNQ",
+            "dates": [
+                "2015-05-29 16:46:17",
+                "2015-06-01 15:03:53"
+            ]
+        },
+        {
+            "business_id": "-2hDBMaza_ldqnZdiU06LQ",
+            "dates": [
+                "2011-10-08 12:02:23",
+                "2014-08-18 02:11:11",
+                "2016-01-07 05:27:51",
+                "2016-10-21 20:15:55",
+                "2016-12-01 03:57:10",
+                "2016-12-29 01:54:42",
+                "2018-07-22 19:55:31",
+                "2018-09-07 01:42:54",
+                "2019-03-08 03:41:06"
+            ]
+        }
+    ]
+);
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/select-unnest-queries/open/use-case-1/use-case-1.3.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/select-unnest-queries/open/use-case-1/use-case-1.3.ddl.sqlpp
new file mode 100644
index 0000000..b933d8a
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/select-unnest-queries/open/use-case-1/use-case-1.3.ddl.sqlpp
@@ -0,0 +1,22 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+USE TestYelp;
+
+CREATE INDEX IdxYelpCheckinDates ON YelpCheckin (UNNEST dates : string ?) ;
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/select-unnest-queries/open/use-case-1/use-case-1.4.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/select-unnest-queries/open/use-case-1/use-case-1.4.query.sqlpp
new file mode 100644
index 0000000..ec79bc6
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/select-unnest-queries/open/use-case-1/use-case-1.4.query.sqlpp
@@ -0,0 +1,25 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+SET `compiler.arrayindex` "true";
+
+USE TestYelp;
+
+SELECT C.business_id
+FROM YelpCheckin C, C.dates D
+WHERE "2016-04-26 19:49:16" = D;
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/select-unnest-queries/open/use-case-1/use-case-1.5.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/select-unnest-queries/open/use-case-1/use-case-1.5.query.sqlpp
new file mode 100644
index 0000000..cba5055
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/select-unnest-queries/open/use-case-1/use-case-1.5.query.sqlpp
@@ -0,0 +1,25 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+SET `compiler.arrayindex` "true";
+
+USE TestYelp;
+
+SELECT COUNT(*)
+FROM YelpCheckin C, C.dates D
+WHERE D > "2016" AND D < "2017";
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/select-unnest-queries/open/use-case-2/use-case-2.1.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/select-unnest-queries/open/use-case-2/use-case-2.1.ddl.sqlpp
new file mode 100644
index 0000000..6dbc88c
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/select-unnest-queries/open/use-case-2/use-case-2.1.ddl.sqlpp
@@ -0,0 +1,34 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+/*
+ * Description: Verify that we are able to correctly perform the following types of queries using a record->array->atomic index.
+ *     1) UNNEST query, performing a point search.
+ *     2) UNNEST query, performing a range search.
+ */
+
+DROP DATAVERSE TestYelp IF EXISTS;
+CREATE DATAVERSE TestYelp;
+USE TestYelp;
+
+CREATE TYPE CheckinType AS {
+    checkin_id: uuid
+};
+
+CREATE DATASET YelpCheckin(CheckinType) PRIMARY KEY checkin_id AUTOGENERATED;
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/select-unnest-queries/open/use-case-2/use-case-2.2.update.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/select-unnest-queries/open/use-case-2/use-case-2.2.update.sqlpp
new file mode 100644
index 0000000..2549e8f
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/select-unnest-queries/open/use-case-2/use-case-2.2.update.sqlpp
@@ -0,0 +1,497 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+USE TestYelp;
+
+INSERT INTO YelpCheckin (
+    [
+        {
+            "business_id": "--1UhMGODdWsrMastO9DZw",
+            "checkin_times": {
+                "dates": [
+                    "2016-04-26",
+                    "2016-08-30",
+                    "2016-10-15",
+                    "2016-11-18",
+                    "2017-04-20",
+                    "2017-05-03",
+                    "2019-03-19"
+                ],
+                "times": [
+                    "19:49:16",
+                    "18:36:57",
+                    "02:45:18",
+                    "01:54:50",
+                    "18:39:06",
+                    "17:58:02",
+                    "22:04:48"
+                ]
+            }
+        },
+        {
+            "business_id": "--EF5N7P70J_UYBTPypYlA",
+            "checkin_times": {
+                "dates": [
+                    "2018-05-25",
+                    "2018-09-18",
+                    "2019-10-18"
+                ],
+                "times": [
+                    "19:52:07",
+                    "16:09:44",
+                    "21:29:09"
+                ]
+            }
+        },
+        {
+            "business_id": "--Ni3oJ4VOqfOEu7Sj2Vzg",
+            "checkin_times": {
+                "dates": [
+                    "2019-06-07"
+                ],
+                "times": [
+                    "17:54:58"
+                ]
+            }
+        },
+        {
+            "business_id": "--Y1Adl1YUWfYIRSd8vkmA",
+            "checkin_times": {
+                "dates": [
+                    "2011-05-03",
+                    "2011-08-23",
+                    "2014-12-04",
+                    "2016-11-16"
+                ],
+                "times": [
+                    "20:54:05",
+                    "20:49:45",
+                    "06:13:01",
+                    "19:25:55"
+                ]
+            }
+        },
+        {
+            "business_id": "--YPwqIlRJrhHkJcjY3eiA",
+            "checkin_times": {
+                "dates": [
+                    "2016-06-18",
+                    "2016-10-15"
+                ],
+                "times": [
+                    "21:35:45",
+                    "18:17:51"
+                ]
+            }
+        },
+        {
+            "business_id": "--e8PjCNhEz32pprnPhCwQ",
+            "checkin_times": {
+                "dates": [
+                    "2015-04-02"
+                ],
+                "times": [
+                    "21:45:17"
+                ]
+            }
+        },
+        {
+            "business_id": "--kinfHwmtdjz03g8B8z8Q",
+            "checkin_times": {
+                "dates": [
+                    "2014-08-27",
+                    "2015-12-19",
+                    "2018-11-27"
+                ],
+                "times": [
+                    "17:49:18",
+                    "21:30:31",
+                    "15:53:50"
+                ]
+            }
+        },
+        {
+            "business_id": "--q6datkI-f0EoVheXNEeQ",
+            "checkin_times": {
+                "dates": [
+                    "2014-01-28",
+                    "2014-11-16",
+                    "2015-11-15",
+                    "2015-11-15"
+                ],
+                "times": [
+                    "20:56:04",
+                    "16:11:58",
+                    "19:21:53",
+                    "19:33:39"
+                ]
+            }
+        },
+        {
+            "business_id": "--qvQS4MigHPykD2GV0-zw",
+            "checkin_times": {
+                "dates": [
+                    "2019-04-11"
+                ],
+                "times": [
+                    "18:30:12"
+                ]
+            }
+        },
+        {
+            "business_id": "--wIGbLEhlpl_UeAIyDmZQ",
+            "checkin_times": {
+                "dates": [
+                    "2015-06-06",
+                    "2019-03-14"
+                ],
+                "times": [
+                    "20:01:06",
+                    "22:01:52"
+                ]
+            }
+        },
+        {
+            "business_id": "-0FA-Qdi3SPYIoJz9UQw-A",
+            "checkin_times": {
+                "dates": [
+                    "2018-09-29",
+                    "2018-10-20",
+                    "2018-10-20"
+                ],
+                "times": [
+                    "18:55:17",
+                    "16:48:05",
+                    "22:20:24"
+                ]
+            }
+        },
+        {
+            "business_id": "-0Hj1hb_XW6ybWq2M7QhGA",
+            "checkin_times": {
+                "dates": [
+                    "2011-04-23",
+                    "2014-05-04",
+                    "2014-05-11",
+                    "2014-06-04",
+                    "2015-12-05",
+                    "2017-05-15"
+                ],
+                "times": [
+                    "21:11:22",
+                    "19:42:48",
+                    "19:16:08",
+                    "19:14:18",
+                    "19:22:42",
+                    "23:19:00"
+                ]
+            }
+        },
+        {
+            "business_id": "-0KMvRFwDWdVBeTpT11iHw",
+            "checkin_times": {
+                "dates": [
+                    "2012-07-13",
+                    "2016-12-24",
+                    "2017-08-31"
+                ],
+                "times": [
+                    "21:43:57",
+                    "02:27:31",
+                    "00:35:26"
+                ]
+            }
+        },
+        {
+            "business_id": "-0LPtgJC31FWMrMv317p0Q",
+            "checkin_times": {
+                "dates": [
+                    "2013-04-13",
+                    "2013-08-19",
+                    "2013-10-04"
+                ],
+                "times": [
+                    "12:35:33",
+                    "23:35:49",
+                    "19:14:56"
+                ]
+            }
+        },
+        {
+            "business_id": "-0M3o2uWBnQZwd3hmfEwuw",
+            "checkin_times": {
+                "dates": [
+                    "2016-09-10",
+                    "2018-09-08",
+                    "2019-09-13"
+                ],
+                "times": [
+                    "19:26:19",
+                    "14:15:37",
+                    "22:47:25"
+                ]
+            }
+        },
+        {
+            "business_id": "-0RRiWDtfnS16AKCtfvBZg",
+            "checkin_times": {
+                "dates": [
+                    "2017-05-19",
+                    "2017-05-19",
+                    "2017-08-28",
+                    "2017-09-20",
+                    "2017-10-01",
+                    "2017-10-01",
+                    "2017-12-27"
+                ],
+                "times": [
+                    "14:30:16",
+                    "14:30:25",
+                    "15:49:37",
+                    "20:19:51",
+                    "16:31:05",
+                    "16:56:27",
+                    "23:33:20"
+                ]
+            }
+        },
+        {
+            "business_id": "-0Soj75v-XoRcf2ERr8Bmg",
+            "checkin_times": {
+                "dates": [
+                    "2019-06-05"
+                ],
+                "times": [
+                    "18:22:49"
+                ]
+            }
+        },
+        {
+            "business_id": "-0ZumLlFjMh4ZW1z2nXGug",
+            "checkin_times": {
+                "dates": [
+                    "2011-09-24",
+                    "2014-03-10",
+                    "2015-05-27",
+                    "2015-08-29",
+                    "2018-03-16"
+                ],
+                "times": [
+                    "21:37:32",
+                    "20:20:07",
+                    "00:40:24",
+                    "17:58:15",
+                    "15:03:26"
+                ]
+            }
+        },
+        {
+            "business_id": "-0aOudcaAyac0VJbMX-L1g",
+            "checkin_times": {
+                "dates": [
+                    "2015-03-16",
+                    "2015-12-21",
+                    "2016-10-28",
+                    "2016-10-28"
+                ],
+                "times": [
+                    "23:51:16",
+                    "04:48:01",
+                    "20:22:42",
+                    "20:23:00"
+                ]
+            }
+        },
+        {
+            "business_id": "-0b86isaXMY0v4g-V8GZ9Q",
+            "checkin_times": {
+                "dates": [
+                    "2013-10-22",
+                    "2014-11-21"
+                ],
+                "times": [
+                    "16:49:21",
+                    "17:39:24"
+                ]
+            }
+        },
+        {
+            "business_id": "-0d-BfFSU0bwLcnMaGRxYw",
+            "checkin_times": {
+                "dates": [
+                    "2014-08-07",
+                    "2014-09-16",
+                    "2014-10-12",
+                    "2015-07-21",
+                    "2015-07-21"
+                ],
+                "times": [
+                    "18:30:48",
+                    "20:41:45",
+                    "23:22:27",
+                    "20:43:56",
+                    "20:45:07"
+                ]
+            }
+        },
+        {
+            "business_id": "-0jz6c3C6i7RG7Ag22K-Pg",
+            "checkin_times": {
+                "dates": [
+                    "2015-05-02",
+                    "2015-05-06",
+                    "2015-09-26"
+                ],
+                "times": [
+                    "19:49:05",
+                    "03:52:18",
+                    "01:13:19"
+                ]
+            }
+        },
+        {
+            "business_id": "-0y3MZU2oYP8r1ruDP1bfQ",
+            "checkin_times": {
+                "dates": [
+                    "2015-04-11",
+                    "2015-11-21",
+                    "2016-05-06",
+                    "2017-08-09",
+                    "2017-10-21"
+                ],
+                "times": [
+                    "13:14:14",
+                    "16:05:56",
+                    "14:10:04",
+                    "15:15:10",
+                    "15:12:56"
+                ]
+            }
+        },
+        {
+            "business_id": "-1BPe8UjF2_l3nVk-DFUjA",
+            "checkin_times": {
+                "dates": [
+                    "2015-12-03",
+                    "2016-03-17",
+                    "2016-11-02"
+                ],
+                "times": [
+                    "18:44:00",
+                    "18:19:21",
+                    "15:58:38"
+                ]
+            }
+        },
+        {
+            "business_id": "-1E2CQu_38mkghvmZgCCRw",
+            "checkin_times": {
+                "dates": [
+                    "2019-04-04"
+                ],
+                "times": [
+                    "22:02:37"
+                ]
+            }
+        },
+        {
+            "business_id": "-1wzk43IZ5D9Ysu6kzb5xA",
+            "checkin_times": {
+                "dates": [
+                    "2019-02-27"
+                ],
+                "times": [
+                    "14:03:08"
+                ]
+            }
+        },
+        {
+            "business_id": "-23R9P2eG7VTc6DVLjFKzA",
+            "checkin_times": {
+                "dates": [
+                    "2011-12-21",
+                    "2012-04-15",
+                    "2012-04-15",
+                    "2013-06-30",
+                    "2013-10-04",
+                    "2014-07-16"
+                ],
+                "times": [
+                    "19:02:51",
+                    "04:21:39",
+                    "14:23:56",
+                    "22:39:51",
+                    "20:34:13",
+                    "02:28:40"
+                ]
+            }
+        },
+        {
+            "business_id": "-26MGfikhJiTfCI-GqmzhQ",
+            "checkin_times": {
+                "dates": [
+                    "2018-06-13"
+                ],
+                "times": [
+                    "20:16:07"
+                ]
+            }
+        },
+        {
+            "business_id": "-2bLuJsMZ0WhI9daurVQNQ",
+            "checkin_times": {
+                "dates": [
+                    "2015-05-29",
+                    "2015-06-01"
+                ],
+                "times": [
+                    "16:46:17",
+                    "15:03:53"
+                ]
+            }
+        },
+        {
+            "business_id": "-2hDBMaza_ldqnZdiU06LQ",
+            "checkin_times": {
+                "dates": [
+                    "2011-10-08",
+                    "2014-08-18",
+                    "2016-01-07",
+                    "2016-10-21",
+                    "2016-12-01",
+                    "2016-12-29",
+                    "2018-07-22",
+                    "2018-09-07",
+                    "2019-03-08"
+                ],
+                "times": [
+                    "12:02:23",
+                    "02:11:11",
+                    "05:27:51",
+                    "20:15:55",
+                    "03:57:10",
+                    "01:54:42",
+                    "19:55:31",
+                    "01:42:54",
+                    "03:41:06"
+                ]
+            }
+        }
+    ]
+);
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/select-unnest-queries/open/use-case-2/use-case-2.3.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/select-unnest-queries/open/use-case-2/use-case-2.3.ddl.sqlpp
new file mode 100644
index 0000000..51a1c82
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/select-unnest-queries/open/use-case-2/use-case-2.3.ddl.sqlpp
@@ -0,0 +1,22 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+USE TestYelp;
+
+CREATE INDEX IdxYelpCheckinDates ON YelpCheckin (UNNEST checkin_times.dates : string ?) ;
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/select-unnest-queries/open/use-case-2/use-case-2.4.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/select-unnest-queries/open/use-case-2/use-case-2.4.query.sqlpp
new file mode 100644
index 0000000..0f23cd4
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/select-unnest-queries/open/use-case-2/use-case-2.4.query.sqlpp
@@ -0,0 +1,25 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+SET `compiler.arrayindex` "true";
+
+USE TestYelp;
+
+SELECT C.business_id
+FROM YelpCheckin C, C.checkin_times.dates D
+WHERE "2016-04-26" = D;
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/select-unnest-queries/open/use-case-2/use-case-2.5.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/select-unnest-queries/open/use-case-2/use-case-2.5.query.sqlpp
new file mode 100644
index 0000000..9887068
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/select-unnest-queries/open/use-case-2/use-case-2.5.query.sqlpp
@@ -0,0 +1,25 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+SET `compiler.arrayindex` "true";
+
+USE TestYelp;
+
+SELECT COUNT(*)
+FROM YelpCheckin C, C.checkin_times.dates D
+WHERE D > "2016" AND D < "2017";
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/select-unnest-queries/open/use-case-3/use-case-3.1.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/select-unnest-queries/open/use-case-3/use-case-3.1.ddl.sqlpp
new file mode 100644
index 0000000..ca44009
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/select-unnest-queries/open/use-case-3/use-case-3.1.ddl.sqlpp
@@ -0,0 +1,35 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+/*
+ * Description: Verify that we are able to correctly perform the following types of queries using a array->record->atomic index.
+ *     1) UNNEST query, performing a point search.
+ *     2) UNNEST query, performing a range search.
+ *     3) UNNEST query, performing a point search with an additional predicate after the UNNEST.
+ */
+
+DROP DATAVERSE TestYelp IF EXISTS;
+CREATE DATAVERSE TestYelp;
+USE TestYelp;
+
+CREATE TYPE CheckinType AS {
+    checkin_id: uuid
+};
+
+CREATE DATASET YelpCheckin(CheckinType) PRIMARY KEY checkin_id AUTOGENERATED;
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/select-unnest-queries/open/use-case-3/use-case-3.2.update.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/select-unnest-queries/open/use-case-3/use-case-3.2.update.sqlpp
new file mode 100644
index 0000000..02ddf21
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/select-unnest-queries/open/use-case-3/use-case-3.2.update.sqlpp
@@ -0,0 +1,276 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+USE TestYelp;
+
+INSERT INTO YelpCheckin (
+    [
+        {
+            "business_id": "--1UhMGODdWsrMastO9DZw",
+            "checkin_times": [ 
+                {"date": "2016-04-26", "time": "19:49:16"},
+                {"date": "2016-08-30", "time": "18:36:57"},
+                {"date": "2016-10-15", "time": "02:45:18"},
+                {"date": "2016-11-18", "time": "01:54:50"},
+                {"date": "2017-04-20", "time": "18:39:06"},
+                {"date": "2017-05-03", "time": "17:58:02"},
+                {"date": "2019-03-19", "time": "22:04:48"}
+             ]
+        },
+        {
+            "business_id": "--EF5N7P70J_UYBTPypYlA",
+            "checkin_times": [ 
+                {"date": "2018-05-25", "time": "19:52:07"},
+                {"date": "2018-09-18", "time": "16:09:44"},
+                {"date": "2019-10-18", "time": "21:29:09"}
+             ]
+        },
+        {
+            "business_id": "--Ni3oJ4VOqfOEu7Sj2Vzg",
+            "checkin_times": [ 
+                {"date": "2019-06-07", "time": "17:54:58"}
+             ]
+        },
+        {
+            "business_id": "--Y1Adl1YUWfYIRSd8vkmA",
+            "checkin_times": [ 
+                {"date": "2011-05-03", "time": "20:54:05"},
+                {"date": "2011-08-23", "time": "20:49:45"},
+                {"date": "2014-12-04", "time": "06:13:01"},
+                {"date": "2016-11-16", "time": "19:25:55"}
+             ]
+        },
+        {
+            "business_id": "--YPwqIlRJrhHkJcjY3eiA",
+            "checkin_times": [ 
+                {"date": "2016-06-18", "time": "21:35:45"},
+                {"date": "2016-10-15", "time": "18:17:51"}
+             ]
+        },
+        {
+            "business_id": "--e8PjCNhEz32pprnPhCwQ",
+            "checkin_times": [ 
+                {"date": "2015-04-02", "time": "21:45:17"}
+             ]
+        },
+        {
+            "business_id": "--kinfHwmtdjz03g8B8z8Q",
+            "checkin_times": [ 
+                {"date": "2014-08-27", "time": "17:49:18"},
+                {"date": "2015-12-19", "time": "21:30:31"},
+                {"date": "2018-11-27", "time": "15:53:50"}
+             ]
+        },
+        {
+            "business_id": "--q6datkI-f0EoVheXNEeQ",
+            "checkin_times": [ 
+                {"date": "2014-01-28", "time": "20:56:04"},
+                {"date": "2014-11-16", "time": "16:11:58"},
+                {"date": "2015-11-15", "time": "19:21:53"},
+                {"date": "2015-11-15", "time": "19:33:39"}
+             ]
+        },
+        {
+            "business_id": "--qvQS4MigHPykD2GV0-zw",
+            "checkin_times": [ 
+                {"date": "2019-04-11", "time": "18:30:12"}
+             ]
+        },
+        {
+            "business_id": "--wIGbLEhlpl_UeAIyDmZQ",
+            "checkin_times": [ 
+                {"date": "2015-06-06", "time": "20:01:06"},
+                {"date": "2019-03-14", "time": "22:01:52"}
+             ]
+        },
+        {
+            "business_id": "-0FA-Qdi3SPYIoJz9UQw-A",
+            "checkin_times": [ 
+                {"date": "2018-09-29", "time": "18:55:17"},
+                {"date": "2018-10-20", "time": "16:48:05"},
+                {"date": "2018-10-20", "time": "22:20:24"}
+             ]
+        },
+        {
+            "business_id": "-0Hj1hb_XW6ybWq2M7QhGA",
+            "checkin_times": [ 
+                {"date": "2011-04-23", "time": "21:11:22"},
+                {"date": "2014-05-04", "time": "19:42:48"},
+                {"date": "2014-05-11", "time": "19:16:08"},
+                {"date": "2014-06-04", "time": "19:14:18"},
+                {"date": "2015-12-05", "time": "19:22:42"},
+                {"date": "2017-05-15", "time": "23:19:00"}
+             ]
+        },
+        {
+            "business_id": "-0KMvRFwDWdVBeTpT11iHw",
+            "checkin_times": [ 
+                {"date": "2012-07-13", "time": "21:43:57"},
+                {"date": "2016-12-24", "time": "02:27:31"},
+                {"date": "2017-08-31", "time": "00:35:26"}
+             ]
+        },
+        {
+            "business_id": "-0LPtgJC31FWMrMv317p0Q",
+            "checkin_times": [ 
+                {"date": "2013-04-13", "time": "12:35:33"},
+                {"date": "2013-08-19", "time": "23:35:49"},
+                {"date": "2013-10-04", "time": "19:14:56"}
+             ]
+        },
+        {
+            "business_id": "-0M3o2uWBnQZwd3hmfEwuw",
+            "checkin_times": [ 
+                {"date": "2016-09-10", "time": "19:26:19"},
+                {"date": "2018-09-08", "time": "14:15:37"},
+                {"date": "2019-09-13", "time": "22:47:25"}
+             ]
+        },
+        {
+            "business_id": "-0RRiWDtfnS16AKCtfvBZg",
+            "checkin_times": [ 
+                {"date": "2017-05-19", "time": "14:30:16"},
+                {"date": "2017-05-19", "time": "14:30:25"},
+                {"date": "2017-08-28", "time": "15:49:37"},
+                {"date": "2017-09-20", "time": "20:19:51"},
+                {"date": "2017-10-01", "time": "16:31:05"},
+                {"date": "2017-10-01", "time": "16:56:27"},
+                {"date": "2017-12-27", "time": "23:33:20"}
+             ]
+        },
+        {
+            "business_id": "-0Soj75v-XoRcf2ERr8Bmg",
+            "checkin_times": [ 
+                {"date": "2019-06-05", "time": "18:22:49"}
+             ]
+        },
+        {
+            "business_id": "-0ZumLlFjMh4ZW1z2nXGug",
+            "checkin_times": [ 
+                {"date": "2011-09-24", "time": "21:37:32"},
+                {"date": "2014-03-10", "time": "20:20:07"},
+                {"date": "2015-05-27", "time": "00:40:24"},
+                {"date": "2015-08-29", "time": "17:58:15"},
+                {"date": "2018-03-16", "time": "15:03:26"}
+             ]
+        },
+        {
+            "business_id": "-0aOudcaAyac0VJbMX-L1g",
+            "checkin_times": [ 
+                {"date": "2015-03-16", "time": "23:51:16"},
+                {"date": "2015-12-21", "time": "04:48:01"},
+                {"date": "2016-10-28", "time": "20:22:42"},
+                {"date": "2016-10-28", "time": "20:23:00"}
+             ]
+        },
+        {
+            "business_id": "-0b86isaXMY0v4g-V8GZ9Q",
+            "checkin_times": [ 
+                {"date": "2013-10-22", "time": "16:49:21"},
+                {"date": "2014-11-21", "time": "17:39:24"}
+             ]
+        },
+        {
+            "business_id": "-0d-BfFSU0bwLcnMaGRxYw",
+            "checkin_times": [ 
+                {"date": "2014-08-07", "time": "18:30:48"},
+                {"date": "2014-09-16", "time": "20:41:45"},
+                {"date": "2014-10-12", "time": "23:22:27"},
+                {"date": "2015-07-21", "time": "20:43:56"},
+                {"date": "2015-07-21", "time": "20:45:07"}
+             ]
+        },
+        {
+            "business_id": "-0jz6c3C6i7RG7Ag22K-Pg",
+            "checkin_times": [ 
+                {"date": "2015-05-02", "time": "19:49:05"},
+                {"date": "2015-05-06", "time": "03:52:18"},
+                {"date": "2015-09-26", "time": "01:13:19"}
+             ]
+        },
+        {
+            "business_id": "-0y3MZU2oYP8r1ruDP1bfQ",
+            "checkin_times": [ 
+                {"date": "2015-04-11", "time": "13:14:14"},
+                {"date": "2015-11-21", "time": "16:05:56"},
+                {"date": "2016-05-06", "time": "14:10:04"},
+                {"date": "2017-08-09", "time": "15:15:10"},
+                {"date": "2017-10-21", "time": "15:12:56"}
+             ]
+        },
+        {
+            "business_id": "-1BPe8UjF2_l3nVk-DFUjA",
+            "checkin_times": [ 
+                {"date": "2015-12-03", "time": "18:44:00"},
+                {"date": "2016-03-17", "time": "18:19:21"},
+                {"date": "2016-11-02", "time": "15:58:38"}
+             ]
+        },
+        {
+            "business_id": "-1E2CQu_38mkghvmZgCCRw",
+            "checkin_times": [ 
+                {"date": "2019-04-04", "time": "22:02:37"}
+             ]
+        },
+        {
+            "business_id": "-1wzk43IZ5D9Ysu6kzb5xA",
+            "checkin_times": [ 
+                {"date": "2019-02-27", "time": "14:03:08"}
+             ]
+        },
+        {
+            "business_id": "-23R9P2eG7VTc6DVLjFKzA",
+            "checkin_times": [ 
+                {"date": "2011-12-21", "time": "19:02:51"},
+                {"date": "2012-04-15", "time": "04:21:39"},
+                {"date": "2012-04-15", "time": "14:23:56"},
+                {"date": "2013-06-30", "time": "22:39:51"},
+                {"date": "2013-10-04", "time": "20:34:13"},
+                {"date": "2014-07-16", "time": "02:28:40"}
+             ]
+        },
+        {
+            "business_id": "-26MGfikhJiTfCI-GqmzhQ",
+            "checkin_times": [ 
+                {"date": "2018-06-13", "time": "20:16:07"}
+             ]
+        },
+        {
+            "business_id": "-2bLuJsMZ0WhI9daurVQNQ",
+            "checkin_times": [ 
+                {"date": "2015-05-29", "time": "16:46:17"},
+                {"date": "2015-06-01", "time": "15:03:53"}
+             ]
+        },
+        {
+            "business_id": "-2hDBMaza_ldqnZdiU06LQ",
+            "checkin_times": [ 
+                {"date": "2011-10-08", "time": "12:02:23"},
+                {"date": "2014-08-18", "time": "02:11:11"},
+                {"date": "2016-01-07", "time": "05:27:51"},
+                {"date": "2016-10-21", "time": "20:15:55"},
+                {"date": "2016-12-01", "time": "03:57:10"},
+                {"date": "2016-12-29", "time": "01:54:42"},
+                {"date": "2018-07-22", "time": "19:55:31"},
+                {"date": "2018-09-07", "time": "01:42:54"},
+                {"date": "2019-03-08", "time": "03:41:06"}
+             ]
+        }
+    ]
+);
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/select-unnest-queries/open/use-case-3/use-case-3.3.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/select-unnest-queries/open/use-case-3/use-case-3.3.ddl.sqlpp
new file mode 100644
index 0000000..aa50e82
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/select-unnest-queries/open/use-case-3/use-case-3.3.ddl.sqlpp
@@ -0,0 +1,22 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+USE TestYelp;
+
+CREATE INDEX IdxYelpCheckinDates ON YelpCheckin (UNNEST checkin_times SELECT date : string ?) ;
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/select-unnest-queries/open/use-case-3/use-case-3.4.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/select-unnest-queries/open/use-case-3/use-case-3.4.query.sqlpp
new file mode 100644
index 0000000..912cfcb
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/select-unnest-queries/open/use-case-3/use-case-3.4.query.sqlpp
@@ -0,0 +1,25 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+SET `compiler.arrayindex` "true";
+
+USE TestYelp;
+
+SELECT C.business_id
+FROM YelpCheckin C, C.checkin_times D
+WHERE "2016-04-26" = D.date;
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/select-unnest-queries/open/use-case-3/use-case-3.5.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/select-unnest-queries/open/use-case-3/use-case-3.5.query.sqlpp
new file mode 100644
index 0000000..244ae62
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/select-unnest-queries/open/use-case-3/use-case-3.5.query.sqlpp
@@ -0,0 +1,25 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+SET `compiler.arrayindex` "true";
+
+USE TestYelp;
+
+SELECT COUNT(*)
+FROM YelpCheckin C, C.checkin_times D
+WHERE D.date > "2016" AND D.date < "2017";
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/select-unnest-queries/open/use-case-3/use-case-3.6.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/select-unnest-queries/open/use-case-3/use-case-3.6.query.sqlpp
new file mode 100644
index 0000000..4ab2742
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/select-unnest-queries/open/use-case-3/use-case-3.6.query.sqlpp
@@ -0,0 +1,26 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+SET `compiler.arrayindex` "true";
+
+USE TestYelp;
+
+SELECT C.business_id
+FROM YelpCheckin C, C.checkin_times D
+WHERE D.date = "2016-04-26" AND 
+      D.time = "19:49:16";
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/select-unnest-queries/open/use-case-4/use-case-4.1.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/select-unnest-queries/open/use-case-4/use-case-4.1.ddl.sqlpp
new file mode 100644
index 0000000..f1cc5fb
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/select-unnest-queries/open/use-case-4/use-case-4.1.ddl.sqlpp
@@ -0,0 +1,34 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+/*
+ * Description: Verify that we are able to correctly perform the following types of queries using a array->record->array->atomic index.
+ *     1) UNNEST query, performing a point search.
+ *     2) UNNEST query, performing a range search.
+ */
+
+DROP DATAVERSE TestYelp IF EXISTS;
+CREATE DATAVERSE TestYelp;
+USE TestYelp;
+
+CREATE TYPE CheckinType AS {
+    checkin_id: uuid
+};
+
+CREATE DATASET YelpCheckin(CheckinType) PRIMARY KEY checkin_id AUTOGENERATED;
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/select-unnest-queries/open/use-case-4/use-case-4.2.update.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/select-unnest-queries/open/use-case-4/use-case-4.2.update.sqlpp
new file mode 100644
index 0000000..7fcf9e4
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/select-unnest-queries/open/use-case-4/use-case-4.2.update.sqlpp
@@ -0,0 +1,425 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+USE TestYelp;
+
+INSERT INTO YelpCheckin (
+    [
+        {
+            "business_id": "--1UhMGODdWsrMastO9DZw",
+            "checkin_times": [ 
+                {
+                    "dates": ["2016-04-26", "2016-08-30", "2016-10-15", "2016-11-18"],
+                    "times": ["19:49:16", "18:36:57", "02:45:18", "01:54:50"]
+                },
+                {
+                    "dates": ["2017-04-20", "2017-05-03"],
+                    "times": ["18:39:06", "17:58:02"]
+                },
+                {
+                    "dates": ["2019-03-19"],
+                    "times": ["22:04:48"]
+                }
+             ]
+        },
+        {
+            "business_id": "--EF5N7P70J_UYBTPypYlA",
+            "checkin_times": [ 
+                {
+                    "dates": ["2018-05-25", "2018-09-18"],
+                    "times": ["19:52:07", "16:09:44"]
+                },
+                {
+                    "dates": ["2019-10-18"],
+                    "times": ["21:29:09"]
+                }
+             ]
+        },
+        {
+            "business_id": "--Ni3oJ4VOqfOEu7Sj2Vzg",
+            "checkin_times": [ 
+                {
+                    "dates": ["2019-06-07"],
+                    "times": ["17:54:58"]
+                }
+             ]
+        },
+        {
+            "business_id": "--Y1Adl1YUWfYIRSd8vkmA",
+            "checkin_times": [ 
+                {
+                    "dates": ["2011-05-03", "2011-08-23"],
+                    "times": ["20:54:05", "20:49:45"]
+                },
+                {
+                    "dates": ["2014-12-04"],
+                    "times": ["06:13:01"]
+                },
+                {
+                    "dates": ["2016-11-16"],
+                    "times": ["19:25:55"]
+                }
+             ]
+        },
+        {
+            "business_id": "--YPwqIlRJrhHkJcjY3eiA",
+            "checkin_times": [ 
+               {
+                    "dates": ["2016-06-18", "2016-10-15"],
+                    "times": ["21:35:45", "18:17:51"]
+                }
+             ]
+        },
+        {
+            "business_id": "--e8PjCNhEz32pprnPhCwQ",
+            "checkin_times": [ 
+                {
+                    "dates": ["2015-04-02"],
+                    "times": ["21:45:17"]
+                }
+             ]
+        },
+        {
+            "business_id": "--kinfHwmtdjz03g8B8z8Q",
+            "checkin_times": [ 
+                {
+                    "dates": ["2014-08-27"],
+                    "times": ["17:49:18"]
+                },
+                {
+                    "dates": ["2015-12-19"],
+                    "times": ["21:30:31"]
+                },
+                {
+                    "dates": ["2018-11-27"],
+                    "times": ["15:53:50"]
+                }
+             ]
+        },
+        {
+            "business_id": "--q6datkI-f0EoVheXNEeQ",
+            "checkin_times": [ 
+                {
+                    "dates": ["2014-01-28", "2014-11-16"],
+                    "times": ["20:56:04", "16:11:58"]
+                },
+                {
+                    "dates": ["2015-11-15", "2015-11-15"],
+                    "times": ["19:21:53", "19:33:39"]
+                }
+             ]
+        },
+        {
+            "business_id": "--qvQS4MigHPykD2GV0-zw",
+            "checkin_times": [ 
+                {
+                    "dates": ["2019-04-11"],
+                    "times": ["18:30:12"]
+                }
+             ]
+        },
+        {
+            "business_id": "--wIGbLEhlpl_UeAIyDmZQ",
+            "checkin_times": [ 
+                {
+                    "dates": ["2015-06-06"],
+                    "times": ["20:01:06"]
+                },
+                {
+                    "dates": ["2019-03-14"],
+                    "times": ["22:01:52"]
+                }
+             ]
+        },
+        {
+            "business_id": "-0FA-Qdi3SPYIoJz9UQw-A",
+            "checkin_times": [ 
+                {
+                    "dates": ["2018-09-29", "2018-10-20", "2018-10-20"],
+                    "times": ["18:55:17", "16:48:05", "22:20:24"]
+                }
+             ]
+        },
+        {
+            "business_id": "-0Hj1hb_XW6ybWq2M7QhGA",
+            "checkin_times": [ 
+                {
+                    "dates": ["2011-04-23"],
+                    "times": ["21:11:22"]
+                },
+                {
+                    "dates": ["2014-05-04", "2014-05-11", "2014-06-04"],
+                    "times": ["19:42:48", "19:16:08", "19:14:18"]
+                },
+                {
+                    "dates": ["2015-12-05"],
+                    "times": ["19:22:42"]
+                },
+                {
+                    "dates": ["2017-05-15"],
+                    "times": ["23:19:00"]
+                }
+             ]
+        },
+        {
+            "business_id": "-0KMvRFwDWdVBeTpT11iHw",
+            "checkin_times": [ 
+                {
+                    "dates": ["2012-07-13"],
+                    "times": ["21:43:57"]
+                },
+                {
+                    "dates": ["2016-12-24"],
+                    "times": ["02:27:31"]
+                },
+                {
+                    "dates": ["2017-08-31"],
+                    "times": ["00:35:26"]
+                }
+             ]
+        },
+        {
+            "business_id": "-0LPtgJC31FWMrMv317p0Q",
+            "checkin_times": [ 
+                {
+                    "dates": ["2013-04-13", "2013-08-19", "2013-10-04"],
+                    "times": ["12:35:33", "23:35:49", "19:14:56"]
+                }
+
+             ]
+        },
+        {
+            "business_id": "-0M3o2uWBnQZwd3hmfEwuw",
+            "checkin_times": [ 
+                {
+                    "dates": ["2016-09-10"],
+                    "times": ["19:26:19"]
+                },
+                {
+                    "dates": ["2018-09-08"],
+                    "times": ["14:15:37"]
+                },
+                {
+                    "dates": ["2019-09-13"],
+                    "times": ["22:47:25"]
+                }
+             ]
+        },
+        {
+            "business_id": "-0RRiWDtfnS16AKCtfvBZg",
+            "checkin_times": [ 
+                 {
+                    "dates": ["2017-05-19", "2017-05-19", "2017-08-28", "2017-09-20", "2017-10-01", "2017-10-01", "2017-12-27"],
+                    "times": ["14:30:16", "14:30:25", "15:49:37", "20:19:51", "16:31:05", "16:56:27", "23:33:20"]
+                }
+             ]
+        },
+        {
+            "business_id": "-0Soj75v-XoRcf2ERr8Bmg",
+            "checkin_times": [ 
+                {
+                    "dates": ["2019-06-05"],
+                    "times": ["18:22:49"]
+                }
+             ]
+        },
+        {
+            "business_id": "-0ZumLlFjMh4ZW1z2nXGug",
+            "checkin_times": [ 
+                {
+                    "dates": ["2011-09-24"],
+                    "times": ["21:37:32"]
+                },
+                {
+                    "dates": ["2014-03-10"],
+                    "times": ["20:20:07"]
+                },
+                {
+                    "dates": ["2015-05-27", "2015-08-29"],
+                    "times": ["00:40:24", "17:58:15"]
+                },
+                {
+                    "dates": ["2018-03-16"],
+                    "times": ["15:03:26"]
+                }
+             ]
+        },
+        {
+            "business_id": "-0aOudcaAyac0VJbMX-L1g",
+            "checkin_times": [ 
+                {
+                    "dates": ["2015-03-16", "2015-12-21"],
+                    "times": ["23:51:16", "04:48:01"]
+                },
+                {
+                    "dates": ["2016-10-28", "2016-10-28"],
+                    "times": ["20:22:42", "20:23:00"]
+                }
+             ]
+        },
+        {
+            "business_id": "-0b86isaXMY0v4g-V8GZ9Q",
+            "checkin_times": [ 
+                {
+                    "dates": ["2013-10-22"],
+                    "times": ["16:49:21"]
+                },
+                {
+                    "dates": ["2014-11-21"],
+                    "times": ["17:39:24"]
+                }
+             ]
+        },
+        {
+            "business_id": "-0d-BfFSU0bwLcnMaGRxYw",
+            "checkin_times": [ 
+                {
+                    "dates": ["2014-08-07", "2014-09-16", "2014-10-12"],
+                    "times": ["18:30:48", "20:41:45", "23:22:27"]
+                },
+                {
+                    "dates": ["2015-07-21", "2015-07-21"],
+                    "times": ["20:43:56", "20:45:07"]
+                }
+             ]
+        },
+        {
+            "business_id": "-0jz6c3C6i7RG7Ag22K-Pg",
+            "checkin_times": [ 
+                {
+                    "dates": ["2015-05-02", "2015-05-06", "2015-09-26"],
+                    "times": ["19:49:05", "03:52:18", "01:13:19"]
+                }
+
+             ]
+        },
+        {
+            "business_id": "-0y3MZU2oYP8r1ruDP1bfQ",
+            "checkin_times": [ 
+                {
+                    "dates": ["2015-04-11", "2015-11-21"],
+                    "times": ["13:14:14", "16:05:56"]
+                },
+                {
+                    "dates": ["2016-05-06"],
+                    "times": ["14:10:04"]
+                },
+                {
+                    "dates": ["2017-08-09", "2017-10-21"],
+                    "times": ["15:15:10", "15:12:56"]
+                }
+             ]
+        },
+        {
+            "business_id": "-1BPe8UjF2_l3nVk-DFUjA",
+            "checkin_times": [ 
+                {
+                    "dates": ["2015-12-03"],
+                    "times": ["18:44:00"]
+                },
+                {
+                    "dates": ["2016-03-17", "2016-11-02"],
+                    "times": ["18:19:21", "15:58:38"]
+                }
+             ]
+        },
+        {
+            "business_id": "-1E2CQu_38mkghvmZgCCRw",
+            "checkin_times": [ 
+                {
+                    "dates": ["2019-04-04"],
+                    "times": ["22:02:37"]
+                }
+             ]
+        },
+        {
+            "business_id": "-1wzk43IZ5D9Ysu6kzb5xA",
+            "checkin_times": [ 
+                {
+                    "dates": ["2019-02-27"],
+                    "times": ["14:03:08"]
+                }
+             ]
+        },
+        {
+            "business_id": "-23R9P2eG7VTc6DVLjFKzA",
+            "checkin_times": [ 
+                {
+                    "dates": ["2011-12-21"],
+                    "times": ["19:02:51"]
+                },
+                {
+                    "dates": ["2012-04-15", "2012-04-15"],
+                    "times": ["04:21:39", "14:23:56"]
+                },
+                {
+                    "dates": ["2013-06-30", "2013-10-04"],
+                    "times": ["22:39:51", "20:34:13"]
+                },
+                {
+                    "dates": ["2014-07-16"],
+                    "times": ["02:28:40"]
+                }
+             ]
+        },
+        {
+            "business_id": "-26MGfikhJiTfCI-GqmzhQ",
+            "checkin_times": [ 
+                {
+                    "dates": ["2018-06-13"],
+                    "times": ["20:16:07"]
+                }
+             ]
+        },
+        {
+            "business_id": "-2bLuJsMZ0WhI9daurVQNQ",
+            "checkin_times": [ 
+                {
+                    "dates": ["2015-05-29", "2015-06-01"],
+                    "times": ["16:46:17", "15:03:53"]
+                }
+             ]
+        },
+        {
+            "business_id": "-2hDBMaza_ldqnZdiU06LQ",
+            "checkin_times": [ 
+                {
+                    "dates": ["2011-10-08"],
+                    "times": ["12:02:23"]
+                },
+                {
+                    "dates": ["2014-08-18"],
+                    "times": ["02:11:11"]
+                },
+                {
+                    "dates": ["2016-01-07", "2016-10-21", "2016-12-01", "2016-12-29"],
+                    "times": ["05:27:51", "20:15:55", "03:57:10", "01:54:42"]
+                },
+                {
+                    "dates": ["2018-07-22", "2018-09-07"],
+                    "times": ["19:55:31", "01:42:54"]
+                },
+                {
+                    "dates": ["2019-03-08"],
+                    "times": ["03:41:06"]
+                }
+             ]
+        }
+    ]
+);
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/select-unnest-queries/open/use-case-4/use-case-4.3.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/select-unnest-queries/open/use-case-4/use-case-4.3.ddl.sqlpp
new file mode 100644
index 0000000..b349a8a
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/select-unnest-queries/open/use-case-4/use-case-4.3.ddl.sqlpp
@@ -0,0 +1,22 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+USE TestYelp;
+
+CREATE INDEX IdxYelpCheckinDates ON YelpCheckin (UNNEST checkin_times UNNEST dates : string ?) ;
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/select-unnest-queries/open/use-case-4/use-case-4.4.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/select-unnest-queries/open/use-case-4/use-case-4.4.query.sqlpp
new file mode 100644
index 0000000..cd5b2c8
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/select-unnest-queries/open/use-case-4/use-case-4.4.query.sqlpp
@@ -0,0 +1,26 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+SET `compiler.arrayindex` "true";
+
+USE TestYelp;
+
+SELECT C.business_id
+FROM YelpCheckin C, C.checkin_times CT, CT.dates D
+WHERE "2016-04-26" = D;
+
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/select-unnest-queries/open/use-case-4/use-case-4.5.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/select-unnest-queries/open/use-case-4/use-case-4.5.query.sqlpp
new file mode 100644
index 0000000..2f858dc
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/select-unnest-queries/open/use-case-4/use-case-4.5.query.sqlpp
@@ -0,0 +1,25 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+SET `compiler.arrayindex` "true";
+
+USE TestYelp;
+
+SELECT COUNT(*)
+FROM YelpCheckin C, C.checkin_times CT, CT.dates D
+WHERE D > "2016" AND D < "2017";
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/select-unnest-queries/open/with-3-level-record-path/with-3-level-record-path.1.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/select-unnest-queries/open/with-3-level-record-path/with-3-level-record-path.1.ddl.sqlpp
new file mode 100644
index 0000000..a6e46aa
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/select-unnest-queries/open/with-3-level-record-path/with-3-level-record-path.1.ddl.sqlpp
@@ -0,0 +1,35 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+/*
+ * Description: Verify that we are able to correctly perform the following types of queries using a record->record->record->array->atomic index.
+ *     1) UNNEST query, performing a point search.
+ *     2) UNNEST query, performing a range search.
+ */
+
+
+DROP DATAVERSE TestYelp IF EXISTS;
+CREATE DATAVERSE TestYelp;
+USE TestYelp;
+
+CREATE TYPE CheckinType AS {
+    checkin_id: uuid
+};
+
+CREATE DATASET YelpCheckin(CheckinType) PRIMARY KEY checkin_id AUTOGENERATED;
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/select-unnest-queries/open/with-3-level-record-path/with-3-level-record-path.2.update.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/select-unnest-queries/open/with-3-level-record-path/with-3-level-record-path.2.update.sqlpp
new file mode 100644
index 0000000..884c224
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/select-unnest-queries/open/with-3-level-record-path/with-3-level-record-path.2.update.sqlpp
@@ -0,0 +1,617 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+USE TestYelp;
+
+INSERT INTO YelpCheckin (
+    [
+        {
+            "business_id": "--1UhMGODdWsrMastO9DZw",
+            "checkin_data": {
+                "checkin_temporal": {
+                    "checkin_times": {
+                        "dates": [
+                            "2016-04-26",
+                            "2016-08-30",
+                            "2016-10-15",
+                            "2016-11-18",
+                            "2017-04-20",
+                            "2017-05-03",
+                            "2019-03-19"
+                        ],
+                        "times": [
+                            "19:49:16",
+                            "18:36:57",
+                            "02:45:18",
+                            "01:54:50",
+                            "18:39:06",
+                            "17:58:02",
+                            "22:04:48"
+                        ]
+                    }
+                }
+            }
+        },
+        {
+            "business_id": "--EF5N7P70J_UYBTPypYlA",
+            "checkin_data": {
+                "checkin_temporal": {
+                    "checkin_times": {
+                        "dates": [
+                            "2018-05-25",
+                            "2018-09-18",
+                            "2019-10-18"
+                        ],
+                        "times": [
+                            "19:52:07",
+                            "16:09:44",
+                            "21:29:09"
+                        ]
+                    }
+                }
+            }
+        },
+        {
+            "business_id": "--Ni3oJ4VOqfOEu7Sj2Vzg",
+            "checkin_data": {
+                "checkin_temporal": {
+                    "checkin_times": {
+                        "dates": [
+                            "2019-06-07"
+                        ],
+                        "times": [
+                            "17:54:58"
+                        ]
+                    }
+                }
+            }
+        },
+        {
+            "business_id": "--Y1Adl1YUWfYIRSd8vkmA",
+            "checkin_data": {
+                "checkin_temporal": {
+                    "checkin_times": {
+                        "dates": [
+                            "2011-05-03",
+                            "2011-08-23",
+                            "2014-12-04",
+                            "2016-11-16"
+                        ],
+                        "times": [
+                            "20:54:05",
+                            "20:49:45",
+                            "06:13:01",
+                            "19:25:55"
+                        ]
+                    }
+                }
+            }
+        },
+        {
+            "business_id": "--YPwqIlRJrhHkJcjY3eiA",
+            "checkin_data": {
+                "checkin_temporal": {
+                    "checkin_times": {
+                        "dates": [
+                            "2016-06-18",
+                            "2016-10-15"
+                        ],
+                        "times": [
+                            "21:35:45",
+                            "18:17:51"
+                        ]
+                    }
+                }
+            }
+        },
+        {
+            "business_id": "--e8PjCNhEz32pprnPhCwQ",
+            "checkin_data": {
+                "checkin_temporal": {
+                    "checkin_times": {
+                        "dates": [
+                            "2015-04-02"
+                        ],
+                        "times": [
+                            "21:45:17"
+                        ]
+                    }
+                }
+            }
+        },
+        {
+            "business_id": "--kinfHwmtdjz03g8B8z8Q",
+            "checkin_data": {
+                "checkin_temporal": {
+                    "checkin_times": {
+                        "dates": [
+                            "2014-08-27",
+                            "2015-12-19",
+                            "2018-11-27"
+                        ],
+                        "times": [
+                            "17:49:18",
+                            "21:30:31",
+                            "15:53:50"
+                        ]
+                    }
+                }
+            }
+        },
+        {
+            "business_id": "--q6datkI-f0EoVheXNEeQ",
+            "checkin_data": {
+                "checkin_temporal": {
+                    "checkin_times": {
+                        "dates": [
+                            "2014-01-28",
+                            "2014-11-16",
+                            "2015-11-15",
+                            "2015-11-15"
+                        ],
+                        "times": [
+                            "20:56:04",
+                            "16:11:58",
+                            "19:21:53",
+                            "19:33:39"
+                        ]
+                    }
+                }
+            }
+        },
+        {
+            "business_id": "--qvQS4MigHPykD2GV0-zw",
+            "checkin_data": {
+                "checkin_temporal": {
+                    "checkin_times": {
+                        "dates": [
+                            "2019-04-11"
+                        ],
+                        "times": [
+                            "18:30:12"
+                        ]
+                    }
+                }
+            }
+        },
+        {
+            "business_id": "--wIGbLEhlpl_UeAIyDmZQ",
+            "checkin_data": {
+                "checkin_temporal": {
+                    "checkin_times": {
+                        "dates": [
+                            "2015-06-06",
+                            "2019-03-14"
+                        ],
+                        "times": [
+                            "20:01:06",
+                            "22:01:52"
+                        ]
+                    }
+                }
+            }
+        },
+        {
+            "business_id": "-0FA-Qdi3SPYIoJz9UQw-A",
+            "checkin_data": {
+                "checkin_temporal": {
+                    "checkin_times": {
+                        "dates": [
+                            "2018-09-29",
+                            "2018-10-20",
+                            "2018-10-20"
+                        ],
+                        "times": [
+                            "18:55:17",
+                            "16:48:05",
+                            "22:20:24"
+                        ]
+                    }
+                }
+            }
+        },
+        {
+            "business_id": "-0Hj1hb_XW6ybWq2M7QhGA",
+            "checkin_data": {
+                "checkin_temporal": {
+                    "checkin_times": {
+                        "dates": [
+                            "2011-04-23",
+                            "2014-05-04",
+                            "2014-05-11",
+                            "2014-06-04",
+                            "2015-12-05",
+                            "2017-05-15"
+                        ],
+                        "times": [
+                            "21:11:22",
+                            "19:42:48",
+                            "19:16:08",
+                            "19:14:18",
+                            "19:22:42",
+                            "23:19:00"
+                        ]
+                    }
+                }
+            }
+        },
+        {
+            "business_id": "-0KMvRFwDWdVBeTpT11iHw",
+            "checkin_data": {
+                "checkin_temporal": {
+                    "checkin_times": {
+                        "dates": [
+                            "2012-07-13",
+                            "2016-12-24",
+                            "2017-08-31"
+                        ],
+                        "times": [
+                            "21:43:57",
+                            "02:27:31",
+                            "00:35:26"
+                        ]
+                    }
+                }
+            }
+        },
+        {
+            "business_id": "-0LPtgJC31FWMrMv317p0Q",
+            "checkin_data": {
+                "checkin_temporal": {
+                    "checkin_times": {
+                        "dates": [
+                            "2013-04-13",
+                            "2013-08-19",
+                            "2013-10-04"
+                        ],
+                        "times": [
+                            "12:35:33",
+                            "23:35:49",
+                            "19:14:56"
+                        ]
+                    }
+                }
+            }
+        },
+        {
+            "business_id": "-0M3o2uWBnQZwd3hmfEwuw",
+            "checkin_data": {
+                "checkin_temporal": {
+                    "checkin_times": {
+                        "dates": [
+                            "2016-09-10",
+                            "2018-09-08",
+                            "2019-09-13"
+                        ],
+                        "times": [
+                            "19:26:19",
+                            "14:15:37",
+                            "22:47:25"
+                        ]
+                    }
+                }
+            }
+        },
+        {
+            "business_id": "-0RRiWDtfnS16AKCtfvBZg",
+            "checkin_data": {
+                "checkin_temporal": {
+                    "checkin_times": {
+                        "dates": [
+                            "2017-05-19",
+                            "2017-05-19",
+                            "2017-08-28",
+                            "2017-09-20",
+                            "2017-10-01",
+                            "2017-10-01",
+                            "2017-12-27"
+                        ],
+                        "times": [
+                            "14:30:16",
+                            "14:30:25",
+                            "15:49:37",
+                            "20:19:51",
+                            "16:31:05",
+                            "16:56:27",
+                            "23:33:20"
+                        ]
+                    }
+                }
+            }
+        },
+        {
+            "business_id": "-0Soj75v-XoRcf2ERr8Bmg",
+            "checkin_data": {
+                "checkin_temporal": {
+                    "checkin_times": {
+                        "dates": [
+                            "2019-06-05"
+                        ],
+                        "times": [
+                            "18:22:49"
+                        ]
+                    }
+                }
+            }
+        },
+        {
+            "business_id": "-0ZumLlFjMh4ZW1z2nXGug",
+            "checkin_data": {
+                "checkin_temporal": {
+                    "checkin_times": {
+                        "dates": [
+                            "2011-09-24",
+                            "2014-03-10",
+                            "2015-05-27",
+                            "2015-08-29",
+                            "2018-03-16"
+                        ],
+                        "times": [
+                            "21:37:32",
+                            "20:20:07",
+                            "00:40:24",
+                            "17:58:15",
+                            "15:03:26"
+                        ]
+                    }
+                }
+            }
+        },
+        {
+            "business_id": "-0aOudcaAyac0VJbMX-L1g",
+            "checkin_data": {
+                "checkin_temporal": {
+                    "checkin_times": {
+                        "dates": [
+                            "2015-03-16",
+                            "2015-12-21",
+                            "2016-10-28",
+                            "2016-10-28"
+                        ],
+                        "times": [
+                            "23:51:16",
+                            "04:48:01",
+                            "20:22:42",
+                            "20:23:00"
+                        ]
+                    }
+                }
+            }
+        },
+        {
+            "business_id": "-0b86isaXMY0v4g-V8GZ9Q",
+            "checkin_data": {
+                "checkin_temporal": {
+                    "checkin_times": {
+                        "dates": [
+                            "2013-10-22",
+                            "2014-11-21"
+                        ],
+                        "times": [
+                            "16:49:21",
+                            "17:39:24"
+                        ]
+                    }
+                }
+            }
+        },
+        {
+            "business_id": "-0d-BfFSU0bwLcnMaGRxYw",
+            "checkin_data": {
+                "checkin_temporal": {
+                    "checkin_times": {
+                        "dates": [
+                            "2014-08-07",
+                            "2014-09-16",
+                            "2014-10-12",
+                            "2015-07-21",
+                            "2015-07-21"
+                        ],
+                        "times": [
+                            "18:30:48",
+                            "20:41:45",
+                            "23:22:27",
+                            "20:43:56",
+                            "20:45:07"
+                        ]
+                    }
+                }
+            }
+        },
+        {
+            "business_id": "-0jz6c3C6i7RG7Ag22K-Pg",
+            "checkin_data": {
+                "checkin_temporal": {
+                    "checkin_times": {
+                        "dates": [
+                            "2015-05-02",
+                            "2015-05-06",
+                            "2015-09-26"
+                        ],
+                        "times": [
+                            "19:49:05",
+                            "03:52:18",
+                            "01:13:19"
+                        ]
+                    }
+                }
+            }
+        },
+        {
+            "business_id": "-0y3MZU2oYP8r1ruDP1bfQ",
+            "checkin_data": {
+                "checkin_temporal": {
+                    "checkin_times": {
+                        "dates": [
+                            "2015-04-11",
+                            "2015-11-21",
+                            "2016-05-06",
+                            "2017-08-09",
+                            "2017-10-21"
+                        ],
+                        "times": [
+                            "13:14:14",
+                            "16:05:56",
+                            "14:10:04",
+                            "15:15:10",
+                            "15:12:56"
+                        ]
+                    }
+                }
+            }
+        },
+        {
+            "business_id": "-1BPe8UjF2_l3nVk-DFUjA",
+            "checkin_data": {
+                "checkin_temporal": {
+                    "checkin_times": {
+                        "dates": [
+                            "2015-12-03",
+                            "2016-03-17",
+                            "2016-11-02"
+                        ],
+                        "times": [
+                            "18:44:00",
+                            "18:19:21",
+                            "15:58:38"
+                        ]
+                    }
+                }
+            }
+        },
+        {
+            "business_id": "-1E2CQu_38mkghvmZgCCRw",
+            "checkin_data": {
+                "checkin_temporal": {
+                    "checkin_times": {
+                        "dates": [
+                            "2019-04-04"
+                        ],
+                        "times": [
+                            "22:02:37"
+                        ]
+                    }
+                }
+            }
+        },
+        {
+            "business_id": "-1wzk43IZ5D9Ysu6kzb5xA",
+            "checkin_data": {
+                "checkin_temporal": {
+                    "checkin_times": {
+                        "dates": [
+                            "2019-02-27"
+                        ],
+                        "times": [
+                            "14:03:08"
+                        ]
+                    }
+                }
+            }
+        },
+        {
+            "business_id": "-23R9P2eG7VTc6DVLjFKzA",
+            "checkin_data": {
+                "checkin_temporal": {
+                    "checkin_times": {
+                        "dates": [
+                            "2011-12-21",
+                            "2012-04-15",
+                            "2012-04-15",
+                            "2013-06-30",
+                            "2013-10-04",
+                            "2014-07-16"
+                        ],
+                        "times": [
+                            "19:02:51",
+                            "04:21:39",
+                            "14:23:56",
+                            "22:39:51",
+                            "20:34:13",
+                            "02:28:40"
+                        ]
+                    }
+                }
+            }
+        },
+        {
+            "business_id": "-26MGfikhJiTfCI-GqmzhQ",
+            "checkin_data": {
+                "checkin_temporal": {
+                    "checkin_times": {
+                        "dates": [
+                            "2018-06-13"
+                        ],
+                        "times": [
+                            "20:16:07"
+                        ]
+                    }
+                }
+            }
+        },
+        {
+            "business_id": "-2bLuJsMZ0WhI9daurVQNQ",
+            "checkin_data": {
+                "checkin_temporal": {
+                    "checkin_times": {
+                        "dates": [
+                            "2015-05-29",
+                            "2015-06-01"
+                        ],
+                        "times": [
+                            "16:46:17",
+                            "15:03:53"
+                        ]
+                    }
+                }
+            }
+        },
+        {
+            "business_id": "-2hDBMaza_ldqnZdiU06LQ",
+            "checkin_data": {
+                "checkin_temporal": {
+                    "checkin_times": {
+                        "dates": [
+                            "2011-10-08",
+                            "2014-08-18",
+                            "2016-01-07",
+                            "2016-10-21",
+                            "2016-12-01",
+                            "2016-12-29",
+                            "2018-07-22",
+                            "2018-09-07",
+                            "2019-03-08"
+                        ],
+                        "times": [
+                            "12:02:23",
+                            "02:11:11",
+                            "05:27:51",
+                            "20:15:55",
+                            "03:57:10",
+                            "01:54:42",
+                            "19:55:31",
+                            "01:42:54",
+                            "03:41:06"
+                        ]
+                    }
+                }
+            }
+        }
+    ]
+);
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/select-unnest-queries/open/with-3-level-record-path/with-3-level-record-path.3.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/select-unnest-queries/open/with-3-level-record-path/with-3-level-record-path.3.ddl.sqlpp
new file mode 100644
index 0000000..599552a
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/select-unnest-queries/open/with-3-level-record-path/with-3-level-record-path.3.ddl.sqlpp
@@ -0,0 +1,22 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+USE TestYelp;
+
+CREATE INDEX IdxYelpCheckinDates ON YelpCheckin (UNNEST checkin_data.checkin_temporal.checkin_times.dates : string ?) ;
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/select-unnest-queries/open/with-3-level-record-path/with-3-level-record-path.4.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/select-unnest-queries/open/with-3-level-record-path/with-3-level-record-path.4.query.sqlpp
new file mode 100644
index 0000000..cee1887
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/select-unnest-queries/open/with-3-level-record-path/with-3-level-record-path.4.query.sqlpp
@@ -0,0 +1,25 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+SET `compiler.arrayindex` "true";
+
+USE TestYelp;
+
+SELECT C.business_id
+FROM YelpCheckin C, C.checkin_data.checkin_temporal.checkin_times.dates D
+WHERE "2016-04-26" = D;
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/select-unnest-queries/open/with-3-level-record-path/with-3-level-record-path.5.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/select-unnest-queries/open/with-3-level-record-path/with-3-level-record-path.5.query.sqlpp
new file mode 100644
index 0000000..847e73f
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/select-unnest-queries/open/with-3-level-record-path/with-3-level-record-path.5.query.sqlpp
@@ -0,0 +1,25 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+SET `compiler.arrayindex` "true";
+
+USE TestYelp;
+
+SELECT COUNT(*)
+FROM YelpCheckin C, C.checkin_data.checkin_temporal.checkin_times.dates D
+WHERE D > "2016" AND D < "2017";
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/select-unnest-queries/open/with-composite-sk/with-composite-sk.1.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/select-unnest-queries/open/with-composite-sk/with-composite-sk.1.ddl.sqlpp
new file mode 100644
index 0000000..f3cae85
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/select-unnest-queries/open/with-composite-sk/with-composite-sk.1.ddl.sqlpp
@@ -0,0 +1,34 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+/*
+ * Description: Verify that we are able to correctly perform the following types of queries using a array->record->atomic index on both array fields + an atomic field.
+ *     1) UNNEST query, performing a composite point search.
+ *     2) UNNEST query, performing a range search.
+ */
+
+DROP DATAVERSE TestYelp IF EXISTS;
+CREATE DATAVERSE TestYelp;
+USE TestYelp;
+
+CREATE TYPE CheckinType AS {
+    checkin_id: uuid
+};
+
+CREATE DATASET YelpCheckin(CheckinType) PRIMARY KEY checkin_id AUTOGENERATED;
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/select-unnest-queries/open/with-composite-sk/with-composite-sk.2.update.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/select-unnest-queries/open/with-composite-sk/with-composite-sk.2.update.sqlpp
new file mode 100644
index 0000000..02ddf21
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/select-unnest-queries/open/with-composite-sk/with-composite-sk.2.update.sqlpp
@@ -0,0 +1,276 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+USE TestYelp;
+
+INSERT INTO YelpCheckin (
+    [
+        {
+            "business_id": "--1UhMGODdWsrMastO9DZw",
+            "checkin_times": [ 
+                {"date": "2016-04-26", "time": "19:49:16"},
+                {"date": "2016-08-30", "time": "18:36:57"},
+                {"date": "2016-10-15", "time": "02:45:18"},
+                {"date": "2016-11-18", "time": "01:54:50"},
+                {"date": "2017-04-20", "time": "18:39:06"},
+                {"date": "2017-05-03", "time": "17:58:02"},
+                {"date": "2019-03-19", "time": "22:04:48"}
+             ]
+        },
+        {
+            "business_id": "--EF5N7P70J_UYBTPypYlA",
+            "checkin_times": [ 
+                {"date": "2018-05-25", "time": "19:52:07"},
+                {"date": "2018-09-18", "time": "16:09:44"},
+                {"date": "2019-10-18", "time": "21:29:09"}
+             ]
+        },
+        {
+            "business_id": "--Ni3oJ4VOqfOEu7Sj2Vzg",
+            "checkin_times": [ 
+                {"date": "2019-06-07", "time": "17:54:58"}
+             ]
+        },
+        {
+            "business_id": "--Y1Adl1YUWfYIRSd8vkmA",
+            "checkin_times": [ 
+                {"date": "2011-05-03", "time": "20:54:05"},
+                {"date": "2011-08-23", "time": "20:49:45"},
+                {"date": "2014-12-04", "time": "06:13:01"},
+                {"date": "2016-11-16", "time": "19:25:55"}
+             ]
+        },
+        {
+            "business_id": "--YPwqIlRJrhHkJcjY3eiA",
+            "checkin_times": [ 
+                {"date": "2016-06-18", "time": "21:35:45"},
+                {"date": "2016-10-15", "time": "18:17:51"}
+             ]
+        },
+        {
+            "business_id": "--e8PjCNhEz32pprnPhCwQ",
+            "checkin_times": [ 
+                {"date": "2015-04-02", "time": "21:45:17"}
+             ]
+        },
+        {
+            "business_id": "--kinfHwmtdjz03g8B8z8Q",
+            "checkin_times": [ 
+                {"date": "2014-08-27", "time": "17:49:18"},
+                {"date": "2015-12-19", "time": "21:30:31"},
+                {"date": "2018-11-27", "time": "15:53:50"}
+             ]
+        },
+        {
+            "business_id": "--q6datkI-f0EoVheXNEeQ",
+            "checkin_times": [ 
+                {"date": "2014-01-28", "time": "20:56:04"},
+                {"date": "2014-11-16", "time": "16:11:58"},
+                {"date": "2015-11-15", "time": "19:21:53"},
+                {"date": "2015-11-15", "time": "19:33:39"}
+             ]
+        },
+        {
+            "business_id": "--qvQS4MigHPykD2GV0-zw",
+            "checkin_times": [ 
+                {"date": "2019-04-11", "time": "18:30:12"}
+             ]
+        },
+        {
+            "business_id": "--wIGbLEhlpl_UeAIyDmZQ",
+            "checkin_times": [ 
+                {"date": "2015-06-06", "time": "20:01:06"},
+                {"date": "2019-03-14", "time": "22:01:52"}
+             ]
+        },
+        {
+            "business_id": "-0FA-Qdi3SPYIoJz9UQw-A",
+            "checkin_times": [ 
+                {"date": "2018-09-29", "time": "18:55:17"},
+                {"date": "2018-10-20", "time": "16:48:05"},
+                {"date": "2018-10-20", "time": "22:20:24"}
+             ]
+        },
+        {
+            "business_id": "-0Hj1hb_XW6ybWq2M7QhGA",
+            "checkin_times": [ 
+                {"date": "2011-04-23", "time": "21:11:22"},
+                {"date": "2014-05-04", "time": "19:42:48"},
+                {"date": "2014-05-11", "time": "19:16:08"},
+                {"date": "2014-06-04", "time": "19:14:18"},
+                {"date": "2015-12-05", "time": "19:22:42"},
+                {"date": "2017-05-15", "time": "23:19:00"}
+             ]
+        },
+        {
+            "business_id": "-0KMvRFwDWdVBeTpT11iHw",
+            "checkin_times": [ 
+                {"date": "2012-07-13", "time": "21:43:57"},
+                {"date": "2016-12-24", "time": "02:27:31"},
+                {"date": "2017-08-31", "time": "00:35:26"}
+             ]
+        },
+        {
+            "business_id": "-0LPtgJC31FWMrMv317p0Q",
+            "checkin_times": [ 
+                {"date": "2013-04-13", "time": "12:35:33"},
+                {"date": "2013-08-19", "time": "23:35:49"},
+                {"date": "2013-10-04", "time": "19:14:56"}
+             ]
+        },
+        {
+            "business_id": "-0M3o2uWBnQZwd3hmfEwuw",
+            "checkin_times": [ 
+                {"date": "2016-09-10", "time": "19:26:19"},
+                {"date": "2018-09-08", "time": "14:15:37"},
+                {"date": "2019-09-13", "time": "22:47:25"}
+             ]
+        },
+        {
+            "business_id": "-0RRiWDtfnS16AKCtfvBZg",
+            "checkin_times": [ 
+                {"date": "2017-05-19", "time": "14:30:16"},
+                {"date": "2017-05-19", "time": "14:30:25"},
+                {"date": "2017-08-28", "time": "15:49:37"},
+                {"date": "2017-09-20", "time": "20:19:51"},
+                {"date": "2017-10-01", "time": "16:31:05"},
+                {"date": "2017-10-01", "time": "16:56:27"},
+                {"date": "2017-12-27", "time": "23:33:20"}
+             ]
+        },
+        {
+            "business_id": "-0Soj75v-XoRcf2ERr8Bmg",
+            "checkin_times": [ 
+                {"date": "2019-06-05", "time": "18:22:49"}
+             ]
+        },
+        {
+            "business_id": "-0ZumLlFjMh4ZW1z2nXGug",
+            "checkin_times": [ 
+                {"date": "2011-09-24", "time": "21:37:32"},
+                {"date": "2014-03-10", "time": "20:20:07"},
+                {"date": "2015-05-27", "time": "00:40:24"},
+                {"date": "2015-08-29", "time": "17:58:15"},
+                {"date": "2018-03-16", "time": "15:03:26"}
+             ]
+        },
+        {
+            "business_id": "-0aOudcaAyac0VJbMX-L1g",
+            "checkin_times": [ 
+                {"date": "2015-03-16", "time": "23:51:16"},
+                {"date": "2015-12-21", "time": "04:48:01"},
+                {"date": "2016-10-28", "time": "20:22:42"},
+                {"date": "2016-10-28", "time": "20:23:00"}
+             ]
+        },
+        {
+            "business_id": "-0b86isaXMY0v4g-V8GZ9Q",
+            "checkin_times": [ 
+                {"date": "2013-10-22", "time": "16:49:21"},
+                {"date": "2014-11-21", "time": "17:39:24"}
+             ]
+        },
+        {
+            "business_id": "-0d-BfFSU0bwLcnMaGRxYw",
+            "checkin_times": [ 
+                {"date": "2014-08-07", "time": "18:30:48"},
+                {"date": "2014-09-16", "time": "20:41:45"},
+                {"date": "2014-10-12", "time": "23:22:27"},
+                {"date": "2015-07-21", "time": "20:43:56"},
+                {"date": "2015-07-21", "time": "20:45:07"}
+             ]
+        },
+        {
+            "business_id": "-0jz6c3C6i7RG7Ag22K-Pg",
+            "checkin_times": [ 
+                {"date": "2015-05-02", "time": "19:49:05"},
+                {"date": "2015-05-06", "time": "03:52:18"},
+                {"date": "2015-09-26", "time": "01:13:19"}
+             ]
+        },
+        {
+            "business_id": "-0y3MZU2oYP8r1ruDP1bfQ",
+            "checkin_times": [ 
+                {"date": "2015-04-11", "time": "13:14:14"},
+                {"date": "2015-11-21", "time": "16:05:56"},
+                {"date": "2016-05-06", "time": "14:10:04"},
+                {"date": "2017-08-09", "time": "15:15:10"},
+                {"date": "2017-10-21", "time": "15:12:56"}
+             ]
+        },
+        {
+            "business_id": "-1BPe8UjF2_l3nVk-DFUjA",
+            "checkin_times": [ 
+                {"date": "2015-12-03", "time": "18:44:00"},
+                {"date": "2016-03-17", "time": "18:19:21"},
+                {"date": "2016-11-02", "time": "15:58:38"}
+             ]
+        },
+        {
+            "business_id": "-1E2CQu_38mkghvmZgCCRw",
+            "checkin_times": [ 
+                {"date": "2019-04-04", "time": "22:02:37"}
+             ]
+        },
+        {
+            "business_id": "-1wzk43IZ5D9Ysu6kzb5xA",
+            "checkin_times": [ 
+                {"date": "2019-02-27", "time": "14:03:08"}
+             ]
+        },
+        {
+            "business_id": "-23R9P2eG7VTc6DVLjFKzA",
+            "checkin_times": [ 
+                {"date": "2011-12-21", "time": "19:02:51"},
+                {"date": "2012-04-15", "time": "04:21:39"},
+                {"date": "2012-04-15", "time": "14:23:56"},
+                {"date": "2013-06-30", "time": "22:39:51"},
+                {"date": "2013-10-04", "time": "20:34:13"},
+                {"date": "2014-07-16", "time": "02:28:40"}
+             ]
+        },
+        {
+            "business_id": "-26MGfikhJiTfCI-GqmzhQ",
+            "checkin_times": [ 
+                {"date": "2018-06-13", "time": "20:16:07"}
+             ]
+        },
+        {
+            "business_id": "-2bLuJsMZ0WhI9daurVQNQ",
+            "checkin_times": [ 
+                {"date": "2015-05-29", "time": "16:46:17"},
+                {"date": "2015-06-01", "time": "15:03:53"}
+             ]
+        },
+        {
+            "business_id": "-2hDBMaza_ldqnZdiU06LQ",
+            "checkin_times": [ 
+                {"date": "2011-10-08", "time": "12:02:23"},
+                {"date": "2014-08-18", "time": "02:11:11"},
+                {"date": "2016-01-07", "time": "05:27:51"},
+                {"date": "2016-10-21", "time": "20:15:55"},
+                {"date": "2016-12-01", "time": "03:57:10"},
+                {"date": "2016-12-29", "time": "01:54:42"},
+                {"date": "2018-07-22", "time": "19:55:31"},
+                {"date": "2018-09-07", "time": "01:42:54"},
+                {"date": "2019-03-08", "time": "03:41:06"}
+             ]
+        }
+    ]
+);
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/select-unnest-queries/open/with-composite-sk/with-composite-sk.3.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/select-unnest-queries/open/with-composite-sk/with-composite-sk.3.ddl.sqlpp
new file mode 100644
index 0000000..ab9a6f8
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/select-unnest-queries/open/with-composite-sk/with-composite-sk.3.ddl.sqlpp
@@ -0,0 +1,22 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+USE TestYelp;
+
+CREATE INDEX IdxYelpCheckinDatesTimesBusiness ON YelpCheckin (UNNEST checkin_times SELECT date : string ?, time : string ?);
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/select-unnest-queries/open/with-composite-sk/with-composite-sk.4.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/select-unnest-queries/open/with-composite-sk/with-composite-sk.4.query.sqlpp
new file mode 100644
index 0000000..3023de1
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/select-unnest-queries/open/with-composite-sk/with-composite-sk.4.query.sqlpp
@@ -0,0 +1,27 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+SET `compiler.arrayindex` "true";
+
+USE TestYelp;
+
+SELECT C.business_id
+FROM YelpCheckin C, C.checkin_times D
+WHERE "2016-04-26" = D.date AND 
+      "19:49:16" = D.time AND
+      "--1UhMGODdWsrMastO9DZw" = C.business_id;
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/select-unnest-queries/open/with-composite-sk/with-composite-sk.5.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/select-unnest-queries/open/with-composite-sk/with-composite-sk.5.query.sqlpp
new file mode 100644
index 0000000..244ae62
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/select-unnest-queries/open/with-composite-sk/with-composite-sk.5.query.sqlpp
@@ -0,0 +1,25 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+SET `compiler.arrayindex` "true";
+
+USE TestYelp;
+
+SELECT COUNT(*)
+FROM YelpCheckin C, C.checkin_times D
+WHERE D.date > "2016" AND D.date < "2017";
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/api/cluster_state_1/cluster_state_1.1.regexadm b/asterixdb/asterix-app/src/test/resources/runtimets/results/api/cluster_state_1/cluster_state_1.1.regexadm
index b6a49a6..0df55e5 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/results/api/cluster_state_1/cluster_state_1.1.regexadm
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/api/cluster_state_1/cluster_state_1.1.regexadm
@@ -8,6 +8,7 @@
     "active\.memory\.global\.budget" : 67108864,
     "active\.stop\.timeout" : 3600,
     "active\.suspend\.timeout" : 3600,
+    "compiler\.arrayindex" : false,
     "compiler\.external\.field\.pushdown" : false,
     "compiler\.framesize" : 32768,
     "compiler\.groupmemory" : 163840,
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/api/cluster_state_1_full/cluster_state_1_full.1.regexadm b/asterixdb/asterix-app/src/test/resources/runtimets/results/api/cluster_state_1_full/cluster_state_1_full.1.regexadm
index dedb40f..e00f4aa 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/results/api/cluster_state_1_full/cluster_state_1_full.1.regexadm
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/api/cluster_state_1_full/cluster_state_1_full.1.regexadm
@@ -8,6 +8,7 @@
     "active\.memory\.global\.budget" : 67108864,
     "active\.stop\.timeout" : 3600,
     "active\.suspend\.timeout" : 3600,
+    "compiler\.arrayindex" : false,
     "compiler\.external\.field\.pushdown" : false,
     "compiler\.framesize" : 32768,
     "compiler\.groupmemory" : 163840,
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/api/cluster_state_1_less/cluster_state_1_less.1.regexadm b/asterixdb/asterix-app/src/test/resources/runtimets/results/api/cluster_state_1_less/cluster_state_1_less.1.regexadm
index 985d3bd..c847562 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/results/api/cluster_state_1_less/cluster_state_1_less.1.regexadm
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/api/cluster_state_1_less/cluster_state_1_less.1.regexadm
@@ -8,6 +8,7 @@
     "active\.memory\.global\.budget" : 67108864,
     "active\.stop\.timeout" : 3600,
     "active\.suspend\.timeout" : 3600,
+    "compiler\.arrayindex" : false,
     "compiler\.external\.field\.pushdown" : false,
     "compiler\.framesize" : 32768,
     "compiler\.groupmemory" : 163840,
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/array-index/bulk-loading/after-index-creation/use-case-1/use-case-1.1.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/array-index/bulk-loading/after-index-creation/use-case-1/use-case-1.1.adm
new file mode 100644
index 0000000..6fa1293
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/array-index/bulk-loading/after-index-creation/use-case-1/use-case-1.1.adm
@@ -0,0 +1 @@
+{ "business_id": "--1UhMGODdWsrMastO9DZw" }
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/array-index/bulk-loading/after-index-creation/use-case-2/use-case-2.1.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/array-index/bulk-loading/after-index-creation/use-case-2/use-case-2.1.adm
new file mode 100644
index 0000000..6fa1293
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/array-index/bulk-loading/after-index-creation/use-case-2/use-case-2.1.adm
@@ -0,0 +1 @@
+{ "business_id": "--1UhMGODdWsrMastO9DZw" }
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/array-index/bulk-loading/after-index-creation/use-case-3/use-case-3.1.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/array-index/bulk-loading/after-index-creation/use-case-3/use-case-3.1.adm
new file mode 100644
index 0000000..6fa1293
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/array-index/bulk-loading/after-index-creation/use-case-3/use-case-3.1.adm
@@ -0,0 +1 @@
+{ "business_id": "--1UhMGODdWsrMastO9DZw" }
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/array-index/bulk-loading/after-index-creation/use-case-4/use-case-4.1.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/array-index/bulk-loading/after-index-creation/use-case-4/use-case-4.1.adm
new file mode 100644
index 0000000..6fa1293
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/array-index/bulk-loading/after-index-creation/use-case-4/use-case-4.1.adm
@@ -0,0 +1 @@
+{ "business_id": "--1UhMGODdWsrMastO9DZw" }
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/array-index/bulk-loading/after-index-creation/with-3-level-record-path/with-3-level-record-path.1.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/array-index/bulk-loading/after-index-creation/with-3-level-record-path/with-3-level-record-path.1.adm
new file mode 100644
index 0000000..6fa1293
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/array-index/bulk-loading/after-index-creation/with-3-level-record-path/with-3-level-record-path.1.adm
@@ -0,0 +1 @@
+{ "business_id": "--1UhMGODdWsrMastO9DZw" }
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/array-index/bulk-loading/after-index-creation/with-composite-pk/with-composite-pk.1.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/array-index/bulk-loading/after-index-creation/with-composite-pk/with-composite-pk.1.adm
new file mode 100644
index 0000000..6fa1293
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/array-index/bulk-loading/after-index-creation/with-composite-pk/with-composite-pk.1.adm
@@ -0,0 +1 @@
+{ "business_id": "--1UhMGODdWsrMastO9DZw" }
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/array-index/bulk-loading/after-index-creation/with-filter-fields/with-filter-fields.1.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/array-index/bulk-loading/after-index-creation/with-filter-fields/with-filter-fields.1.adm
new file mode 100644
index 0000000..6fa1293
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/array-index/bulk-loading/after-index-creation/with-filter-fields/with-filter-fields.1.adm
@@ -0,0 +1 @@
+{ "business_id": "--1UhMGODdWsrMastO9DZw" }
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/array-index/bulk-loading/after-index-creation/with-open-index/with-open-index.1.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/array-index/bulk-loading/after-index-creation/with-open-index/with-open-index.1.adm
new file mode 100644
index 0000000..6fa1293
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/array-index/bulk-loading/after-index-creation/with-open-index/with-open-index.1.adm
@@ -0,0 +1 @@
+{ "business_id": "--1UhMGODdWsrMastO9DZw" }
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/array-index/bulk-loading/on-index-creation/closed/use-case-1/use-case-1.1.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/array-index/bulk-loading/on-index-creation/closed/use-case-1/use-case-1.1.adm
new file mode 100644
index 0000000..6fa1293
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/array-index/bulk-loading/on-index-creation/closed/use-case-1/use-case-1.1.adm
@@ -0,0 +1 @@
+{ "business_id": "--1UhMGODdWsrMastO9DZw" }
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/array-index/bulk-loading/on-index-creation/closed/use-case-2/use-case-2.1.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/array-index/bulk-loading/on-index-creation/closed/use-case-2/use-case-2.1.adm
new file mode 100644
index 0000000..6fa1293
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/array-index/bulk-loading/on-index-creation/closed/use-case-2/use-case-2.1.adm
@@ -0,0 +1 @@
+{ "business_id": "--1UhMGODdWsrMastO9DZw" }
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/array-index/bulk-loading/on-index-creation/closed/use-case-3/use-case-3.1.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/array-index/bulk-loading/on-index-creation/closed/use-case-3/use-case-3.1.adm
new file mode 100644
index 0000000..6fa1293
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/array-index/bulk-loading/on-index-creation/closed/use-case-3/use-case-3.1.adm
@@ -0,0 +1 @@
+{ "business_id": "--1UhMGODdWsrMastO9DZw" }
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/array-index/bulk-loading/on-index-creation/closed/use-case-3/use-case-3.2.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/array-index/bulk-loading/on-index-creation/closed/use-case-3/use-case-3.2.adm
new file mode 100644
index 0000000..6fa1293
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/array-index/bulk-loading/on-index-creation/closed/use-case-3/use-case-3.2.adm
@@ -0,0 +1 @@
+{ "business_id": "--1UhMGODdWsrMastO9DZw" }
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/array-index/bulk-loading/on-index-creation/closed/use-case-4/use-case-4.1.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/array-index/bulk-loading/on-index-creation/closed/use-case-4/use-case-4.1.adm
new file mode 100644
index 0000000..6fa1293
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/array-index/bulk-loading/on-index-creation/closed/use-case-4/use-case-4.1.adm
@@ -0,0 +1 @@
+{ "business_id": "--1UhMGODdWsrMastO9DZw" }
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/array-index/bulk-loading/on-index-creation/closed/with-3-level-record-path/with-3-level-record-path.1.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/array-index/bulk-loading/on-index-creation/closed/with-3-level-record-path/with-3-level-record-path.1.adm
new file mode 100644
index 0000000..6fa1293
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/array-index/bulk-loading/on-index-creation/closed/with-3-level-record-path/with-3-level-record-path.1.adm
@@ -0,0 +1 @@
+{ "business_id": "--1UhMGODdWsrMastO9DZw" }
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/array-index/bulk-loading/on-index-creation/closed/with-composite-pk/with-composite-pk.1.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/array-index/bulk-loading/on-index-creation/closed/with-composite-pk/with-composite-pk.1.adm
new file mode 100644
index 0000000..6fa1293
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/array-index/bulk-loading/on-index-creation/closed/with-composite-pk/with-composite-pk.1.adm
@@ -0,0 +1 @@
+{ "business_id": "--1UhMGODdWsrMastO9DZw" }
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/array-index/bulk-loading/on-index-creation/closed/with-filter-fields/with-filter-fields.1.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/array-index/bulk-loading/on-index-creation/closed/with-filter-fields/with-filter-fields.1.adm
new file mode 100644
index 0000000..6fa1293
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/array-index/bulk-loading/on-index-creation/closed/with-filter-fields/with-filter-fields.1.adm
@@ -0,0 +1 @@
+{ "business_id": "--1UhMGODdWsrMastO9DZw" }
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/array-index/bulk-loading/on-index-creation/open/use-case-1/use-case-1.1.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/array-index/bulk-loading/on-index-creation/open/use-case-1/use-case-1.1.adm
new file mode 100644
index 0000000..6fa1293
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/array-index/bulk-loading/on-index-creation/open/use-case-1/use-case-1.1.adm
@@ -0,0 +1 @@
+{ "business_id": "--1UhMGODdWsrMastO9DZw" }
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/array-index/bulk-loading/on-index-creation/open/use-case-2/use-case-2.1.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/array-index/bulk-loading/on-index-creation/open/use-case-2/use-case-2.1.adm
new file mode 100644
index 0000000..6fa1293
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/array-index/bulk-loading/on-index-creation/open/use-case-2/use-case-2.1.adm
@@ -0,0 +1 @@
+{ "business_id": "--1UhMGODdWsrMastO9DZw" }
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/array-index/bulk-loading/on-index-creation/open/use-case-3/use-case-3.1.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/array-index/bulk-loading/on-index-creation/open/use-case-3/use-case-3.1.adm
new file mode 100644
index 0000000..6fa1293
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/array-index/bulk-loading/on-index-creation/open/use-case-3/use-case-3.1.adm
@@ -0,0 +1 @@
+{ "business_id": "--1UhMGODdWsrMastO9DZw" }
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/array-index/bulk-loading/on-index-creation/open/use-case-3/use-case-3.2.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/array-index/bulk-loading/on-index-creation/open/use-case-3/use-case-3.2.adm
new file mode 100644
index 0000000..6fa1293
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/array-index/bulk-loading/on-index-creation/open/use-case-3/use-case-3.2.adm
@@ -0,0 +1 @@
+{ "business_id": "--1UhMGODdWsrMastO9DZw" }
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/array-index/bulk-loading/on-index-creation/open/use-case-4/use-case-4.1.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/array-index/bulk-loading/on-index-creation/open/use-case-4/use-case-4.1.adm
new file mode 100644
index 0000000..6fa1293
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/array-index/bulk-loading/on-index-creation/open/use-case-4/use-case-4.1.adm
@@ -0,0 +1 @@
+{ "business_id": "--1UhMGODdWsrMastO9DZw" }
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/array-index/insert-upsert-delete/closed/use-case-1/use-case-1.1.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/array-index/insert-upsert-delete/closed/use-case-1/use-case-1.1.adm
new file mode 100644
index 0000000..fcc26c8
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/array-index/insert-upsert-delete/closed/use-case-1/use-case-1.1.adm
@@ -0,0 +1 @@
+{ "$1": 99 }
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/array-index/insert-upsert-delete/closed/use-case-1/use-case-1.2.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/array-index/insert-upsert-delete/closed/use-case-1/use-case-1.2.adm
new file mode 100644
index 0000000..3e1a847
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/array-index/insert-upsert-delete/closed/use-case-1/use-case-1.2.adm
@@ -0,0 +1 @@
+{ "$1": 7 }
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/array-index/insert-upsert-delete/closed/use-case-1/use-case-1.3.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/array-index/insert-upsert-delete/closed/use-case-1/use-case-1.3.adm
new file mode 100644
index 0000000..dc7ba8b
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/array-index/insert-upsert-delete/closed/use-case-1/use-case-1.3.adm
@@ -0,0 +1 @@
+{ "$1": 101 }
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/array-index/insert-upsert-delete/closed/use-case-2/use-case-2.1.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/array-index/insert-upsert-delete/closed/use-case-2/use-case-2.1.adm
new file mode 100644
index 0000000..dc7ba8b
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/array-index/insert-upsert-delete/closed/use-case-2/use-case-2.1.adm
@@ -0,0 +1 @@
+{ "$1": 101 }
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/array-index/insert-upsert-delete/closed/use-case-2/use-case-2.2.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/array-index/insert-upsert-delete/closed/use-case-2/use-case-2.2.adm
new file mode 100644
index 0000000..3e1a847
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/array-index/insert-upsert-delete/closed/use-case-2/use-case-2.2.adm
@@ -0,0 +1 @@
+{ "$1": 7 }
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/array-index/insert-upsert-delete/closed/use-case-2/use-case-2.3.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/array-index/insert-upsert-delete/closed/use-case-2/use-case-2.3.adm
new file mode 100644
index 0000000..dc7ba8b
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/array-index/insert-upsert-delete/closed/use-case-2/use-case-2.3.adm
@@ -0,0 +1 @@
+{ "$1": 101 }
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/array-index/insert-upsert-delete/closed/use-case-3/use-case-3.1.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/array-index/insert-upsert-delete/closed/use-case-3/use-case-3.1.adm
new file mode 100644
index 0000000..dc7ba8b
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/array-index/insert-upsert-delete/closed/use-case-3/use-case-3.1.adm
@@ -0,0 +1 @@
+{ "$1": 101 }
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/array-index/insert-upsert-delete/closed/use-case-3/use-case-3.2.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/array-index/insert-upsert-delete/closed/use-case-3/use-case-3.2.adm
new file mode 100644
index 0000000..3e1a847
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/array-index/insert-upsert-delete/closed/use-case-3/use-case-3.2.adm
@@ -0,0 +1 @@
+{ "$1": 7 }
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/array-index/insert-upsert-delete/closed/use-case-3/use-case-3.3.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/array-index/insert-upsert-delete/closed/use-case-3/use-case-3.3.adm
new file mode 100644
index 0000000..dc7ba8b
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/array-index/insert-upsert-delete/closed/use-case-3/use-case-3.3.adm
@@ -0,0 +1 @@
+{ "$1": 101 }
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/array-index/insert-upsert-delete/closed/use-case-4/use-case-4.1.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/array-index/insert-upsert-delete/closed/use-case-4/use-case-4.1.adm
new file mode 100644
index 0000000..dc7ba8b
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/array-index/insert-upsert-delete/closed/use-case-4/use-case-4.1.adm
@@ -0,0 +1 @@
+{ "$1": 101 }
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/array-index/insert-upsert-delete/closed/use-case-4/use-case-4.2.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/array-index/insert-upsert-delete/closed/use-case-4/use-case-4.2.adm
new file mode 100644
index 0000000..3e1a847
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/array-index/insert-upsert-delete/closed/use-case-4/use-case-4.2.adm
@@ -0,0 +1 @@
+{ "$1": 7 }
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/array-index/insert-upsert-delete/closed/use-case-4/use-case-4.3.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/array-index/insert-upsert-delete/closed/use-case-4/use-case-4.3.adm
new file mode 100644
index 0000000..dc7ba8b
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/array-index/insert-upsert-delete/closed/use-case-4/use-case-4.3.adm
@@ -0,0 +1 @@
+{ "$1": 101 }
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/array-index/insert-upsert-delete/closed/with-additional-atomic-index/with-additional-atomic-index.1.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/array-index/insert-upsert-delete/closed/with-additional-atomic-index/with-additional-atomic-index.1.adm
new file mode 100644
index 0000000..fcc26c8
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/array-index/insert-upsert-delete/closed/with-additional-atomic-index/with-additional-atomic-index.1.adm
@@ -0,0 +1 @@
+{ "$1": 99 }
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/array-index/insert-upsert-delete/closed/with-additional-atomic-index/with-additional-atomic-index.2.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/array-index/insert-upsert-delete/closed/with-additional-atomic-index/with-additional-atomic-index.2.adm
new file mode 100644
index 0000000..3e1a847
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/array-index/insert-upsert-delete/closed/with-additional-atomic-index/with-additional-atomic-index.2.adm
@@ -0,0 +1 @@
+{ "$1": 7 }
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/array-index/insert-upsert-delete/closed/with-additional-atomic-index/with-additional-atomic-index.3.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/array-index/insert-upsert-delete/closed/with-additional-atomic-index/with-additional-atomic-index.3.adm
new file mode 100644
index 0000000..dc7ba8b
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/array-index/insert-upsert-delete/closed/with-additional-atomic-index/with-additional-atomic-index.3.adm
@@ -0,0 +1 @@
+{ "$1": 101 }
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/array-index/insert-upsert-delete/closed/with-composite-sk/with-composite-sk.1.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/array-index/insert-upsert-delete/closed/with-composite-sk/with-composite-sk.1.adm
new file mode 100644
index 0000000..dc7ba8b
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/array-index/insert-upsert-delete/closed/with-composite-sk/with-composite-sk.1.adm
@@ -0,0 +1 @@
+{ "$1": 101 }
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/array-index/insert-upsert-delete/closed/with-composite-sk/with-composite-sk.2.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/array-index/insert-upsert-delete/closed/with-composite-sk/with-composite-sk.2.adm
new file mode 100644
index 0000000..3e1a847
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/array-index/insert-upsert-delete/closed/with-composite-sk/with-composite-sk.2.adm
@@ -0,0 +1 @@
+{ "$1": 7 }
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/array-index/insert-upsert-delete/closed/with-composite-sk/with-composite-sk.3.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/array-index/insert-upsert-delete/closed/with-composite-sk/with-composite-sk.3.adm
new file mode 100644
index 0000000..dc7ba8b
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/array-index/insert-upsert-delete/closed/with-composite-sk/with-composite-sk.3.adm
@@ -0,0 +1 @@
+{ "$1": 101 }
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/array-index/insert-upsert-delete/closed/with-filter-fields/with-filter-fields.1.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/array-index/insert-upsert-delete/closed/with-filter-fields/with-filter-fields.1.adm
new file mode 100644
index 0000000..dc7ba8b
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/array-index/insert-upsert-delete/closed/with-filter-fields/with-filter-fields.1.adm
@@ -0,0 +1 @@
+{ "$1": 101 }
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/array-index/insert-upsert-delete/closed/with-filter-fields/with-filter-fields.2.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/array-index/insert-upsert-delete/closed/with-filter-fields/with-filter-fields.2.adm
new file mode 100644
index 0000000..3e1a847
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/array-index/insert-upsert-delete/closed/with-filter-fields/with-filter-fields.2.adm
@@ -0,0 +1 @@
+{ "$1": 7 }
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/array-index/insert-upsert-delete/closed/with-filter-fields/with-filter-fields.3.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/array-index/insert-upsert-delete/closed/with-filter-fields/with-filter-fields.3.adm
new file mode 100644
index 0000000..dc7ba8b
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/array-index/insert-upsert-delete/closed/with-filter-fields/with-filter-fields.3.adm
@@ -0,0 +1 @@
+{ "$1": 101 }
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/array-index/insert-upsert-delete/open/use-case-1/use-case-1.1.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/array-index/insert-upsert-delete/open/use-case-1/use-case-1.1.adm
new file mode 100644
index 0000000..fcc26c8
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/array-index/insert-upsert-delete/open/use-case-1/use-case-1.1.adm
@@ -0,0 +1 @@
+{ "$1": 99 }
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/array-index/insert-upsert-delete/open/use-case-1/use-case-1.2.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/array-index/insert-upsert-delete/open/use-case-1/use-case-1.2.adm
new file mode 100644
index 0000000..3e1a847
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/array-index/insert-upsert-delete/open/use-case-1/use-case-1.2.adm
@@ -0,0 +1 @@
+{ "$1": 7 }
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/array-index/insert-upsert-delete/open/use-case-1/use-case-1.3.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/array-index/insert-upsert-delete/open/use-case-1/use-case-1.3.adm
new file mode 100644
index 0000000..dc7ba8b
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/array-index/insert-upsert-delete/open/use-case-1/use-case-1.3.adm
@@ -0,0 +1 @@
+{ "$1": 101 }
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/array-index/insert-upsert-delete/open/use-case-2/use-case-2.1.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/array-index/insert-upsert-delete/open/use-case-2/use-case-2.1.adm
new file mode 100644
index 0000000..dc7ba8b
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/array-index/insert-upsert-delete/open/use-case-2/use-case-2.1.adm
@@ -0,0 +1 @@
+{ "$1": 101 }
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/array-index/insert-upsert-delete/open/use-case-2/use-case-2.2.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/array-index/insert-upsert-delete/open/use-case-2/use-case-2.2.adm
new file mode 100644
index 0000000..3e1a847
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/array-index/insert-upsert-delete/open/use-case-2/use-case-2.2.adm
@@ -0,0 +1 @@
+{ "$1": 7 }
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/array-index/insert-upsert-delete/open/use-case-2/use-case-2.3.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/array-index/insert-upsert-delete/open/use-case-2/use-case-2.3.adm
new file mode 100644
index 0000000..dc7ba8b
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/array-index/insert-upsert-delete/open/use-case-2/use-case-2.3.adm
@@ -0,0 +1 @@
+{ "$1": 101 }
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/array-index/insert-upsert-delete/open/use-case-3/use-case-3.1.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/array-index/insert-upsert-delete/open/use-case-3/use-case-3.1.adm
new file mode 100644
index 0000000..dc7ba8b
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/array-index/insert-upsert-delete/open/use-case-3/use-case-3.1.adm
@@ -0,0 +1 @@
+{ "$1": 101 }
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/array-index/insert-upsert-delete/open/use-case-3/use-case-3.2.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/array-index/insert-upsert-delete/open/use-case-3/use-case-3.2.adm
new file mode 100644
index 0000000..3e1a847
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/array-index/insert-upsert-delete/open/use-case-3/use-case-3.2.adm
@@ -0,0 +1 @@
+{ "$1": 7 }
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/array-index/insert-upsert-delete/open/use-case-3/use-case-3.3.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/array-index/insert-upsert-delete/open/use-case-3/use-case-3.3.adm
new file mode 100644
index 0000000..dc7ba8b
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/array-index/insert-upsert-delete/open/use-case-3/use-case-3.3.adm
@@ -0,0 +1 @@
+{ "$1": 101 }
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/array-index/insert-upsert-delete/open/use-case-4/use-case-4.1.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/array-index/insert-upsert-delete/open/use-case-4/use-case-4.1.adm
new file mode 100644
index 0000000..dc7ba8b
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/array-index/insert-upsert-delete/open/use-case-4/use-case-4.1.adm
@@ -0,0 +1 @@
+{ "$1": 101 }
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/array-index/insert-upsert-delete/open/use-case-4/use-case-4.2.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/array-index/insert-upsert-delete/open/use-case-4/use-case-4.2.adm
new file mode 100644
index 0000000..3e1a847
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/array-index/insert-upsert-delete/open/use-case-4/use-case-4.2.adm
@@ -0,0 +1 @@
+{ "$1": 7 }
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/array-index/insert-upsert-delete/open/use-case-4/use-case-4.3.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/array-index/insert-upsert-delete/open/use-case-4/use-case-4.3.adm
new file mode 100644
index 0000000..dc7ba8b
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/array-index/insert-upsert-delete/open/use-case-4/use-case-4.3.adm
@@ -0,0 +1 @@
+{ "$1": 101 }
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/array-index/insert-upsert-delete/open/with-additional-atomic-index/with-additional-atomic-index.1.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/array-index/insert-upsert-delete/open/with-additional-atomic-index/with-additional-atomic-index.1.adm
new file mode 100644
index 0000000..fcc26c8
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/array-index/insert-upsert-delete/open/with-additional-atomic-index/with-additional-atomic-index.1.adm
@@ -0,0 +1 @@
+{ "$1": 99 }
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/array-index/insert-upsert-delete/open/with-additional-atomic-index/with-additional-atomic-index.2.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/array-index/insert-upsert-delete/open/with-additional-atomic-index/with-additional-atomic-index.2.adm
new file mode 100644
index 0000000..3e1a847
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/array-index/insert-upsert-delete/open/with-additional-atomic-index/with-additional-atomic-index.2.adm
@@ -0,0 +1 @@
+{ "$1": 7 }
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/array-index/insert-upsert-delete/open/with-additional-atomic-index/with-additional-atomic-index.3.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/array-index/insert-upsert-delete/open/with-additional-atomic-index/with-additional-atomic-index.3.adm
new file mode 100644
index 0000000..dc7ba8b
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/array-index/insert-upsert-delete/open/with-additional-atomic-index/with-additional-atomic-index.3.adm
@@ -0,0 +1 @@
+{ "$1": 101 }
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/array-index/insert-upsert-delete/open/with-composite-sk/with-composite-sk.1.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/array-index/insert-upsert-delete/open/with-composite-sk/with-composite-sk.1.adm
new file mode 100644
index 0000000..dc7ba8b
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/array-index/insert-upsert-delete/open/with-composite-sk/with-composite-sk.1.adm
@@ -0,0 +1 @@
+{ "$1": 101 }
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/array-index/insert-upsert-delete/open/with-composite-sk/with-composite-sk.2.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/array-index/insert-upsert-delete/open/with-composite-sk/with-composite-sk.2.adm
new file mode 100644
index 0000000..3e1a847
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/array-index/insert-upsert-delete/open/with-composite-sk/with-composite-sk.2.adm
@@ -0,0 +1 @@
+{ "$1": 7 }
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/array-index/insert-upsert-delete/open/with-composite-sk/with-composite-sk.3.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/array-index/insert-upsert-delete/open/with-composite-sk/with-composite-sk.3.adm
new file mode 100644
index 0000000..dc7ba8b
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/array-index/insert-upsert-delete/open/with-composite-sk/with-composite-sk.3.adm
@@ -0,0 +1 @@
+{ "$1": 101 }
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/array-index/join-unnest-queries/use-case-1/with-open-index.1.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/array-index/join-unnest-queries/use-case-1/with-open-index.1.adm
new file mode 100644
index 0000000..d274256
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/array-index/join-unnest-queries/use-case-1/with-open-index.1.adm
@@ -0,0 +1 @@
+{ "marker": "991ed2f7-3374-33a5-ee65-77f067848381" }
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/array-index/join-unnest-queries/use-case-1/with-open-index.2.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/array-index/join-unnest-queries/use-case-1/with-open-index.2.adm
new file mode 100644
index 0000000..5ba90eb
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/array-index/join-unnest-queries/use-case-1/with-open-index.2.adm
@@ -0,0 +1 @@
+{ "marker": "4fa1e250-7a68-3adb-04ec-6569c36882f5" }
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/array-index/join-unnest-queries/use-case-1/with-open-index.3.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/array-index/join-unnest-queries/use-case-1/with-open-index.3.adm
new file mode 100644
index 0000000..dc7ba8b
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/array-index/join-unnest-queries/use-case-1/with-open-index.3.adm
@@ -0,0 +1 @@
+{ "$1": 101 }
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/array-index/join-unnest-queries/use-case-2/use-case-2.1.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/array-index/join-unnest-queries/use-case-2/use-case-2.1.adm
new file mode 100644
index 0000000..d274256
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/array-index/join-unnest-queries/use-case-2/use-case-2.1.adm
@@ -0,0 +1 @@
+{ "marker": "991ed2f7-3374-33a5-ee65-77f067848381" }
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/array-index/join-unnest-queries/use-case-2/use-case-2.2.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/array-index/join-unnest-queries/use-case-2/use-case-2.2.adm
new file mode 100644
index 0000000..5ba90eb
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/array-index/join-unnest-queries/use-case-2/use-case-2.2.adm
@@ -0,0 +1 @@
+{ "marker": "4fa1e250-7a68-3adb-04ec-6569c36882f5" }
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/array-index/join-unnest-queries/use-case-2/use-case-2.3.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/array-index/join-unnest-queries/use-case-2/use-case-2.3.adm
new file mode 100644
index 0000000..965e258
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/array-index/join-unnest-queries/use-case-2/use-case-2.3.adm
@@ -0,0 +1 @@
+{ "$1": 119 }
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/array-index/join-unnest-queries/use-case-3/use-case-3.1.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/array-index/join-unnest-queries/use-case-3/use-case-3.1.adm
new file mode 100644
index 0000000..d274256
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/array-index/join-unnest-queries/use-case-3/use-case-3.1.adm
@@ -0,0 +1 @@
+{ "marker": "991ed2f7-3374-33a5-ee65-77f067848381" }
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/array-index/join-unnest-queries/use-case-3/use-case-3.2.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/array-index/join-unnest-queries/use-case-3/use-case-3.2.adm
new file mode 100644
index 0000000..5ba90eb
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/array-index/join-unnest-queries/use-case-3/use-case-3.2.adm
@@ -0,0 +1 @@
+{ "marker": "4fa1e250-7a68-3adb-04ec-6569c36882f5" }
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/array-index/join-unnest-queries/use-case-3/use-case-3.3.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/array-index/join-unnest-queries/use-case-3/use-case-3.3.adm
new file mode 100644
index 0000000..5ba90eb
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/array-index/join-unnest-queries/use-case-3/use-case-3.3.adm
@@ -0,0 +1 @@
+{ "marker": "4fa1e250-7a68-3adb-04ec-6569c36882f5" }
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/array-index/join-unnest-queries/use-case-3/use-case-3.4.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/array-index/join-unnest-queries/use-case-3/use-case-3.4.adm
new file mode 100644
index 0000000..965e258
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/array-index/join-unnest-queries/use-case-3/use-case-3.4.adm
@@ -0,0 +1 @@
+{ "$1": 119 }
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/array-index/join-unnest-queries/use-case-4/use-case-4.1.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/array-index/join-unnest-queries/use-case-4/use-case-4.1.adm
new file mode 100644
index 0000000..d274256
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/array-index/join-unnest-queries/use-case-4/use-case-4.1.adm
@@ -0,0 +1 @@
+{ "marker": "991ed2f7-3374-33a5-ee65-77f067848381" }
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/array-index/join-unnest-queries/use-case-4/use-case-4.2.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/array-index/join-unnest-queries/use-case-4/use-case-4.2.adm
new file mode 100644
index 0000000..d274256
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/array-index/join-unnest-queries/use-case-4/use-case-4.2.adm
@@ -0,0 +1 @@
+{ "marker": "991ed2f7-3374-33a5-ee65-77f067848381" }
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/array-index/join-unnest-queries/use-case-4/use-case-4.3.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/array-index/join-unnest-queries/use-case-4/use-case-4.3.adm
new file mode 100644
index 0000000..5ba90eb
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/array-index/join-unnest-queries/use-case-4/use-case-4.3.adm
@@ -0,0 +1 @@
+{ "marker": "4fa1e250-7a68-3adb-04ec-6569c36882f5" }
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/array-index/join-unnest-queries/use-case-4/use-case-4.4.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/array-index/join-unnest-queries/use-case-4/use-case-4.4.adm
new file mode 100644
index 0000000..965e258
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/array-index/join-unnest-queries/use-case-4/use-case-4.4.adm
@@ -0,0 +1 @@
+{ "$1": 119 }
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/array-index/join-unnest-queries/with-open-index/use-case-1.1.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/array-index/join-unnest-queries/with-open-index/use-case-1.1.adm
new file mode 100644
index 0000000..d274256
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/array-index/join-unnest-queries/with-open-index/use-case-1.1.adm
@@ -0,0 +1 @@
+{ "marker": "991ed2f7-3374-33a5-ee65-77f067848381" }
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/array-index/join-unnest-queries/with-open-index/use-case-1.2.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/array-index/join-unnest-queries/with-open-index/use-case-1.2.adm
new file mode 100644
index 0000000..5ba90eb
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/array-index/join-unnest-queries/with-open-index/use-case-1.2.adm
@@ -0,0 +1 @@
+{ "marker": "4fa1e250-7a68-3adb-04ec-6569c36882f5" }
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/array-index/join-unnest-queries/with-open-index/use-case-1.3.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/array-index/join-unnest-queries/with-open-index/use-case-1.3.adm
new file mode 100644
index 0000000..dc7ba8b
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/array-index/join-unnest-queries/with-open-index/use-case-1.3.adm
@@ -0,0 +1 @@
+{ "$1": 101 }
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/array-index/metadata/closed/use-case-1/use-case-1.1.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/array-index/metadata/closed/use-case-1/use-case-1.1.adm
new file mode 100644
index 0000000..27cc984
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/array-index/metadata/closed/use-case-1/use-case-1.1.adm
@@ -0,0 +1 @@
+{ "SearchKey": [  ], "SearchKeyElements": [ { "UnnestList": [ [ "dates" ] ], "ProjectList": [ null ] } ] }
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/array-index/metadata/closed/use-case-2/use-case-2.1.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/array-index/metadata/closed/use-case-2/use-case-2.1.adm
new file mode 100644
index 0000000..8f0aa97
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/array-index/metadata/closed/use-case-2/use-case-2.1.adm
@@ -0,0 +1 @@
+{ "SearchKey": [  ], "SearchKeyElements": [ { "UnnestList": [ [ "checkin_times", "dates" ] ], "ProjectList": [ null ] } ] }
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/array-index/metadata/closed/use-case-3/use-case-3.1.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/array-index/metadata/closed/use-case-3/use-case-3.1.adm
new file mode 100644
index 0000000..0b7a959
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/array-index/metadata/closed/use-case-3/use-case-3.1.adm
@@ -0,0 +1 @@
+{ "SearchKey": [  ], "SearchKeyElements": [ { "UnnestList": [ [ "checkin_times" ] ], "ProjectList": [ [ "date" ] ] } ] }
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/array-index/metadata/closed/use-case-3/use-case-3.2.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/array-index/metadata/closed/use-case-3/use-case-3.2.adm
new file mode 100644
index 0000000..36048c8
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/array-index/metadata/closed/use-case-3/use-case-3.2.adm
@@ -0,0 +1 @@
+{ "SearchKey": [  ], "SearchKeyElements": [ { "UnnestList": [ [ "checkin_times" ] ], "ProjectList": [ [ "date" ], [ "time" ] ] } ] }
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/array-index/metadata/closed/use-case-4/use-case-4.1.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/array-index/metadata/closed/use-case-4/use-case-4.1.adm
new file mode 100644
index 0000000..a04b36e
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/array-index/metadata/closed/use-case-4/use-case-4.1.adm
@@ -0,0 +1 @@
+{ "SearchKey": [  ], "SearchKeyElements": [ { "UnnestList": [ [ "checkin_times" ], [ "dates" ] ], "ProjectList": [ null ] } ] }
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/array-index/metadata/closed/with-3-level-record-path/with-3-level-record-path.1.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/array-index/metadata/closed/with-3-level-record-path/with-3-level-record-path.1.adm
new file mode 100644
index 0000000..27ae1aa
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/array-index/metadata/closed/with-3-level-record-path/with-3-level-record-path.1.adm
@@ -0,0 +1 @@
+{ "SearchKey": [  ], "SearchKeyElements": [ { "UnnestList": [ [ "checkin_data", "checkin_temporal", "checkin_times", "dates" ] ], "ProjectList": [ null ] } ] }
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/array-index/metadata/closed/with-composite-array-different-indicators/with-composite-array-different-indicators.1.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/array-index/metadata/closed/with-composite-array-different-indicators/with-composite-array-different-indicators.1.adm
new file mode 100644
index 0000000..85ce9aa
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/array-index/metadata/closed/with-composite-array-different-indicators/with-composite-array-different-indicators.1.adm
@@ -0,0 +1 @@
+{ "SearchKey": [  ], "SearchKeyElements": [ { "UnnestList": [ [ "checkin_times" ] ], "ProjectList": [ [ "date" ], [ "t", "time" ] ] } ] }
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/array-index/metadata/closed/with-composite-sk/with-composite-sk.1.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/array-index/metadata/closed/with-composite-sk/with-composite-sk.1.adm
new file mode 100644
index 0000000..cc5313c
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/array-index/metadata/closed/with-composite-sk/with-composite-sk.1.adm
@@ -0,0 +1 @@
+{ "SearchKey": [  ], "SearchKeyElements": [ [ [ "business_id" ] ], { "UnnestList": [ [ "dates" ] ], "ProjectList": [ null ] } ] }
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/array-index/metadata/closed/with-composite-sk/with-composite-sk.2.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/array-index/metadata/closed/with-composite-sk/with-composite-sk.2.adm
new file mode 100644
index 0000000..dcdc307
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/array-index/metadata/closed/with-composite-sk/with-composite-sk.2.adm
@@ -0,0 +1 @@
+{ "SearchKey": [  ], "SearchKeyElements": [ { "UnnestList": [ [ "dates" ] ], "ProjectList": [ null ] }, [ [ "business_id" ] ] ] }
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/array-index/metadata/open/use-case-1/use-case-1.1.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/array-index/metadata/open/use-case-1/use-case-1.1.adm
new file mode 100644
index 0000000..59b8bc5a
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/array-index/metadata/open/use-case-1/use-case-1.1.adm
@@ -0,0 +1 @@
+{ "SearchKey": [  ], "SearchKeyElements": [ { "UnnestList": [ [ "dates" ] ], "ProjectList": [ null ] } ], "SearchKeyType": [ [ "string" ] ] }
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/array-index/metadata/open/use-case-2/use-case-2.1.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/array-index/metadata/open/use-case-2/use-case-2.1.adm
new file mode 100644
index 0000000..daab11d
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/array-index/metadata/open/use-case-2/use-case-2.1.adm
@@ -0,0 +1 @@
+{ "SearchKey": [  ], "SearchKeyElements": [ { "UnnestList": [ [ "checkin_times", "dates" ] ], "ProjectList": [ null ] } ], "SearchKeyType": [ [ "string" ] ] }
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/array-index/metadata/open/use-case-3/use-case-3.1.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/array-index/metadata/open/use-case-3/use-case-3.1.adm
new file mode 100644
index 0000000..b33f547
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/array-index/metadata/open/use-case-3/use-case-3.1.adm
@@ -0,0 +1 @@
+{ "SearchKey": [  ], "SearchKeyElements": [ { "UnnestList": [ [ "checkin_times" ] ], "ProjectList": [ [ "date" ] ] } ], "SearchKeyType": [ [ "string" ] ] }
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/array-index/metadata/open/use-case-3/use-case-3.2.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/array-index/metadata/open/use-case-3/use-case-3.2.adm
new file mode 100644
index 0000000..194db94
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/array-index/metadata/open/use-case-3/use-case-3.2.adm
@@ -0,0 +1 @@
+{ "SearchKey": [  ], "SearchKeyElements": [ { "UnnestList": [ [ "checkin_times" ] ], "ProjectList": [ [ "time" ] ] } ], "SearchKeyType": [ [ "string" ] ] }
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/array-index/metadata/open/use-case-4/use-case-4.1.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/array-index/metadata/open/use-case-4/use-case-4.1.adm
new file mode 100644
index 0000000..099446c
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/array-index/metadata/open/use-case-4/use-case-4.1.adm
@@ -0,0 +1 @@
+{ "SearchKey": [  ], "SearchKeyElements": [ { "UnnestList": [ [ "checkin_times" ], [ "dates" ] ], "ProjectList": [ null ] } ], "SearchKeyType": [ [ "string" ] ] }
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/array-index/metadata/open/with-3-level-record-path/with-3-level-record-path.1.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/array-index/metadata/open/with-3-level-record-path/with-3-level-record-path.1.adm
new file mode 100644
index 0000000..4fafc3e
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/array-index/metadata/open/with-3-level-record-path/with-3-level-record-path.1.adm
@@ -0,0 +1 @@
+{ "SearchKey": [  ], "SearchKeyElements": [ { "UnnestList": [ [ "checkin_data", "checkin_temporal", "checkin_times", "dates" ] ], "ProjectList": [ null ] } ], "SearchKeyType": [ [ "string" ] ] }
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/array-index/metadata/open/with-composite-array-different-indicators/with-composite-array-different-indicators.1.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/array-index/metadata/open/with-composite-array-different-indicators/with-composite-array-different-indicators.1.adm
new file mode 100644
index 0000000..841d0bb
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/array-index/metadata/open/with-composite-array-different-indicators/with-composite-array-different-indicators.1.adm
@@ -0,0 +1 @@
+{ "SearchKey": [  ], "SearchKeyElements": [ { "UnnestList": [ [ "checkin_times" ] ], "ProjectList": [ [ "date" ], [ "t", "time" ] ] } ], "SearchKeyType": [ [ "string", "string" ] ] }
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/array-index/metadata/open/with-composite-sk/with-composite-sk.1.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/array-index/metadata/open/with-composite-sk/with-composite-sk.1.adm
new file mode 100644
index 0000000..9a94949
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/array-index/metadata/open/with-composite-sk/with-composite-sk.1.adm
@@ -0,0 +1 @@
+{ "SearchKey": [  ], "SearchKeyElements": [ [ [ "business_id" ] ], { "UnnestList": [ [ "dates" ] ], "ProjectList": [ null ] } ], "SearchKeyType": [ [ "string" ], [ "string" ] ] }
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/array-index/select-quantified-queries/use-case-1/use-case-1.1.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/array-index/select-quantified-queries/use-case-1/use-case-1.1.adm
new file mode 100644
index 0000000..22fc35e
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/array-index/select-quantified-queries/use-case-1/use-case-1.1.adm
@@ -0,0 +1 @@
+{ "business_id": "--1UhMGODdWsrMastO9DZw" }
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/array-index/select-quantified-queries/use-case-1/use-case-1.2.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/array-index/select-quantified-queries/use-case-1/use-case-1.2.adm
new file mode 100644
index 0000000..eb148f6
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/array-index/select-quantified-queries/use-case-1/use-case-1.2.adm
@@ -0,0 +1 @@
+{ "$1": 9 }
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/array-index/select-quantified-queries/use-case-1/use-case-1.3.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/array-index/select-quantified-queries/use-case-1/use-case-1.3.adm
new file mode 100644
index 0000000..7cc3573
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/array-index/select-quantified-queries/use-case-1/use-case-1.3.adm
@@ -0,0 +1 @@
+{ "$1": 1 }
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/array-index/select-quantified-queries/use-case-2/use-case-2.1.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/array-index/select-quantified-queries/use-case-2/use-case-2.1.adm
new file mode 100644
index 0000000..22fc35e
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/array-index/select-quantified-queries/use-case-2/use-case-2.1.adm
@@ -0,0 +1 @@
+{ "business_id": "--1UhMGODdWsrMastO9DZw" }
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/array-index/select-quantified-queries/use-case-2/use-case-2.2.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/array-index/select-quantified-queries/use-case-2/use-case-2.2.adm
new file mode 100644
index 0000000..eb148f6
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/array-index/select-quantified-queries/use-case-2/use-case-2.2.adm
@@ -0,0 +1 @@
+{ "$1": 9 }
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/array-index/select-quantified-queries/use-case-2/use-case-2.3.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/array-index/select-quantified-queries/use-case-2/use-case-2.3.adm
new file mode 100644
index 0000000..7cc3573
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/array-index/select-quantified-queries/use-case-2/use-case-2.3.adm
@@ -0,0 +1 @@
+{ "$1": 1 }
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/array-index/select-quantified-queries/use-case-3/use-case-3.1.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/array-index/select-quantified-queries/use-case-3/use-case-3.1.adm
new file mode 100644
index 0000000..22fc35e
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/array-index/select-quantified-queries/use-case-3/use-case-3.1.adm
@@ -0,0 +1 @@
+{ "business_id": "--1UhMGODdWsrMastO9DZw" }
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/array-index/select-quantified-queries/use-case-3/use-case-3.2.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/array-index/select-quantified-queries/use-case-3/use-case-3.2.adm
new file mode 100644
index 0000000..22fc35e
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/array-index/select-quantified-queries/use-case-3/use-case-3.2.adm
@@ -0,0 +1 @@
+{ "business_id": "--1UhMGODdWsrMastO9DZw" }
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/array-index/select-quantified-queries/use-case-3/use-case-3.3.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/array-index/select-quantified-queries/use-case-3/use-case-3.3.adm
new file mode 100644
index 0000000..d85b4cf
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/array-index/select-quantified-queries/use-case-3/use-case-3.3.adm
@@ -0,0 +1 @@
+{ "business_id": "--YPwqIlRJrhHkJcjY3eiA" }
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/array-index/select-quantified-queries/use-case-4/use-case-4.1.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/array-index/select-quantified-queries/use-case-4/use-case-4.1.adm
new file mode 100644
index 0000000..22fc35e
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/array-index/select-quantified-queries/use-case-4/use-case-4.1.adm
@@ -0,0 +1 @@
+{ "business_id": "--1UhMGODdWsrMastO9DZw" }
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/array-index/select-quantified-queries/use-case-4/use-case-4.2.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/array-index/select-quantified-queries/use-case-4/use-case-4.2.adm
new file mode 100644
index 0000000..22fc35e
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/array-index/select-quantified-queries/use-case-4/use-case-4.2.adm
@@ -0,0 +1 @@
+{ "business_id": "--1UhMGODdWsrMastO9DZw" }
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/array-index/select-quantified-queries/use-case-4/use-case-4.3.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/array-index/select-quantified-queries/use-case-4/use-case-4.3.adm
new file mode 100644
index 0000000..e81620a
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/array-index/select-quantified-queries/use-case-4/use-case-4.3.adm
@@ -0,0 +1 @@
+{ "business_id": "--Ni3oJ4VOqfOEu7Sj2Vzg" }
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/array-index/select-quantified-queries/use-case-4/use-case-4.4.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/array-index/select-quantified-queries/use-case-4/use-case-4.4.adm
new file mode 100644
index 0000000..e81620a
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/array-index/select-quantified-queries/use-case-4/use-case-4.4.adm
@@ -0,0 +1 @@
+{ "business_id": "--Ni3oJ4VOqfOEu7Sj2Vzg" }
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/array-index/select-quantified-queries/with-composite-pk/with-composite-pk.1.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/array-index/select-quantified-queries/with-composite-pk/with-composite-pk.1.adm
new file mode 100644
index 0000000..22fc35e
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/array-index/select-quantified-queries/with-composite-pk/with-composite-pk.1.adm
@@ -0,0 +1 @@
+{ "business_id": "--1UhMGODdWsrMastO9DZw" }
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/array-index/select-quantified-queries/with-composite-pk/with-composite-pk.2.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/array-index/select-quantified-queries/with-composite-pk/with-composite-pk.2.adm
new file mode 100644
index 0000000..eb148f6
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/array-index/select-quantified-queries/with-composite-pk/with-composite-pk.2.adm
@@ -0,0 +1 @@
+{ "$1": 9 }
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/array-index/select-quantified-queries/with-composite-pk/with-composite-pk.3.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/array-index/select-quantified-queries/with-composite-pk/with-composite-pk.3.adm
new file mode 100644
index 0000000..7cc3573
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/array-index/select-quantified-queries/with-composite-pk/with-composite-pk.3.adm
@@ -0,0 +1 @@
+{ "$1": 1 }
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/array-index/select-quantified-queries/with-open-index/with-open-index.1.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/array-index/select-quantified-queries/with-open-index/with-open-index.1.adm
new file mode 100644
index 0000000..22fc35e
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/array-index/select-quantified-queries/with-open-index/with-open-index.1.adm
@@ -0,0 +1 @@
+{ "business_id": "--1UhMGODdWsrMastO9DZw" }
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/array-index/select-quantified-queries/with-open-index/with-open-index.2.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/array-index/select-quantified-queries/with-open-index/with-open-index.2.adm
new file mode 100644
index 0000000..eb148f6
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/array-index/select-quantified-queries/with-open-index/with-open-index.2.adm
@@ -0,0 +1 @@
+{ "$1": 9 }
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/array-index/select-quantified-queries/with-open-index/with-open-index.3.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/array-index/select-quantified-queries/with-open-index/with-open-index.3.adm
new file mode 100644
index 0000000..7cc3573
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/array-index/select-quantified-queries/with-open-index/with-open-index.3.adm
@@ -0,0 +1 @@
+{ "$1": 1 }
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/array-index/select-unnest-queries/closed/use-case-1/use-case-1.1.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/array-index/select-unnest-queries/closed/use-case-1/use-case-1.1.adm
new file mode 100644
index 0000000..22fc35e
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/array-index/select-unnest-queries/closed/use-case-1/use-case-1.1.adm
@@ -0,0 +1 @@
+{ "business_id": "--1UhMGODdWsrMastO9DZw" }
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/array-index/select-unnest-queries/closed/use-case-1/use-case-1.2.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/array-index/select-unnest-queries/closed/use-case-1/use-case-1.2.adm
new file mode 100644
index 0000000..9687c35
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/array-index/select-unnest-queries/closed/use-case-1/use-case-1.2.adm
@@ -0,0 +1 @@
+{ "$1": 18 }
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/array-index/select-unnest-queries/closed/use-case-2/use-case-2.1.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/array-index/select-unnest-queries/closed/use-case-2/use-case-2.1.adm
new file mode 100644
index 0000000..22fc35e
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/array-index/select-unnest-queries/closed/use-case-2/use-case-2.1.adm
@@ -0,0 +1 @@
+{ "business_id": "--1UhMGODdWsrMastO9DZw" }
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/array-index/select-unnest-queries/closed/use-case-2/use-case-2.2.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/array-index/select-unnest-queries/closed/use-case-2/use-case-2.2.adm
new file mode 100644
index 0000000..9687c35
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/array-index/select-unnest-queries/closed/use-case-2/use-case-2.2.adm
@@ -0,0 +1 @@
+{ "$1": 18 }
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/array-index/select-unnest-queries/closed/use-case-3/use-case-3.1.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/array-index/select-unnest-queries/closed/use-case-3/use-case-3.1.adm
new file mode 100644
index 0000000..22fc35e
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/array-index/select-unnest-queries/closed/use-case-3/use-case-3.1.adm
@@ -0,0 +1 @@
+{ "business_id": "--1UhMGODdWsrMastO9DZw" }
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/array-index/select-unnest-queries/closed/use-case-3/use-case-3.2.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/array-index/select-unnest-queries/closed/use-case-3/use-case-3.2.adm
new file mode 100644
index 0000000..9687c35
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/array-index/select-unnest-queries/closed/use-case-3/use-case-3.2.adm
@@ -0,0 +1 @@
+{ "$1": 18 }
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/array-index/select-unnest-queries/closed/use-case-3/use-case-3.3.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/array-index/select-unnest-queries/closed/use-case-3/use-case-3.3.adm
new file mode 100644
index 0000000..22fc35e
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/array-index/select-unnest-queries/closed/use-case-3/use-case-3.3.adm
@@ -0,0 +1 @@
+{ "business_id": "--1UhMGODdWsrMastO9DZw" }
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/array-index/select-unnest-queries/closed/use-case-4/use-case-4.1.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/array-index/select-unnest-queries/closed/use-case-4/use-case-4.1.adm
new file mode 100644
index 0000000..22fc35e
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/array-index/select-unnest-queries/closed/use-case-4/use-case-4.1.adm
@@ -0,0 +1 @@
+{ "business_id": "--1UhMGODdWsrMastO9DZw" }
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/array-index/select-unnest-queries/closed/use-case-4/use-case-4.2.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/array-index/select-unnest-queries/closed/use-case-4/use-case-4.2.adm
new file mode 100644
index 0000000..9687c35
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/array-index/select-unnest-queries/closed/use-case-4/use-case-4.2.adm
@@ -0,0 +1 @@
+{ "$1": 18 }
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/array-index/select-unnest-queries/closed/with-3-level-record-path/with-3-level-record-path.1.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/array-index/select-unnest-queries/closed/with-3-level-record-path/with-3-level-record-path.1.adm
new file mode 100644
index 0000000..22fc35e
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/array-index/select-unnest-queries/closed/with-3-level-record-path/with-3-level-record-path.1.adm
@@ -0,0 +1 @@
+{ "business_id": "--1UhMGODdWsrMastO9DZw" }
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/array-index/select-unnest-queries/closed/with-3-level-record-path/with-3-level-record-path.2.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/array-index/select-unnest-queries/closed/with-3-level-record-path/with-3-level-record-path.2.adm
new file mode 100644
index 0000000..9687c35
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/array-index/select-unnest-queries/closed/with-3-level-record-path/with-3-level-record-path.2.adm
@@ -0,0 +1 @@
+{ "$1": 18 }
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/array-index/select-unnest-queries/closed/with-composite-pk/with-composite-pk.1.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/array-index/select-unnest-queries/closed/with-composite-pk/with-composite-pk.1.adm
new file mode 100644
index 0000000..22fc35e
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/array-index/select-unnest-queries/closed/with-composite-pk/with-composite-pk.1.adm
@@ -0,0 +1 @@
+{ "business_id": "--1UhMGODdWsrMastO9DZw" }
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/array-index/select-unnest-queries/closed/with-composite-pk/with-composite-pk.2.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/array-index/select-unnest-queries/closed/with-composite-pk/with-composite-pk.2.adm
new file mode 100644
index 0000000..9687c35
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/array-index/select-unnest-queries/closed/with-composite-pk/with-composite-pk.2.adm
@@ -0,0 +1 @@
+{ "$1": 18 }
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/array-index/select-unnest-queries/closed/with-composite-sk/with-composite-sk.1.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/array-index/select-unnest-queries/closed/with-composite-sk/with-composite-sk.1.adm
new file mode 100644
index 0000000..22fc35e
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/array-index/select-unnest-queries/closed/with-composite-sk/with-composite-sk.1.adm
@@ -0,0 +1 @@
+{ "business_id": "--1UhMGODdWsrMastO9DZw" }
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/array-index/select-unnest-queries/closed/with-composite-sk/with-composite-sk.2.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/array-index/select-unnest-queries/closed/with-composite-sk/with-composite-sk.2.adm
new file mode 100644
index 0000000..9687c35
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/array-index/select-unnest-queries/closed/with-composite-sk/with-composite-sk.2.adm
@@ -0,0 +1 @@
+{ "$1": 18 }
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/array-index/select-unnest-queries/closed/with-filter-fields/with-filter-fields.1.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/array-index/select-unnest-queries/closed/with-filter-fields/with-filter-fields.1.adm
new file mode 100644
index 0000000..22fc35e
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/array-index/select-unnest-queries/closed/with-filter-fields/with-filter-fields.1.adm
@@ -0,0 +1 @@
+{ "business_id": "--1UhMGODdWsrMastO9DZw" }
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/array-index/select-unnest-queries/closed/with-filter-fields/with-filter-fields.2.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/array-index/select-unnest-queries/closed/with-filter-fields/with-filter-fields.2.adm
new file mode 100644
index 0000000..9687c35
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/array-index/select-unnest-queries/closed/with-filter-fields/with-filter-fields.2.adm
@@ -0,0 +1 @@
+{ "$1": 18 }
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/array-index/select-unnest-queries/open/use-case-1/use-case-1.1.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/array-index/select-unnest-queries/open/use-case-1/use-case-1.1.adm
new file mode 100644
index 0000000..22fc35e
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/array-index/select-unnest-queries/open/use-case-1/use-case-1.1.adm
@@ -0,0 +1 @@
+{ "business_id": "--1UhMGODdWsrMastO9DZw" }
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/array-index/select-unnest-queries/open/use-case-1/use-case-1.2.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/array-index/select-unnest-queries/open/use-case-1/use-case-1.2.adm
new file mode 100644
index 0000000..9687c35
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/array-index/select-unnest-queries/open/use-case-1/use-case-1.2.adm
@@ -0,0 +1 @@
+{ "$1": 18 }
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/array-index/select-unnest-queries/open/use-case-2/use-case-2.1.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/array-index/select-unnest-queries/open/use-case-2/use-case-2.1.adm
new file mode 100644
index 0000000..22fc35e
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/array-index/select-unnest-queries/open/use-case-2/use-case-2.1.adm
@@ -0,0 +1 @@
+{ "business_id": "--1UhMGODdWsrMastO9DZw" }
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/array-index/select-unnest-queries/open/use-case-2/use-case-2.2.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/array-index/select-unnest-queries/open/use-case-2/use-case-2.2.adm
new file mode 100644
index 0000000..9687c35
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/array-index/select-unnest-queries/open/use-case-2/use-case-2.2.adm
@@ -0,0 +1 @@
+{ "$1": 18 }
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/array-index/select-unnest-queries/open/use-case-3/use-case-3.1.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/array-index/select-unnest-queries/open/use-case-3/use-case-3.1.adm
new file mode 100644
index 0000000..22fc35e
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/array-index/select-unnest-queries/open/use-case-3/use-case-3.1.adm
@@ -0,0 +1 @@
+{ "business_id": "--1UhMGODdWsrMastO9DZw" }
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/array-index/select-unnest-queries/open/use-case-3/use-case-3.2.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/array-index/select-unnest-queries/open/use-case-3/use-case-3.2.adm
new file mode 100644
index 0000000..9687c35
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/array-index/select-unnest-queries/open/use-case-3/use-case-3.2.adm
@@ -0,0 +1 @@
+{ "$1": 18 }
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/array-index/select-unnest-queries/open/use-case-3/use-case-3.3.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/array-index/select-unnest-queries/open/use-case-3/use-case-3.3.adm
new file mode 100644
index 0000000..22fc35e
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/array-index/select-unnest-queries/open/use-case-3/use-case-3.3.adm
@@ -0,0 +1 @@
+{ "business_id": "--1UhMGODdWsrMastO9DZw" }
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/array-index/select-unnest-queries/open/use-case-4/use-case-4.1.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/array-index/select-unnest-queries/open/use-case-4/use-case-4.1.adm
new file mode 100644
index 0000000..22fc35e
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/array-index/select-unnest-queries/open/use-case-4/use-case-4.1.adm
@@ -0,0 +1 @@
+{ "business_id": "--1UhMGODdWsrMastO9DZw" }
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/array-index/select-unnest-queries/open/use-case-4/use-case-4.2.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/array-index/select-unnest-queries/open/use-case-4/use-case-4.2.adm
new file mode 100644
index 0000000..9687c35
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/array-index/select-unnest-queries/open/use-case-4/use-case-4.2.adm
@@ -0,0 +1 @@
+{ "$1": 18 }
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/array-index/select-unnest-queries/open/with-3-level-record-path/with-3-level-record-path.1.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/array-index/select-unnest-queries/open/with-3-level-record-path/with-3-level-record-path.1.adm
new file mode 100644
index 0000000..22fc35e
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/array-index/select-unnest-queries/open/with-3-level-record-path/with-3-level-record-path.1.adm
@@ -0,0 +1 @@
+{ "business_id": "--1UhMGODdWsrMastO9DZw" }
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/array-index/select-unnest-queries/open/with-3-level-record-path/with-3-level-record-path.2.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/array-index/select-unnest-queries/open/with-3-level-record-path/with-3-level-record-path.2.adm
new file mode 100644
index 0000000..9687c35
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/array-index/select-unnest-queries/open/with-3-level-record-path/with-3-level-record-path.2.adm
@@ -0,0 +1 @@
+{ "$1": 18 }
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/array-index/select-unnest-queries/open/with-composite-sk/with-composite-sk.1.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/array-index/select-unnest-queries/open/with-composite-sk/with-composite-sk.1.adm
new file mode 100644
index 0000000..22fc35e
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/array-index/select-unnest-queries/open/with-composite-sk/with-composite-sk.1.adm
@@ -0,0 +1 @@
+{ "business_id": "--1UhMGODdWsrMastO9DZw" }
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/array-index/select-unnest-queries/open/with-composite-sk/with-composite-sk.2.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/array-index/select-unnest-queries/open/with-composite-sk/with-composite-sk.2.adm
new file mode 100644
index 0000000..9687c35
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/array-index/select-unnest-queries/open/with-composite-sk/with-composite-sk.2.adm
@@ -0,0 +1 @@
+{ "$1": 18 }
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/testsuite_sqlpp.xml b/asterixdb/asterix-app/src/test/resources/runtimets/testsuite_sqlpp.xml
index 82f9f77..2b4b983 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/testsuite_sqlpp.xml
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/testsuite_sqlpp.xml
@@ -4373,7 +4373,7 @@
       <compilation-unit name="load-with-autogenerated-no-field">
         <output-dir compare="Text">load-with-autogenerated-no-field</output-dir>
         <expected-error>ASX1014: Field "not_id" is not found</expected-error>
-        <expected-error>ASX1014: Field "not_id" is not found (in line 33, at column 1)</expected-error>
+        <expected-error>ASX1014: Field "not_id" is not found</expected-error>
       </compilation-unit>
     </test-case>
     <test-case FilePath="dml">
@@ -7003,7 +7003,7 @@
       <test-case FilePath="open-index-enforced/error-checking">
         <compilation-unit name="index-on-closed-type">
           <output-dir compare="Text">index-on-closed-type</output-dir>
-          <expected-error>Typed index on "[value]" field could be created only for open datatype</expected-error>
+          <expected-error>ASX1014: Field "value" is not found (in line 33, at column 34)</expected-error>
         </compilation-unit>
       </test-case>
       <test-case FilePath="open-index-enforced/error-checking">
@@ -7808,6 +7808,416 @@
       </compilation-unit>
     </test-case>
   </test-group>
+  <test-group name="array-index">
+    <test-group name="array-index/error-handling">
+      <test-case FilePath="array-index/error-handling">
+        <compilation-unit name="index-two-array-fields">
+          <output-dir compare="Text">index-two-array-fields</output-dir>
+          <expected-error>ASX1079: Compilation error: Cannot create composite index with multiple array fields using different arrays</expected-error>
+          <expected-error>ASX1079: Compilation error: Cannot create composite index with multiple array fields using different arrays</expected-error>
+          <source-location>false</source-location>
+        </compilation-unit>
+      </test-case>
+      <test-case FilePath="array-index/error-handling">
+        <compilation-unit name="invalid-array-path">
+          <output-dir compare="Text">invalid-array-path</output-dir>
+          <expected-error>ASX0037: Type mismatch: expected value of type array or multiset, but got the value of type CheckinType_checkin_time:</expected-error>
+          <expected-error>ASX0037: Type mismatch: expected value of type array or multiset, but got the value of type string</expected-error>
+          <source-location>false</source-location>
+        </compilation-unit>
+      </test-case>
+      <test-case FilePath="array-index/error-handling">
+        <compilation-unit name="index-on-closed-array">
+          <output-dir compare="Text">index-on-closed-array</output-dir>
+          <expected-error>ASX1014: Field "date" is not found</expected-error>
+          <source-location>false</source-location>
+        </compilation-unit>
+      </test-case>
+      <test-case FilePath="array-index/error-handling">
+        <compilation-unit name="index-with-enforced-type">
+          <output-dir compare="Text">index-with-enforced-type</output-dir>
+          <expected-error>ASX1140: Incompatible index type ARRAY</expected-error>
+          <source-location>false</source-location>
+        </compilation-unit>
+      </test-case>
+      <test-case FilePath="array-index/error-handling">
+        <compilation-unit name="index-mixed-composite">
+          <output-dir compare="Text">index-mixed-composite</output-dir>
+          <expected-error>ASX1140: Incompatible index type ARRAY</expected-error>
+          <source-location>false</source-location>
+        </compilation-unit>
+      </test-case>
+    </test-group>
+    <test-group name="array-index/metadata">
+      <test-case FilePath="array-index/metadata/closed">
+        <compilation-unit name="use-case-1">
+          <output-dir compare="Text">use-case-1</output-dir>
+        </compilation-unit>
+      </test-case>
+      <test-case FilePath="array-index/metadata/open">
+        <compilation-unit name="use-case-1">
+          <output-dir compare="Text">use-case-1</output-dir>
+        </compilation-unit>
+      </test-case>
+      <test-case FilePath="array-index/metadata/closed">
+        <compilation-unit name="use-case-2">
+          <output-dir compare="Text">use-case-2</output-dir>
+        </compilation-unit>
+      </test-case>
+      <test-case FilePath="array-index/metadata/open">
+        <compilation-unit name="use-case-2">
+          <output-dir compare="Text">use-case-2</output-dir>
+        </compilation-unit>
+      </test-case>
+      <test-case FilePath="array-index/metadata/closed">
+        <compilation-unit name="use-case-3">
+          <output-dir compare="Text">use-case-3</output-dir>
+        </compilation-unit>
+      </test-case>
+      <test-case FilePath="array-index/metadata/open">
+        <compilation-unit name="use-case-3">
+          <output-dir compare="Text">use-case-3</output-dir>
+        </compilation-unit>
+      </test-case>
+      <test-case FilePath="array-index/metadata/closed">
+        <compilation-unit name="use-case-4">
+          <output-dir compare="Text">use-case-4</output-dir>
+        </compilation-unit>
+      </test-case>
+      <test-case FilePath="array-index/metadata/open">
+        <compilation-unit name="use-case-4">
+          <output-dir compare="Text">use-case-4</output-dir>
+        </compilation-unit>
+      </test-case>
+      <!--      <test-case FilePath="array-index/metadata/closed">-->
+      <!--        <compilation-unit name="with-composite-sk">-->
+      <!--          <output-dir compare="Text">with-composite-sk</output-dir>-->
+      <!--        </compilation-unit>-->
+      <!--      </test-case>-->
+      <!--      <test-case FilePath="array-index/metadata/open">-->
+      <!--        <compilation-unit name="with-composite-sk">-->
+      <!--          <output-dir compare="Text">with-composite-sk</output-dir>-->
+      <!--        </compilation-unit>-->
+      <!--      </test-case>-->
+      <test-case FilePath="array-index/metadata/closed">
+        <compilation-unit name="with-composite-array-different-indicators">
+          <output-dir compare="Text">with-composite-array-different-indicators</output-dir>
+        </compilation-unit>
+      </test-case>
+      <test-case FilePath="array-index/metadata/open">
+        <compilation-unit name="with-composite-array-different-indicators">
+          <output-dir compare="Text">with-composite-array-different-indicators</output-dir>
+        </compilation-unit>
+      </test-case>
+      <test-case FilePath="array-index/metadata/closed">
+        <compilation-unit name="with-3-level-record-path">
+          <output-dir compare="Text">with-3-level-record-path</output-dir>
+        </compilation-unit>
+      </test-case>
+      <test-case FilePath="array-index/metadata/open">
+        <compilation-unit name="with-3-level-record-path">
+          <output-dir compare="Text">with-3-level-record-path</output-dir>
+        </compilation-unit>
+      </test-case>
+    </test-group>
+    <test-group name="array-index/bulk-loading/on-index-creation">
+      <test-case FilePath="array-index/bulk-loading/on-index-creation/closed">
+        <compilation-unit name="use-case-1">
+          <output-dir compare="Text">use-case-1</output-dir>
+        </compilation-unit>
+      </test-case>
+      <test-case FilePath="array-index/bulk-loading/on-index-creation/open">
+        <compilation-unit name="use-case-1">
+          <output-dir compare="Text">use-case-1</output-dir>
+        </compilation-unit>
+      </test-case>
+      <test-case FilePath="array-index/bulk-loading/on-index-creation/closed">
+        <compilation-unit name="use-case-2">
+          <output-dir compare="Text">use-case-2</output-dir>
+        </compilation-unit>
+      </test-case>
+      <test-case FilePath="array-index/bulk-loading/on-index-creation/open">
+        <compilation-unit name="use-case-2">
+          <output-dir compare="Text">use-case-2</output-dir>
+        </compilation-unit>
+      </test-case>
+      <test-case FilePath="array-index/bulk-loading/on-index-creation/closed">
+        <compilation-unit name="use-case-3">
+          <output-dir compare="Text">use-case-3</output-dir>
+        </compilation-unit>
+      </test-case>
+      <test-case FilePath="array-index/bulk-loading/on-index-creation/open">
+        <compilation-unit name="use-case-3">
+          <output-dir compare="Text">use-case-3</output-dir>
+        </compilation-unit>
+      </test-case>
+      <test-case FilePath="array-index/bulk-loading/on-index-creation/closed">
+        <compilation-unit name="use-case-4">
+          <output-dir compare="Text">use-case-4</output-dir>
+        </compilation-unit>
+      </test-case>
+      <test-case FilePath="array-index/bulk-loading/on-index-creation/open">
+        <compilation-unit name="use-case-4">
+          <output-dir compare="Text">use-case-4</output-dir>
+        </compilation-unit>
+      </test-case>
+      <test-case FilePath="array-index/bulk-loading/on-index-creation/closed">
+        <compilation-unit name="with-composite-pk">
+          <output-dir compare="Text">with-composite-pk</output-dir>
+        </compilation-unit>
+      </test-case>
+      <test-case FilePath="array-index/bulk-loading/on-index-creation/closed">
+        <compilation-unit name="with-filter-fields">
+          <output-dir compare="Text">with-filter-fields</output-dir>
+        </compilation-unit>
+      </test-case>
+      <test-case FilePath="array-index/bulk-loading/on-index-creation/closed">
+        <compilation-unit name="with-3-level-record-path">
+          <output-dir compare="Text">with-3-level-record-path</output-dir>
+        </compilation-unit>
+      </test-case>
+    </test-group>
+    <test-group name="array-index/bulk-loading/after-index-creation">
+      <test-case FilePath="array-index/bulk-loading/after-index-creation">
+        <compilation-unit name="use-case-1">
+          <output-dir compare="Text">use-case-1</output-dir>
+        </compilation-unit>
+      </test-case>
+      <test-case FilePath="array-index/bulk-loading/after-index-creation">
+        <compilation-unit name="use-case-2">
+          <output-dir compare="Text">use-case-2</output-dir>
+        </compilation-unit>
+      </test-case>
+      <test-case FilePath="array-index/bulk-loading/after-index-creation">
+        <compilation-unit name="use-case-3">
+          <output-dir compare="Text">use-case-3</output-dir>
+        </compilation-unit>
+      </test-case>
+      <test-case FilePath="array-index/bulk-loading/after-index-creation">
+        <compilation-unit name="use-case-4">
+          <output-dir compare="Text">use-case-4</output-dir>
+        </compilation-unit>
+      </test-case>
+      <test-case FilePath="array-index/bulk-loading/after-index-creation">
+        <compilation-unit name="with-3-level-record-path">
+          <output-dir compare="Text">with-3-level-record-path</output-dir>
+        </compilation-unit>
+      </test-case>
+      <test-case FilePath="array-index/bulk-loading/after-index-creation">
+        <compilation-unit name="with-composite-pk">
+          <output-dir compare="Text">with-composite-pk</output-dir>
+        </compilation-unit>
+      </test-case>
+      <test-case FilePath="array-index/bulk-loading/after-index-creation">
+        <compilation-unit name="with-filter-fields">
+          <output-dir compare="Text">with-filter-fields</output-dir>
+        </compilation-unit>
+      </test-case>
+      <test-case FilePath="array-index/bulk-loading/after-index-creation">
+        <compilation-unit name="with-open-index">
+          <output-dir compare="Text">with-open-index</output-dir>
+        </compilation-unit>
+      </test-case>
+    </test-group>
+    <test-group name="array-index/insert-upsert-delete">
+      <test-case FilePath="array-index/insert-upsert-delete/closed">
+        <compilation-unit name="use-case-1">
+          <output-dir compare="Text">use-case-1</output-dir>
+        </compilation-unit>
+      </test-case>
+      <test-case FilePath="array-index/insert-upsert-delete/open">
+        <compilation-unit name="use-case-1">
+          <output-dir compare="Text">use-case-1</output-dir>
+        </compilation-unit>
+      </test-case>
+      <test-case FilePath="array-index/insert-upsert-delete/closed">
+        <compilation-unit name="use-case-2">
+          <output-dir compare="Text">use-case-2</output-dir>
+        </compilation-unit>
+      </test-case>
+      <test-case FilePath="array-index/insert-upsert-delete/open">
+        <compilation-unit name="use-case-2">
+          <output-dir compare="Text">use-case-2</output-dir>
+        </compilation-unit>
+      </test-case>
+      <test-case FilePath="array-index/insert-upsert-delete/closed">
+        <compilation-unit name="use-case-3">
+          <output-dir compare="Text">use-case-3</output-dir>
+        </compilation-unit>
+      </test-case>
+      <test-case FilePath="array-index/insert-upsert-delete/open">
+        <compilation-unit name="use-case-3">
+          <output-dir compare="Text">use-case-3</output-dir>
+        </compilation-unit>
+      </test-case>
+      <test-case FilePath="array-index/insert-upsert-delete/closed">
+        <compilation-unit name="use-case-4">
+          <output-dir compare="Text">use-case-4</output-dir>
+        </compilation-unit>
+      </test-case>
+      <test-case FilePath="array-index/insert-upsert-delete/open">
+        <compilation-unit name="use-case-4">
+          <output-dir compare="Text">use-case-4</output-dir>
+        </compilation-unit>
+      </test-case>
+      <test-case FilePath="array-index/insert-upsert-delete/closed">
+        <compilation-unit name="with-composite-sk">
+          <output-dir compare="Text">with-composite-sk</output-dir>
+        </compilation-unit>
+      </test-case>
+      <test-case FilePath="array-index/insert-upsert-delete/open">
+        <compilation-unit name="with-composite-sk">
+          <output-dir compare="Text">with-composite-sk</output-dir>
+        </compilation-unit>
+      </test-case>
+      <test-case FilePath="array-index/insert-upsert-delete/closed">
+        <compilation-unit name="with-additional-atomic-index">
+          <output-dir compare="Text">with-additional-atomic-index</output-dir>
+        </compilation-unit>
+      </test-case>
+      <test-case FilePath="array-index/insert-upsert-delete/open">
+        <compilation-unit name="with-additional-atomic-index">
+          <output-dir compare="Text">with-additional-atomic-index</output-dir>
+        </compilation-unit>
+      </test-case>
+      <test-case FilePath="array-index/insert-upsert-delete/closed">
+        <compilation-unit name="with-filter-fields">
+          <output-dir compare="Text">with-filter-fields</output-dir>
+        </compilation-unit>
+      </test-case>
+    </test-group>
+    <test-group name="array-index/join-unnest-queries">
+      <test-case FilePath="array-index/join-unnest-queries">
+        <compilation-unit name="use-case-1">
+          <output-dir compare="Text">use-case-1</output-dir>
+        </compilation-unit>
+      </test-case>
+      <test-case FilePath="array-index/join-unnest-queries">
+        <compilation-unit name="use-case-2">
+          <output-dir compare="Text">use-case-2</output-dir>
+        </compilation-unit>
+      </test-case>
+      <test-case FilePath="array-index/join-unnest-queries">
+        <compilation-unit name="use-case-3">
+          <output-dir compare="Text">use-case-3</output-dir>
+        </compilation-unit>
+      </test-case>
+      <test-case FilePath="array-index/join-unnest-queries">
+        <compilation-unit name="use-case-4">
+          <output-dir compare="Text">use-case-4</output-dir>
+        </compilation-unit>
+      </test-case>
+      <test-case FilePath="array-index/join-unnest-queries">
+        <compilation-unit name="with-open-index">
+          <output-dir compare="Text">with-open-index</output-dir>
+        </compilation-unit>
+      </test-case>
+    </test-group>
+    <test-group name="array-index/select-unnest-queries">
+      <test-case FilePath="array-index/select-unnest-queries/closed">
+        <compilation-unit name="use-case-1">
+          <output-dir compare="Text">use-case-1</output-dir>
+        </compilation-unit>
+      </test-case>
+      <test-case FilePath="array-index/select-unnest-queries/open">
+        <compilation-unit name="use-case-1">
+          <output-dir compare="Text">use-case-1</output-dir>
+        </compilation-unit>
+      </test-case>
+      <test-case FilePath="array-index/select-unnest-queries/closed">
+        <compilation-unit name="use-case-2">
+          <output-dir compare="Text">use-case-2</output-dir>
+        </compilation-unit>
+      </test-case>
+      <test-case FilePath="array-index/select-unnest-queries/open">
+        <compilation-unit name="use-case-2">
+          <output-dir compare="Text">use-case-2</output-dir>
+        </compilation-unit>
+      </test-case>
+      <test-case FilePath="array-index/select-unnest-queries/closed">
+        <compilation-unit name="use-case-3">
+          <output-dir compare="Text">use-case-3</output-dir>
+        </compilation-unit>
+      </test-case>
+      <test-case FilePath="array-index/select-unnest-queries/open">
+        <compilation-unit name="use-case-3">
+          <output-dir compare="Text">use-case-3</output-dir>
+        </compilation-unit>
+      </test-case>
+      <test-case FilePath="array-index/select-unnest-queries/closed">
+        <compilation-unit name="use-case-4">
+          <output-dir compare="Text">use-case-4</output-dir>
+        </compilation-unit>
+      </test-case>
+      <test-case FilePath="array-index/select-unnest-queries/open">
+        <compilation-unit name="use-case-4">
+          <output-dir compare="Text">use-case-4</output-dir>
+        </compilation-unit>
+      </test-case>
+      <test-case FilePath="array-index/select-unnest-queries/closed">
+        <compilation-unit name="with-3-level-record-path">
+          <output-dir compare="Text">with-3-level-record-path</output-dir>
+        </compilation-unit>
+      </test-case>
+      <test-case FilePath="array-index/select-unnest-queries/open">
+        <compilation-unit name="with-3-level-record-path">
+          <output-dir compare="Text">with-3-level-record-path</output-dir>
+        </compilation-unit>
+      </test-case>
+      <test-case FilePath="array-index/select-unnest-queries/closed">
+        <compilation-unit name="with-composite-sk">
+          <output-dir compare="Text">with-composite-sk</output-dir>
+        </compilation-unit>
+      </test-case>
+      <test-case FilePath="array-index/select-unnest-queries/open">
+        <compilation-unit name="with-composite-sk">
+          <output-dir compare="Text">with-composite-sk</output-dir>
+        </compilation-unit>
+      </test-case>
+      <test-case FilePath="array-index/select-unnest-queries/closed">
+        <compilation-unit name="with-composite-pk">
+          <output-dir compare="Text">with-composite-pk</output-dir>
+        </compilation-unit>
+      </test-case>
+      <test-case FilePath="array-index/select-unnest-queries/closed">
+        <compilation-unit name="with-filter-fields">
+          <output-dir compare="Text">with-filter-fields</output-dir>
+        </compilation-unit>
+      </test-case>
+    </test-group>
+    <test-group name="array-index/select-quantified-queries">
+      <test-case FilePath="array-index/select-quantified-queries">
+        <compilation-unit name="use-case-1">
+          <output-dir compare="Text">use-case-1</output-dir>
+        </compilation-unit>
+      </test-case>
+      <test-case FilePath="array-index/select-quantified-queries">
+        <compilation-unit name="use-case-2">
+          <output-dir compare="Text">use-case-2</output-dir>
+        </compilation-unit>
+      </test-case>
+      <test-case FilePath="array-index/select-quantified-queries">
+        <compilation-unit name="use-case-3">
+          <output-dir compare="Text">use-case-3</output-dir>
+        </compilation-unit>
+      </test-case>
+      <test-case FilePath="array-index/select-quantified-queries">
+        <compilation-unit name="use-case-4">
+          <output-dir compare="Text">use-case-4</output-dir>
+        </compilation-unit>
+      </test-case>
+      <test-case FilePath="array-index/select-quantified-queries">
+        <compilation-unit name="with-composite-pk">
+          <output-dir compare="Text">with-composite-pk</output-dir>
+        </compilation-unit>
+      </test-case>
+      <test-case FilePath="array-index/select-quantified-queries">
+        <compilation-unit name="with-open-index">
+          <output-dir compare="Text">with-open-index</output-dir>
+        </compilation-unit>
+      </test-case>
+    </test-group>
+  </test-group>
   <test-group name="nestrecords">
     <test-case FilePath="nestrecords">
       <compilation-unit name="nestrecord">
@@ -12628,7 +13038,7 @@
     <test-case FilePath="feeds">
       <compilation-unit name="change-feed-with-meta-with-mixed-index">
         <output-dir compare="Text">change-feed-with-meta-with-mixed-index</output-dir>
-        <expected-error>Compilation error: Cannot create index on meta fields (in line 61, at column 1)</expected-error>
+        <expected-error>Compilation error: Cannot create index on meta fields (in line 61, at column 35)</expected-error>
       </compilation-unit>
     </test-case>
     <test-case FilePath="feeds">
@@ -12644,7 +13054,7 @@
     <test-case FilePath="feeds">
       <compilation-unit name="change-feed-with-meta-open-index-in-meta">
         <output-dir compare="Text">change-feed-with-meta-open-index-in-meta</output-dir>
-        <expected-error>Compilation error: Cannot create index on meta fields (in line 60, at column 1)</expected-error>
+        <expected-error>Compilation error: Cannot create index on meta fields (in line 60, at column 34)</expected-error>
       </compilation-unit>
     </test-case>
     <test-case FilePath="feeds">
@@ -12655,13 +13065,13 @@
     <test-case FilePath="feeds">
       <compilation-unit name="change-feed-with-meta-pk-in-meta-index-after-ingest">
         <output-dir compare="Text">change-feed-with-meta-pk-in-meta-index-after-ingest</output-dir>
-        <expected-error>Compilation error: Cannot create index on meta fields (in line 27, at column 1)</expected-error>
+        <expected-error>Compilation error: Cannot create index on meta fields (in line 27, at column 37)</expected-error>
       </compilation-unit>
     </test-case>
     <test-case FilePath="feeds">
       <compilation-unit name="change-feed-with-meta-pk-in-meta-index-in-meta">
         <output-dir compare="Text">change-feed-with-meta-pk-in-meta-index-in-meta</output-dir>
-        <expected-error>Compilation error: Cannot create index on meta fields (in line 60, at column 1)</expected-error>
+        <expected-error>Compilation error: Cannot create index on meta fields (in line 60, at column 37)</expected-error>
       </compilation-unit>
     </test-case>
     <test-case FilePath="feeds">
diff --git a/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/config/CompilerProperties.java b/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/config/CompilerProperties.java
index 39142e5..8064c05 100644
--- a/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/config/CompilerProperties.java
+++ b/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/config/CompilerProperties.java
@@ -95,7 +95,11 @@
         COMPILER_MIN_MEMORY_ALLOCATION(
                 BOOLEAN,
                 AlgebricksConfig.MIN_MEMORY_ALLOCATION_DEFAULT,
-                "Enable/disable allocating minimum budget for certain queries");
+                "Enable/disable allocating minimum budget for certain queries"),
+        COMPILER_ARRAYINDEX(
+                BOOLEAN,
+                AlgebricksConfig.ARRAY_INDEX_DEFAULT,
+                "Enable/disable using array-indexes in queries");
 
         private final IOptionType type;
         private final Object defaultValue;
@@ -156,6 +160,8 @@
 
     public static final String COMPILER_MIN_MEMORY_ALLOCATION_KEY = Option.COMPILER_MIN_MEMORY_ALLOCATION.ini();
 
+    public static final String COMPILER_ARRAYINDEX_KEY = Option.COMPILER_ARRAYINDEX.ini();
+
     public static final int COMPILER_PARALLELISM_AS_STORAGE = 0;
 
     public CompilerProperties(PropertiesAccessor accessor) {
@@ -221,4 +227,8 @@
     public boolean getMinMemoryAllocation() {
         return accessor.getBoolean(Option.COMPILER_MIN_MEMORY_ALLOCATION);
     }
+
+    public boolean isArrayIndex() {
+        return accessor.getBoolean(Option.COMPILER_ARRAYINDEX);
+    }
 }
diff --git a/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/config/DatasetConfig.java b/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/config/DatasetConfig.java
index 3c9acf4..f673d24 100644
--- a/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/config/DatasetConfig.java
+++ b/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/config/DatasetConfig.java
@@ -39,7 +39,8 @@
         SINGLE_PARTITION_WORD_INVIX,
         SINGLE_PARTITION_NGRAM_INVIX,
         LENGTH_PARTITIONED_WORD_INVIX,
-        LENGTH_PARTITIONED_NGRAM_INVIX
+        LENGTH_PARTITIONED_NGRAM_INVIX,
+        ARRAY;
     }
 
     public enum TransactionState {
diff --git a/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/config/OptimizationConfUtil.java b/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/config/OptimizationConfUtil.java
index 8832054..81974fa 100644
--- a/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/config/OptimizationConfUtil.java
+++ b/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/config/OptimizationConfUtil.java
@@ -72,6 +72,8 @@
                 CompilerProperties.COMPILER_SUBPLAN_NESTEDPUSHDOWN_KEY, compilerProperties.getSubplanNestedPushdown());
         boolean minMemoryAllocation = getBoolean(querySpecificConfig,
                 CompilerProperties.COMPILER_MIN_MEMORY_ALLOCATION_KEY, compilerProperties.getMinMemoryAllocation());
+        boolean arrayIndex = getBoolean(querySpecificConfig, CompilerProperties.COMPILER_ARRAYINDEX_KEY,
+                compilerProperties.isArrayIndex());
 
         PhysicalOptimizationConfig physOptConf = new PhysicalOptimizationConfig();
         physOptConf.setFrameSize(frameSize);
@@ -88,6 +90,7 @@
         physOptConf.setSubplanMerge(subplanMerge);
         physOptConf.setSubplanNestedPushdown(subplanNestedPushdown);
         physOptConf.setMinMemoryAllocation(minMemoryAllocation);
+        physOptConf.setArrayIndexEnabled(arrayIndex);
         return physOptConf;
     }
 
diff --git a/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/exceptions/ErrorCode.java b/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/exceptions/ErrorCode.java
index 90a1f8b..bcaf0ae 100644
--- a/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/exceptions/ErrorCode.java
+++ b/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/exceptions/ErrorCode.java
@@ -224,6 +224,7 @@
     FULL_TEXT_CONFIG_NOT_FOUND(1137),
     FULL_TEXT_FILTER_NOT_FOUND(1138),
     FULL_TEXT_DEFAULT_CONFIG_CANNOT_BE_DELETED_OR_CREATED(1139),
+    COMPILATION_INCOMPATIBLE_INDEX_TYPE(1140),
 
     // Feed errors
     DATAFLOW_ILLEGAL_STATE(3001),
diff --git a/asterixdb/asterix-common/src/main/resources/asx_errormsg/en.properties b/asterixdb/asterix-common/src/main/resources/asx_errormsg/en.properties
index 5b993a7..f8f00db 100644
--- a/asterixdb/asterix-common/src/main/resources/asx_errormsg/en.properties
+++ b/asterixdb/asterix-common/src/main/resources/asx_errormsg/en.properties
@@ -226,6 +226,7 @@
 1137 = Full-text config %1$s not found
 1138 = Only a single authentication method is allowed: connectionString, accountName & accountKey, or accountName & sharedAccessSignature
 1139 = No authentication parameters provided
+1140 = Incompatible index type %1$s
 
 # Feed Errors
 3001 = Illegal state.
diff --git a/asterixdb/asterix-doc/pom.xml b/asterixdb/asterix-doc/pom.xml
index c58d983..4e84647 100644
--- a/asterixdb/asterix-doc/pom.xml
+++ b/asterixdb/asterix-doc/pom.xml
@@ -52,7 +52,7 @@
             <configuration>
               <target>
                 <concat destfile="${project.build.directory}/generated-site/markdown/sqlpp/manual.md">
-                  <filelist dir="${project.basedir}/src/main/markdown/sqlpp" files="0_toc.md,1_intro.md,2_expr_title.md,2_expr.md,3_query_title.md,3_query.md,4_windowfunctions_title.md,4_windowfunctions.md,5_error_title.md,5_error.md,6_sql_diff_title.md,6_sql_diff.md,7_ddl_head.md,7_ddl_dml.md,appendix_1_title.md,appendix_1_keywords.md,appendix_2_title.md,appendix_2_parameters.md,appendix_2_parallel_sort.md,appendix_2_index_only.md,appendix_2_hints.md,appendix_3_title.md,appendix_3_resolution.md,appendix_4_title.md,appendix_4_manual_data.md" />
+                  <filelist dir="${project.basedir}/src/main/markdown/sqlpp" files="0_toc.md,1_intro.md,2_expr_title.md,2_expr.md,3_query_title.md,3_query.md,4_windowfunctions_title.md,4_windowfunctions.md,5_error_title.md,5_error.md,6_sql_diff_title.md,6_sql_diff.md,7_ddl_head.md,7_ddl_dml.md,appendix_1_title.md,appendix_1_keywords.md,appendix_2_title.md,appendix_2_parameters.md,appendix_2_parallel_sort.md,appendix_2_index_only.md,appendix_2_arrayindex.md,appendix_2_hints.md,appendix_3_title.md,appendix_3_resolution.md,appendix_4_title.md,appendix_4_manual_data.md" />
                 </concat>
                 <concat destfile="${project.build.directory}/generated-site/markdown/sqlpp/builtins.md">
                   <filelist dir="${project.basedir}/src/main/markdown/builtins" files="0_toc.md,0_toc_sqlpp.md,0_toc_common.md,1_numeric_common.md,1_numeric_delta.md,2_string_common.md,2_string_delta.md,3_binary.md,4_spatial.md,5_similarity.md,6_tokenizing.md,7_temporal.md,7_allens.md,8_record.md,9_aggregate_sql.md,10_comparison.md,11_type_common.md,11_type_delta.md,11_type_conversion.md,13_conditional.md,12_misc.md,15_bitwise.md,14_window.md" />
diff --git a/asterixdb/asterix-doc/src/main/grammar/sqlpp.ebnf b/asterixdb/asterix-doc/src/main/grammar/sqlpp.ebnf
index aae81ec..2a35a31 100644
--- a/asterixdb/asterix-doc/src/main/grammar/sqlpp.ebnf
+++ b/asterixdb/asterix-doc/src/main/grammar/sqlpp.ebnf
@@ -210,10 +210,16 @@
 CreateIndex ::= CreateSecondaryIndex | CreatePrimaryKeyIndex
 
 CreateSecondaryIndex ::= "CREATE" "INDEX" Identifier ("IF" "NOT" "EXISTS")? "ON" QualifiedName
-                       "(" IndexField ( "," IndexField )* ")" ("TYPE" IndexType)? ("ENFORCED")?
+                       "(" IndexedElement ( "," IndexedElement )* ")" ("TYPE" IndexType)? ("ENFORCED")?
 
 CreatePrimaryKeyIndex ::= "CREATE" "PRIMARY" "INDEX" Identifier? ("IF" "NOT" "EXISTS")? "ON" QualifiedName ("TYPE" "BTREE")?
 
+IndexedElement ::= ArrayIndexElement
+               | IndexField
+               | "(" ( ArrayIndexElement | IndexField ) ")"
+
+ArrayIndexElement ::= "UNNEST" NestedField ( "UNNEST" NestedField )* ( ( ":" TypeReference "?"? ) | ( "SELECT" IndexField ( "," IndexField )* ) )?
+
 IndexField ::= NestedField ( ":" TypeReference "?"? )?
 
 IndexType ::= "BTREE"
diff --git a/asterixdb/asterix-doc/src/main/markdown/sqlpp/7_ddl_dml.md b/asterixdb/asterix-doc/src/main/markdown/sqlpp/7_ddl_dml.md
index 222c7a4..b591b3e 100644
--- a/asterixdb/asterix-doc/src/main/markdown/sqlpp/7_ddl_dml.md
+++ b/asterixdb/asterix-doc/src/main/markdown/sqlpp/7_ddl_dml.md
@@ -322,6 +322,12 @@
 ### CreatePrimaryKeyIndex
 **![](../images/diagrams/CreatePrimaryKeyIndex.png)**
 
+### IndexedElement
+**![](../images/diagrams/IndexedElement.png)**
+
+### ArrayIndexElement
+**![](../images/diagrams/ArrayIndexElement.png)**
+
 ### IndexField
 **![](../images/diagrams/IndexField.png)**
 
@@ -337,6 +343,8 @@
 Supported index types include `BTREE` for totally ordered datatypes, `RTREE` for spatial data,
 and `KEYWORD` and `NGRAM` for textual (string) data.
 An index can be created on a nested field (or fields) by providing a valid path expression as an index field identifier.
+An array index can be created on an array or multiset datatype by providing a sequence of `UNNEST` and `SELECT`s to
+identify the field(s) to be indexed.
 
 An indexed field is not required to be part of the datatype associated with a dataset if the dataset's datatype
 is declared as open **and** if the field's type is provided along with its name and if the `ENFORCED` keyword is
@@ -375,12 +383,19 @@
 a nested field residing within a object-valued user field in the `orders` dataset.
 This index can be useful for accelerating exact-match queries, range search queries,
 and joins involving the nested `orderUserName` field.
-Such nested fields must be singular, i.e., one cannot index through (or on) an array-valued field.
 
 #### Example
 
     CREATE INDEX oOrderUserNameIdx ON orders(order.orderUserName) TYPE BTREE;
 
+The following example creates an array index called `oItemsPriceIdx` on the `price` field inside the `items` array of the `orders` dataset.
+This index can be useful for accelerating membership queries, existential or universal quantification queries, or joins involving the `price` field inside this array.
+(To enable array index query optimization, be sure to set the [`arrayindex` compiler option](manual.html#ArrayIndexFlag).)
+
+#### Example
+
+    CREATE INDEX oItemsPriceIdx ON orders(UNNEST items SELECT price);
+
 The following example creates an open rtree index called `oOrderLocIdx` on the order-location field of the `orders` dataset. This index can be useful for accelerating queries that use the [`spatial-intersect` function](builtins.html#spatial_intersect) in a predicate involving the sender-location field.
 
 #### Example
diff --git a/asterixdb/asterix-doc/src/main/markdown/sqlpp/appendix_2_arrayindex.md b/asterixdb/asterix-doc/src/main/markdown/sqlpp/appendix_2_arrayindex.md
new file mode 100644
index 0000000..ee90efb
--- /dev/null
+++ b/asterixdb/asterix-doc/src/main/markdown/sqlpp/appendix_2_arrayindex.md
@@ -0,0 +1,34 @@
+<!--
+ ! Licensed to the Apache Software Foundation (ASF) under one
+ ! or more contributor license agreements.  See the NOTICE file
+ ! distributed with this work for additional information
+ ! regarding copyright ownership.  The ASF licenses this file
+ ! to you under the Apache License, Version 2.0 (the
+ ! "License"); you may not use this file except in compliance
+ ! with the License.  You may obtain a copy of the License at
+ !
+ !   http://www.apache.org/licenses/LICENSE-2.0
+ !
+ ! Unless required by applicable law or agreed to in writing,
+ ! software distributed under the License is distributed on an
+ ! "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ ! KIND, either express or implied.  See the License for the
+ ! specific language governing permissions and limitations
+ ! under the License.
+ !-->
+
+## <a id="ArrayIndexFlag">Controlling Array-Index Access Method Plan Parameter</a>
+By default, the system does not attempt to utilize array indexes as an access method (even if an array index is present and is applicable).
+If you believe that your query will benefit from an array index, toggle the parameter below.
+
+*  **compiler.arrayindex**: if this is set to true, array indexes will be considered as an access method for applicable queries; the default value is false.
+
+
+#### Example
+
+    set `compiler.arrayindex` "true";
+
+    SELECT o.orderno
+    FROM orders o
+    WHERE SOME i IN o.items
+    SATISFIES i.price = 19.91;
diff --git a/asterixdb/asterix-doc/src/site/markdown/sqlpp/arrayindex.md b/asterixdb/asterix-doc/src/site/markdown/sqlpp/arrayindex.md
new file mode 100644
index 0000000..5841e67
--- /dev/null
+++ b/asterixdb/asterix-doc/src/site/markdown/sqlpp/arrayindex.md
@@ -0,0 +1,188 @@
+<!--
+ ! 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.
+ !-->
+
+# AsterixDB Support of Array Indexes #
+
+## <a id="toc">Table of Contents</a> ##
+
+* [Overview](#Overview)
+* [Quantification Queries](#QuantificationQueries)
+* [Explicit Unnesting Queries](#ExplicitUnnestQueries)
+* [Join Queries](#JoinQueries)
+* [Complex Indexing Examples](#ComplexIndexingExamples)
+
+
+## <a id="Overview">Overview</a> <font size="4"><a href="#toc">[Back to TOC]</a></font> ##
+
+Array indexes are used in applications where users want to accelerate a query that involves some array-valued or multiset-valued field.
+This enables fast evaluation of predicates in queries involving arrays or multisets in datasets.
+For brevity, all further mentions of array-valued fields are also applicable to multiset-valued fields.
+
+Array-valued fields are a natural data modeling concept for documents.
+In the traditional inventory management example, it is natural for the line items of an order to exist as a part of the order itself.
+Previously if an AsterixDB user wanted to optimize a query involving a predicate on the line items of an order, they would a) have to undertake some form of schema migration to separate the line items from the orders into different datasets, b) create an index on the new dataset for line items, and finally c) modify their query to join orders and line items.
+With the introduction of array indexes in AsterixDB, users can keep their arrays intact and still reap the performance benefits of an index.
+
+It should be noted that in AsterixDB, array indexes are *not* meant to serve as covering indexes.
+In fact due to AsterixDB's record-level locking, index-only plans involving multi-valued fields (i.e. array indexes and inverted indexes) are not currently possible.
+Instead, array indexes are simply meant to accelerate queries involving multi-valued fields.
+
+
+## <a id="QuantificationQueries">Quantification Queries</a> <font size="4"><a href="#toc">[Back to TOC]</a></font> ##
+
+A common use-case for array indexes involves quantifying some or all elements within an array.
+Quantification queries have two variants: existential and universal.
+Existential queries ask if *any* element in some array satisfies a given predicate.
+Membership queries are a specific type of existential query, asking if any element in some array is equal to a particular value.
+Universal queries ask if *all* elements in some array satisfy a particular predicate.
+Empty arrays are not stored in an array index, meaning that a user must additionally specify that the array is non-empty to tell AsterixDB that it is possible to use an array index as an access method for the given query.
+
+All query examples here will use the orders and products datasets below.
+
+    CREATE TYPE ordersType AS {
+        orderno:        int,
+        custid:			string,
+        items:          [{ itemno: int, productno: int, qty: int, price: float }]
+    };
+    CREATE DATASET orders (ordersType) PRIMARY KEY orderno;
+
+    CREATE TYPE productsType AS {
+        productno:      int,
+        categories:     {{ string }}
+    };
+    CREATE DATASET products (productsType) PRIMARY KEY productno;
+
+Let us now create an index on the `categories` multiset of the `products` dataset.
+
+    CREATE INDEX pCategoriesIdx ON products (UNNEST categories);
+
+Suppose we now want to find all products that have the category "Food".
+The following membership query will utilize the index we just created.
+
+    SET `compiler.arrayindex` "true";
+
+    SELECT p
+    FROM products p
+    WHERE "Food" IN p.categories;
+
+We can also rewrite the query above as an explicit existential quantification query with an equality predicate and the index will be utilized.
+
+    SET `compiler.arrayindex` "true";
+
+    SELECT p
+    FROM products p
+    WHERE SOME c IN p.categories SATISFIES c = "Food";
+
+Let us now create an index on the `qty` and `price` fields in the `items` array of the `orders` dataset.
+
+    CREATE INDEX oItemsQtyPriceIdx ON orders (UNNEST items SELECT qty, price);
+
+Now suppose we want to find all orders that only have items with large quantities and low prices, not counting orders without any items.
+The following universal quantification query will utilize the index we just created.
+
+    SET `compiler.arrayindex` "true";
+
+    SELECT o
+    FROM orders o
+    WHERE LEN(o.items) > 0 AND
+          (EVERY i IN o.items SATISFIES i.qty > 100 AND i.price < 5.00);
+
+Take note of the `LEN(o.items) > 0` conjunct.
+Array indexes cannot be used for queries with potentially empty arrays.
+
+
+## <a id="ExplicitUnnestQueries">Explicit Unnesting Queries</a> <font size="4"><a href="#toc">[Back to TOC]</a></font> ##
+
+Array indexes can also be used to accelerate queries that involve the explicit unnesting of array fields.
+We can express the same membership / existential example above using an explicit `UNNEST` query.
+(To keep the same cardinality as the query above (i.e. to undo the `UNNEST`), we add a `DISTINCT` clause, though the index would be utilized either way.)
+
+    SET `compiler.arrayindex` "true";
+
+    SELECT DISTINCT p
+    FROM products p, p.categories c
+    WHERE c = "Food";
+
+As another example, suppose that we want to find all orders that have *some* item with a large quantity.
+The following query will utilize the `oItemsQtyPriceIdx` we created, using only the first field in the index `qty`.
+
+    SET `compiler.arrayindex` "true";
+
+    SELECT DISTINCT o
+    FROM orders o, o.items i
+    WHERE i.qty > 100;
+
+
+## <a id="JoinQueries">Join Queries</a> <font size="4"><a href="#toc">[Back to TOC]</a></font> ##
+
+Finally, array indexes can also be used for index nested-loop joins if the field being joined is located within an array.
+Let us create another index on the `items` array of the `orders` dataset, this time on the `productno` field.
+
+    CREATE INDEX oProductIDIdx ON orders (UNNEST items SELECT productno);
+
+Now suppose we want to find all products located in a specific order.
+We can accomplish this with the join query below.
+Note that we must specify the `indexnl` join hint to tell AsterixDB that we want to optimize this specific join, as hash joins are the default join method otherwise.
+
+    SET `compiler.arrayindex` "true";
+
+    SELECT DISTINCT p
+    FROM products p, orders o, o.items i
+    WHERE i.productno /*+ indexnl */ = p.productno
+          AND o.custid = "C41";
+
+
+## <a id="ComplexIndexingExamples">Complex Indexing Examples</a> <font size="4"><a href="#toc">[Back to TOC]</a></font> ##
+
+### Open Indexes
+
+Similar to atomic indexes, array indexes are not limited to closed fields.
+The following DDLs illustrate how we could express `CREATE INDEX` statements comparable to those above if the to-be-indexed fields were not included in the their dataset's type definitions.
+
+    CREATE INDEX pCategoriesIdx ON products (UNNEST categories : string?);
+    CREATE INDEX oItemsQtyPriceIdx ON orders (UNNEST items SELECT qty : int?, price : int?);
+    CREATE INDEX oProductIDIdx ON orders (UNNEST items SELECT productno : int?);
+
+### Arrays in Arrays
+
+Array indexes are not just limited to arrays of depth = 1.
+We can generalize this to arrays of arbitrary depth, as long as an object encapsulates each array.
+The following DDLs describe indexing the `qty` field in an `items` array at various depths.
+
+    // { orderno: ..., items0: [ { items1: [ { qty: int, ... } ] } ] }
+    CREATE INDEX oItemItemQtyIdx ON orders (UNNEST items0 UNNEST items1 SELECT qty);
+
+    // { orderno: ..., items0: [ { items1: [ { items2: [ { qty: int, ... } ] } ] } ] }
+    CREATE INDEX oItemItemItemQtyIdx ON orders (UNNEST items0 UNNEST items1 UNNEST items2 SELECT qty);
+
+The queries below will utilize the indexes above.
+The first query utilizes the `oItemItemQtyIdx` index through nested existential quantification.
+The second query utilizes the `oItemItemItemQtyIdx` index with three unnesting clauses.
+
+    SET `compiler.arrayindex` "true";
+
+    SELECT o
+    FROM orders o
+    WHERE SOME o0 IN o.items0 SATISFIES (
+        SOME o1 IN o0.items1 SATISFIES o1.qty = 100
+    );
+
+    SELECT DISTINCT o
+    FROM orders o, o.items0 o0, o0.items1 o1, o1.items2 o2
+    WHERE o2.qty = 100;
diff --git a/asterixdb/asterix-doc/src/site/site.xml b/asterixdb/asterix-doc/src/site/site.xml
index ee71f9a..45193fc 100644
--- a/asterixdb/asterix-doc/src/site/site.xml
+++ b/asterixdb/asterix-doc/src/site/site.xml
@@ -97,6 +97,7 @@
       <item name="Support of Full-text Queries" href="sqlpp/fulltext.html"/>
       <item name="Support of Similarity Queries" href="sqlpp/similarity.html"/>
       <item name="Support of Interval Joins" href="interval_join.html"/>
+      <item name="Support of Array Indexes" href="sqlpp/arrayindex.html"/>
     </menu>
 
     <menu name="Deprecated">
diff --git a/asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/statement/CreateIndexStatement.java b/asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/statement/CreateIndexStatement.java
index 55d0610..468006c 100644
--- a/asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/statement/CreateIndexStatement.java
+++ b/asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/statement/CreateIndexStatement.java
@@ -18,8 +18,10 @@
  */
 package org.apache.asterix.lang.common.statement;
 
-import java.util.ArrayList;
+import java.util.Collections;
 import java.util.List;
+import java.util.Objects;
+import java.util.stream.Collectors;
 
 import org.apache.asterix.common.config.DatasetConfig.IndexType;
 import org.apache.asterix.common.exceptions.CompilationException;
@@ -30,101 +32,67 @@
 import org.apache.asterix.lang.common.struct.Identifier;
 import org.apache.asterix.lang.common.visitor.base.ILangVisitor;
 import org.apache.hyracks.algebricks.common.utils.Pair;
+import org.apache.hyracks.algebricks.common.utils.Triple;
+import org.apache.hyracks.api.exceptions.SourceLocation;
 
 public class CreateIndexStatement extends AbstractStatement {
 
-    private Identifier indexName;
-    private DataverseName dataverseName;
-    private Identifier datasetName;
-    private List<Pair<List<String>, IndexedTypeExpression>> fieldExprs = new ArrayList<>();
-    private List<Integer> fieldIndexIndicators = new ArrayList<>();
-    private IndexType indexType = IndexType.BTREE;
-    private boolean enforced;
-    private boolean ifNotExists;
-
+    private final DataverseName dataverseName;
+    private final Identifier datasetName;
+    private final Identifier indexName;
+    private final IndexType indexType;
+    private final List<IndexedElement> indexedElements;
+    private final boolean enforced;
+    private final boolean ifNotExists;
     // Specific to NGram indexes.
-    private int gramLength = -1;
+    private final int gramLength;
     // Specific to FullText indexes.
-    private String fullTextConfigName;
+    private final String fullTextConfigName;
 
-    public CreateIndexStatement() {
-    }
-
-    public void setGramLength(int gramLength) {
+    public CreateIndexStatement(DataverseName dataverseName, Identifier datasetName, Identifier indexName,
+            IndexType indexType, List<IndexedElement> indexedElements, boolean enforced, int gramLength,
+            String fullTextConfigName, boolean ifNotExists) {
+        this.dataverseName = dataverseName;
+        this.datasetName = Objects.requireNonNull(datasetName);
+        this.indexName = Objects.requireNonNull(indexName);
+        this.indexType = Objects.requireNonNull(indexType);
+        this.indexedElements = Objects.requireNonNull(indexedElements);
+        this.enforced = enforced;
         this.gramLength = gramLength;
-    }
-
-    public int getGramLength() {
-        return gramLength;
-    }
-
-    public void setFullTextConfigName(String fullTextConfigName) {
+        this.ifNotExists = ifNotExists;
         this.fullTextConfigName = fullTextConfigName;
-        return;
     }
 
     public String getFullTextConfigName() {
         return fullTextConfigName;
     }
 
-    public Identifier getIndexName() {
-        return indexName;
-    }
-
-    public void setIndexName(Identifier indexName) {
-        this.indexName = indexName;
-    }
-
     public DataverseName getDataverseName() {
         return dataverseName;
     }
 
-    public void setDataverseName(DataverseName dataverseName) {
-        this.dataverseName = dataverseName;
-    }
-
     public Identifier getDatasetName() {
         return datasetName;
     }
 
-    public void setDatasetName(Identifier datasetName) {
-        this.datasetName = datasetName;
-    }
-
-    public List<Pair<List<String>, IndexedTypeExpression>> getFieldExprs() {
-        return fieldExprs;
-    }
-
-    public void addFieldExprPair(Pair<List<String>, IndexedTypeExpression> fp) {
-        this.fieldExprs.add(fp);
-    }
-
-    public List<Integer> getFieldSourceIndicators() {
-        return fieldIndexIndicators;
-    }
-
-    public void addFieldIndexIndicator(Integer index) {
-        fieldIndexIndicators.add(index);
+    public Identifier getIndexName() {
+        return indexName;
     }
 
     public IndexType getIndexType() {
         return indexType;
     }
 
-    public void setIndexType(IndexType indexType) {
-        this.indexType = indexType;
+    public List<IndexedElement> getIndexedElements() {
+        return indexedElements;
     }
 
     public boolean isEnforced() {
         return enforced;
     }
 
-    public void setEnforced(boolean isEnforced) {
-        this.enforced = isEnforced;
-    }
-
-    public void setIfNotExists(boolean ifNotExists) {
-        this.ifNotExists = ifNotExists;
+    public int getGramLength() {
+        return gramLength;
     }
 
     public boolean getIfNotExists() {
@@ -136,17 +104,6 @@
         return Statement.Kind.CREATE_INDEX;
     }
 
-    public boolean hasMetaField() {
-        if (fieldIndexIndicators != null) {
-            for (Integer indicator : fieldIndexIndicators) {
-                if (indicator.intValue() != 0) {
-                    return true;
-                }
-            }
-        }
-        return false;
-    }
-
     @Override
     public <R, T> R accept(ILangVisitor<R, T> visitor, T arg) throws CompilationException {
         return visitor.visit(this, arg);
@@ -157,4 +114,57 @@
         return Category.DDL;
     }
 
+    public static final class IndexedElement {
+
+        private final int sourceIndicator;
+
+        private final List<List<String>> unnestList;
+
+        private final List<Pair<List<String>, IndexedTypeExpression>> projectList;
+
+        private SourceLocation sourceLoc;
+
+        public IndexedElement(int sourceIndicator, List<List<String>> unnestList,
+                List<Pair<List<String>, IndexedTypeExpression>> projectList) {
+            if (Objects.requireNonNull(projectList).isEmpty()) {
+                throw new IllegalArgumentException();
+            }
+            this.sourceIndicator = sourceIndicator;
+            this.unnestList = unnestList != null ? unnestList : Collections.emptyList();
+            this.projectList = projectList;
+        }
+
+        public int getSourceIndicator() {
+            return sourceIndicator;
+        }
+
+        public boolean hasUnnest() {
+            return !unnestList.isEmpty();
+        }
+
+        public List<List<String>> getUnnestList() {
+            return unnestList;
+        }
+
+        public List<Pair<List<String>, IndexedTypeExpression>> getProjectList() {
+            return projectList;
+        }
+
+        public Triple<Integer, List<List<String>>, List<List<String>>> toIdentifier() {
+            List<List<String>> newProjectList = projectList.stream().map(Pair::getFirst).collect(Collectors.toList());
+            return new Triple<>(sourceIndicator, unnestList, newProjectList);
+        }
+
+        public SourceLocation getSourceLocation() {
+            return sourceLoc;
+        }
+
+        public void setSourceLocation(SourceLocation sourceLoc) {
+            this.sourceLoc = sourceLoc;
+        }
+
+        public String getProjectListDisplayForm() {
+            return projectList.stream().map(Pair::getFirst).map(String::valueOf).collect(Collectors.joining(", "));
+        }
+    }
 }
diff --git a/asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/visitor/FormatPrintVisitor.java b/asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/visitor/FormatPrintVisitor.java
index ac08705..2650ca3 100644
--- a/asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/visitor/FormatPrintVisitor.java
+++ b/asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/visitor/FormatPrintVisitor.java
@@ -642,20 +642,52 @@
         out.print(generateFullName(cis.getDataverseName(), cis.getDatasetName()));
 
         out.print(" (");
-        List<Pair<List<String>, IndexedTypeExpression>> fieldExprs = cis.getFieldExprs();
+        List<CreateIndexStatement.IndexedElement> indexedElements = cis.getIndexedElements();
         int index = 0;
-        int size = fieldExprs.size();
-        for (Pair<List<String>, IndexedTypeExpression> entry : fieldExprs) {
-            printNestField(entry.first);
-            IndexedTypeExpression typeExpr = entry.second;
-            if (typeExpr != null) {
-                out.print(":");
-                typeExpr.getType().accept(this, step);
-                if (typeExpr.isUnknownable()) {
-                    out.print('?');
+        for (CreateIndexStatement.IndexedElement element : indexedElements) {
+            List<Pair<List<String>, IndexedTypeExpression>> projectList = element.getProjectList();
+
+            if (element.hasUnnest()) {
+                int innerIndex = 0;
+                out.print("(");
+                for (List<String> unnest : element.getUnnestList()) {
+                    out.print(" unnest ");
+                    printNestField(unnest);
+                    if (++innerIndex < element.getUnnestList().size()) {
+                        out.print(" ");
+                    }
+                }
+
+                if (projectList.get(0).first != null) {
+                    innerIndex = 0;
+                    out.print(" select ");
+                    for (Pair<List<String>, IndexedTypeExpression> project : projectList) {
+                        printNestField(project.first);
+                        if (project.second != null) {
+                            out.print(":");
+                            project.second.getType().accept(this, step);
+                            if (project.second.isUnknownable()) {
+                                out.print('?');
+                            }
+                        }
+                        if (++innerIndex < element.getProjectList().size()) {
+                            out.print(",");
+                        }
+                    }
+                }
+                out.print(")");
+            } else {
+                printNestField(projectList.get(0).first);
+                IndexedTypeExpression typeExpr = projectList.get(0).second;
+                if (typeExpr != null) {
+                    out.print(":");
+                    typeExpr.getType().accept(this, step);
+                    if (typeExpr.isUnknownable()) {
+                        out.print('?');
+                    }
                 }
             }
-            if (++index < size) {
+            if (++index < indexedElements.size()) {
                 out.print(",");
             }
         }
diff --git a/asterixdb/asterix-lang-sqlpp/src/main/javacc/SQLPP.jj b/asterixdb/asterix-lang-sqlpp/src/main/javacc/SQLPP.jj
index a923551..b650f0e 100644
--- a/asterixdb/asterix-lang-sqlpp/src/main/javacc/SQLPP.jj
+++ b/asterixdb/asterix-lang-sqlpp/src/main/javacc/SQLPP.jj
@@ -227,6 +227,7 @@
     private static final String GROUPS = "GROUPS";
     private static final String IGNORE = "IGNORE";
     private static final String LAST = "LAST";
+    private static final String META = "META";
     private static final String NO = "NO";
     private static final String NULLS = "NULLS";
     private static final String OTHERS = "OTHERS";
@@ -1077,51 +1078,162 @@
 
 CreateIndexStatement IndexSpecification(Token startStmtToken) throws ParseException:
 {
-  CreateIndexStatement stmt = new CreateIndexStatement();
-  String indexName = null;
-  boolean ifNotExists = false;
   Pair<DataverseName,Identifier> nameComponents = null;
-  Pair<Integer, Pair<List<String>, IndexedTypeExpression>> fieldPair = null;
-  IndexParams indexType = null;
+  String indexName = null;
+  IndexParams indexParams = null;
+  CreateIndexStatement.IndexedElement indexedElement = null;
+  List<CreateIndexStatement.IndexedElement> indexedElementList = new ArrayList<CreateIndexStatement.IndexedElement>();
   boolean enforced = false;
+  boolean ifNotExists = false;
+  boolean hasUnnest = false;
+  String fullTextConfigName = null;
+  Token startElementToken = null;
 }
 {
   (
     indexName = Identifier() ifNotExists = IfNotExists()
     <ON> nameComponents = QualifiedName()
-    <LEFTPAREN> ( fieldPair = OpenField()
-      {
-        stmt.addFieldExprPair(fieldPair.second);
-        stmt.addFieldIndexIndicator(fieldPair.first);
+    <LEFTPAREN> { startElementToken  = token; }
+      indexedElement = IndexedElement(startElementToken) {
+        indexedElementList.add(indexedElement);
+        hasUnnest |= indexedElement.hasUnnest();
       }
-    ) (<COMMA> fieldPair = OpenField()
-      {
-        stmt.addFieldExprPair(fieldPair.second);
-        stmt.addFieldIndexIndicator(fieldPair.first);
-      }
-    )* <RIGHTPAREN> ( <TYPE> indexType = IndexType() )? ( <ENFORCED> { enforced = true; } )?
+      (<COMMA> { startElementToken = token; }
+        indexedElement = IndexedElement(startElementToken) {
+          indexedElementList.add(indexedElement);
+          hasUnnest |= indexedElement.hasUnnest();
+        }
+      )*
+    <RIGHTPAREN>
+    ( <TYPE> indexParams = IndexType() )? ( <ENFORCED> { enforced = true; } )?
   )
   {
-    stmt.setIndexName(new Identifier(indexName));
-    stmt.setIfNotExists(ifNotExists);
-    stmt.setDataverseName(nameComponents.first);
-    stmt.setDatasetName(nameComponents.second);
-    if (indexType != null) {
-      stmt.setIndexType(indexType.type);
-      stmt.setGramLength(indexType.gramLength);
-      stmt.setFullTextConfigName(indexType.fullTextConfig);
+    IndexType indexType;
+    int gramLength;
+    if (indexParams != null) {
+      indexType = indexParams.type;
+      gramLength = indexParams.gramLength;
+      fullTextConfigName = indexParams.fullTextConfig;
+    } else {
+      indexType = hasUnnest ? IndexType.ARRAY : IndexType.BTREE;
+      gramLength = -1;
+      fullTextConfigName = null;
     }
-    stmt.setEnforced(enforced);
+    CreateIndexStatement stmt = new CreateIndexStatement(nameComponents.first, nameComponents.second,
+      new Identifier(indexName), indexType, indexedElementList, enforced, gramLength, fullTextConfigName, ifNotExists);
     return addSourceLocation(stmt, startStmtToken);
   }
 }
 
+CreateIndexStatement.IndexedElement IndexedElement(Token startElementToken) throws ParseException:
+{
+  Triple<Integer, List<List<String>>, List<Pair<List<String>, IndexedTypeExpression>>> element = null;
+  Pair<List<String>, IndexedTypeExpression> elementSimple = null;
+  int elementSimpleSource = 0;
+}
+{
+  (
+    element = IndexedElementUnnestSelect()
+    | (
+        LOOKAHEAD({ laIdentifier(META) && laToken(2, LEFTPAREN) && laToken(3, RIGHTPAREN) })
+        <IDENTIFIER> { expectToken(META); } <LEFTPAREN> <RIGHTPAREN>
+        <DOT> elementSimple = IndexedField()
+        { elementSimpleSource = 1; }
+      )
+    | elementSimple = IndexedField()
+    | <LEFTPAREN> ( element = IndexedElementUnnestSelect() | elementSimple = IndexedField() ) <RIGHTPAREN>
+  )
+  {
+    int source;
+    List<List<String>> unnestList;
+    List<Pair<List<String>, IndexedTypeExpression>> projectList;
+    if (elementSimple != null) {
+      source = elementSimpleSource;
+      unnestList = null;
+      projectList = Collections.singletonList(elementSimple);
+    } else {
+      source = element.first;
+      unnestList = element.second;
+      projectList = element.third;
+    }
+    CreateIndexStatement.IndexedElement ie = new CreateIndexStatement.IndexedElement(source, unnestList, projectList);
+    ie.setSourceLocation(getSourceLocation(startElementToken));
+    return ie;
+  }
+}
+
+Triple<Integer, List<List<String>>, List<Pair<List<String>, IndexedTypeExpression>>> IndexedElementUnnestSelect()
+  throws ParseException:
+{
+  int source = 0;
+  Pair<List<List<String>>, List<Pair<List<String>, IndexedTypeExpression>>> element = null;
+}
+{
+  <UNNEST>
+  (
+    (
+       LOOKAHEAD({ laIdentifier(META) && laToken(2, LEFTPAREN) && laToken(3, RIGHTPAREN) })
+       <IDENTIFIER> { expectToken(META); } <LEFTPAREN> <RIGHTPAREN>
+       <DOT> element = IndexedElementUnnestSelectBody() { source = 1; }
+    ) | element = IndexedElementUnnestSelectBody()
+  )
+  {
+    return new Triple<Integer, List<List<String>>, List<Pair<List<String>, IndexedTypeExpression>>>(
+      source, element.first, element.second
+    );
+  }
+}
+
+Pair<List<List<String>>, List<Pair<List<String>, IndexedTypeExpression>>> IndexedElementUnnestSelectBody()
+  throws ParseException:
+{
+  List<String> path = null;
+  IndexedTypeExpression type = null;
+  List<List<String>> unnestList = new ArrayList();
+  Pair<List<String>, IndexedTypeExpression> field = null;
+  List<Pair<List<String>, IndexedTypeExpression>> projectList = new ArrayList();
+}
+{
+  path = MultipartIdentifier() { unnestList.add(path); }
+  ( <UNNEST> path = MultipartIdentifier() { unnestList.add(path); })*
+  (
+    ( <COLON> type = IndexedTypeExpr()
+      {
+        projectList.add(new Pair<List<String>, IndexedTypeExpression>(null, type));
+      }
+    ) |
+    (
+      <SELECT> field = IndexedField() { projectList.add(field); }
+      ( <COMMA> field = IndexedField() { projectList.add(field); } )*
+    )
+  )?
+  {
+    if (projectList.isEmpty()) {
+      // To support the case (<UNNEST> IDENTIFIER)* IDENTIFIER w/o any type specification.
+      projectList.add(new Pair<List<String>, IndexedTypeExpression>(null, null));
+    }
+
+    return new Pair<List<List<String>>, List<Pair<List<String>, IndexedTypeExpression>>>(unnestList, projectList);
+  }
+}
+
+Pair<List<String>, IndexedTypeExpression> IndexedField() throws ParseException:
+{
+  List<String> path = null;
+  IndexedTypeExpression type = null;
+}
+{
+  path = MultipartIdentifier() ( <COLON> type = IndexedTypeExpr() )?
+  {
+    return new Pair<List<String>, IndexedTypeExpression>(path, type);
+  }
+}
+
 CreateIndexStatement PrimaryIndexSpecification(Token startStmtToken) throws ParseException:
 {
-  CreateIndexStatement stmt = new CreateIndexStatement();
+  Pair<DataverseName,Identifier> nameComponents = null;
   String indexName = null;
   boolean ifNotExists = false;
-  Pair<DataverseName,Identifier> nameComponents = null;
 }
 {
   (indexName = Identifier())? ifNotExists = IfNotExists()
@@ -1130,10 +1242,8 @@
     if (indexName == null) {
       indexName = "primary_idx_" + nameComponents.second;
     }
-    stmt.setIndexName(new Identifier(indexName));
-    stmt.setIfNotExists(ifNotExists);
-    stmt.setDataverseName(nameComponents.first);
-    stmt.setDatasetName(nameComponents.second);
+    CreateIndexStatement stmt = new CreateIndexStatement(nameComponents.first, nameComponents.second,
+      new Identifier(indexName), IndexType.BTREE, Collections.emptyList(), false, -1, null, ifNotExists);
     return addSourceLocation(stmt, startStmtToken);
   }
 }
diff --git a/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/MetadataNode.java b/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/MetadataNode.java
index a52fe1c..72b3e9a 100644
--- a/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/MetadataNode.java
+++ b/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/MetadataNode.java
@@ -1238,11 +1238,13 @@
                 //   we may need to include the dataverse of the full-text config in the index.getFullTextConfigDataverse()
                 //   and instead of checking index.getDataverseName(), we need to check index.getFullTextConfigDataverse()
                 //   to see if it is the same as the dataverse of the full-text config
-                String indexConfigName = index.getFullTextConfigName();
-                if (index.getDataverseName().equals(dataverseNameFullTextConfig)
-                        && !Strings.isNullOrEmpty(indexConfigName) && indexConfigName.equals(configName)) {
-                    throw new AlgebricksException("Cannot drop full-text config "
-                            + " because it is being used by index " + index.getIndexName());
+                if (index.getIndexDetails() instanceof Index.TextIndexDetails) {
+                    String indexConfigName = ((Index.TextIndexDetails) index.getIndexDetails()).getFullTextConfigName();
+                    if (index.getDataverseName().equals(dataverseNameFullTextConfig)
+                            && !Strings.isNullOrEmpty(indexConfigName) && indexConfigName.equals(configName)) {
+                        throw new AlgebricksException("Cannot drop full-text config "
+                                + " because it is being used by index " + index.getIndexName());
+                    }
                 }
             }
         }
diff --git a/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/MetadataTransactionContext.java b/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/MetadataTransactionContext.java
index a9ed5ad..a6b1f4b 100644
--- a/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/MetadataTransactionContext.java
+++ b/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/MetadataTransactionContext.java
@@ -144,8 +144,8 @@
     }
 
     public void dropIndex(DataverseName dataverseName, String datasetName, String indexName) {
-        Index index = new Index(dataverseName, datasetName, indexName, null, null, null, null, false, false, false,
-                MetadataUtil.PENDING_NO_OP);
+        Index index =
+                new Index(dataverseName, datasetName, indexName, null, null, false, false, MetadataUtil.PENDING_NO_OP);
         droppedCache.addIndexIfNotExists(index);
         logAndApply(new MetadataLogicalOperation(index, false));
     }
diff --git a/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/declared/ArrayBTreeResourceFactoryProvider.java b/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/declared/ArrayBTreeResourceFactoryProvider.java
new file mode 100644
index 0000000..fc76428
--- /dev/null
+++ b/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/declared/ArrayBTreeResourceFactoryProvider.java
@@ -0,0 +1,219 @@
+/*
+ * 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.metadata.declared;
+
+import java.util.List;
+import java.util.Map;
+
+import org.apache.asterix.common.config.DatasetConfig.DatasetType;
+import org.apache.asterix.common.context.AsterixVirtualBufferCacheProvider;
+import org.apache.asterix.common.context.IStorageComponentProvider;
+import org.apache.asterix.common.exceptions.CompilationException;
+import org.apache.asterix.common.exceptions.ErrorCode;
+import org.apache.asterix.external.indexing.FilesIndexDescription;
+import org.apache.asterix.external.indexing.IndexingConstants;
+import org.apache.asterix.metadata.api.IResourceFactoryProvider;
+import org.apache.asterix.metadata.entities.Dataset;
+import org.apache.asterix.metadata.entities.Index;
+import org.apache.asterix.metadata.utils.ArrayIndexUtil;
+import org.apache.asterix.metadata.utils.IndexUtil;
+import org.apache.asterix.om.types.ARecordType;
+import org.apache.asterix.om.types.IAType;
+import org.apache.hyracks.algebricks.common.exceptions.AlgebricksException;
+import org.apache.hyracks.algebricks.common.utils.Pair;
+import org.apache.hyracks.algebricks.data.IBinaryComparatorFactoryProvider;
+import org.apache.hyracks.algebricks.data.ITypeTraitProvider;
+import org.apache.hyracks.api.compression.ICompressorDecompressorFactory;
+import org.apache.hyracks.api.dataflow.value.IBinaryComparatorFactory;
+import org.apache.hyracks.api.dataflow.value.ITypeTraits;
+import org.apache.hyracks.storage.am.common.api.IMetadataPageManagerFactory;
+import org.apache.hyracks.storage.am.lsm.btree.dataflow.ExternalBTreeLocalResourceFactory;
+import org.apache.hyracks.storage.am.lsm.btree.dataflow.ExternalBTreeWithBuddyLocalResourceFactory;
+import org.apache.hyracks.storage.am.lsm.btree.dataflow.LSMBTreeLocalResourceFactory;
+import org.apache.hyracks.storage.am.lsm.common.api.ILSMIOOperationCallbackFactory;
+import org.apache.hyracks.storage.am.lsm.common.api.ILSMIOOperationSchedulerProvider;
+import org.apache.hyracks.storage.am.lsm.common.api.ILSMMergePolicyFactory;
+import org.apache.hyracks.storage.am.lsm.common.api.ILSMOperationTrackerFactory;
+import org.apache.hyracks.storage.am.lsm.common.api.ILSMPageWriteCallbackFactory;
+import org.apache.hyracks.storage.common.IResourceFactory;
+import org.apache.hyracks.storage.common.IStorageManager;
+import org.apache.hyracks.storage.common.compression.NoOpCompressorDecompressorFactory;
+
+public class ArrayBTreeResourceFactoryProvider implements IResourceFactoryProvider {
+
+    public static final ArrayBTreeResourceFactoryProvider INSTANCE = new ArrayBTreeResourceFactoryProvider();
+
+    private ArrayBTreeResourceFactoryProvider() {
+    }
+
+    @Override
+    public IResourceFactory getResourceFactory(MetadataProvider mdProvider, Dataset dataset, Index index,
+            ARecordType recordType, ARecordType metaType, ILSMMergePolicyFactory mergePolicyFactory,
+            Map<String, String> mergePolicyProperties, ITypeTraits[] filterTypeTraits,
+            IBinaryComparatorFactory[] filterCmpFactories) throws AlgebricksException {
+        int[] filterFields = IndexUtil.getFilterFields(dataset, index, filterTypeTraits);
+        int[] btreeFields = IndexUtil.getBtreeFieldsIfFiltered(dataset, index);
+        IStorageComponentProvider storageComponentProvider = mdProvider.getStorageComponentProvider();
+        ITypeTraits[] typeTraits = getTypeTraits(mdProvider, dataset, index, recordType, metaType);
+        IBinaryComparatorFactory[] cmpFactories = getCmpFactories(mdProvider, dataset, index, recordType, metaType);
+        int[] bloomFilterFields = getBloomFilterFields(dataset, index);
+        double bloomFilterFalsePositiveRate = mdProvider.getStorageProperties().getBloomFilterFalsePositiveRate();
+        ILSMOperationTrackerFactory opTrackerFactory = dataset.getIndexOperationTrackerFactory(index);
+        ILSMIOOperationCallbackFactory ioOpCallbackFactory = dataset.getIoOperationCallbackFactory(index);
+        ILSMPageWriteCallbackFactory pageWriteCallbackFactory = dataset.getPageWriteCallbackFactory();
+        IStorageManager storageManager = storageComponentProvider.getStorageManager();
+        IMetadataPageManagerFactory metadataPageManagerFactory =
+                storageComponentProvider.getMetadataPageManagerFactory();
+        ILSMIOOperationSchedulerProvider ioSchedulerProvider =
+                storageComponentProvider.getIoOperationSchedulerProvider();
+        boolean hasBloomFilter = bloomFilterFields != null;
+        switch (dataset.getDatasetType()) {
+            case EXTERNAL:
+                return index.getIndexName().equals(IndexingConstants.getFilesIndexName(dataset.getDatasetName()))
+                        ? new ExternalBTreeLocalResourceFactory(storageManager, typeTraits, cmpFactories,
+                                filterTypeTraits, filterCmpFactories, filterFields, opTrackerFactory,
+                                ioOpCallbackFactory, pageWriteCallbackFactory, metadataPageManagerFactory,
+                                ioSchedulerProvider, mergePolicyFactory, mergePolicyProperties, true, bloomFilterFields,
+                                bloomFilterFalsePositiveRate, false, btreeFields, hasBloomFilter)
+                        : new ExternalBTreeWithBuddyLocalResourceFactory(storageManager, typeTraits, cmpFactories,
+                                filterTypeTraits, filterCmpFactories, filterFields, opTrackerFactory,
+                                ioOpCallbackFactory, pageWriteCallbackFactory, metadataPageManagerFactory,
+                                ioSchedulerProvider, mergePolicyFactory, mergePolicyProperties, true, bloomFilterFields,
+                                bloomFilterFalsePositiveRate, false, btreeFields, hasBloomFilter);
+            case INTERNAL:
+                AsterixVirtualBufferCacheProvider vbcProvider =
+                        new AsterixVirtualBufferCacheProvider(dataset.getDatasetId());
+
+                final ICompressorDecompressorFactory compDecompFactory;
+                if (index.isPrimaryIndex()) {
+                    //Compress only primary index
+                    compDecompFactory = mdProvider.getCompressionManager().getFactory(dataset.getCompressionScheme());
+                } else {
+                    compDecompFactory = NoOpCompressorDecompressorFactory.INSTANCE;
+                }
+
+                return new LSMBTreeLocalResourceFactory(storageManager, typeTraits, cmpFactories, filterTypeTraits,
+                        filterCmpFactories, filterFields, opTrackerFactory, ioOpCallbackFactory,
+                        pageWriteCallbackFactory, metadataPageManagerFactory, vbcProvider, ioSchedulerProvider,
+                        mergePolicyFactory, mergePolicyProperties, true, bloomFilterFields,
+                        bloomFilterFalsePositiveRate, index.isPrimaryIndex(), btreeFields, compDecompFactory,
+                        hasBloomFilter);
+            default:
+                throw new CompilationException(ErrorCode.COMPILATION_UNKNOWN_DATASET_TYPE,
+                        dataset.getDatasetType().toString());
+        }
+    }
+
+    private static ITypeTraits[] getTypeTraits(MetadataProvider metadataProvider, Dataset dataset, Index index,
+            ARecordType recordType, ARecordType metaType) throws AlgebricksException {
+        ITypeTraits[] primaryTypeTraits = dataset.getPrimaryTypeTraits(metadataProvider, recordType, metaType);
+        if (index.isPrimaryIndex()) {
+            return primaryTypeTraits;
+        } else if (dataset.getDatasetType() == DatasetType.EXTERNAL
+                && index.getIndexName().equals(IndexingConstants.getFilesIndexName(dataset.getDatasetName()))) {
+            return FilesIndexDescription.EXTERNAL_FILE_INDEX_TYPE_TRAITS;
+        }
+        ITypeTraitProvider typeTraitProvider = metadataProvider.getStorageComponentProvider().getTypeTraitProvider();
+        int numPrimaryKeys = dataset.getPrimaryKeys().size();
+        ITypeTraits[] secondaryTypeTraits;
+        Index.ArrayIndexDetails arrayIndexDetails = (Index.ArrayIndexDetails) index.getIndexDetails();
+        int numSecondaryKeys =
+                arrayIndexDetails.getElementList().stream().map(e -> e.getProjectList().size()).reduce(0, Integer::sum);
+        secondaryTypeTraits = new ITypeTraits[numSecondaryKeys + numPrimaryKeys];
+        int secondaryTypeTraitPos = 0;
+        for (Index.ArrayIndexElement e : arrayIndexDetails.getElementList()) {
+            ARecordType sourceType;
+            if (e.getSourceIndicator() == 0) {
+                sourceType = recordType;
+            } else {
+                sourceType = metaType;
+            }
+            for (int i = 0; i < e.getProjectList().size(); i++) {
+                List<String> project = e.getProjectList().get(i);
+                Pair<IAType, Boolean> keyTypePair = ArrayIndexUtil.getNonNullableOpenFieldType(e.getTypeList().get(i),
+                        ArrayIndexUtil.getFlattenedKeyFieldNames(e.getUnnestList(), project), sourceType,
+                        ArrayIndexUtil.getArrayDepthIndicator(e.getUnnestList(), project));
+                IAType keyType = keyTypePair.first;
+                secondaryTypeTraits[secondaryTypeTraitPos++] = typeTraitProvider.getTypeTrait(keyType);
+            }
+        }
+        // Add serializers and comparators for primary index fields.
+        System.arraycopy(primaryTypeTraits, 0, secondaryTypeTraits, numSecondaryKeys, numPrimaryKeys);
+        return secondaryTypeTraits;
+    }
+
+    private static IBinaryComparatorFactory[] getCmpFactories(MetadataProvider metadataProvider, Dataset dataset,
+            Index index, ARecordType recordType, ARecordType metaType) throws AlgebricksException {
+        IBinaryComparatorFactory[] primaryCmpFactories =
+                dataset.getPrimaryComparatorFactories(metadataProvider, recordType, metaType);
+        if (index.isPrimaryIndex()) {
+            return dataset.getPrimaryComparatorFactories(metadataProvider, recordType, metaType);
+        } else if (dataset.getDatasetType() == DatasetType.EXTERNAL
+                && index.getIndexName().equals(IndexingConstants.getFilesIndexName(dataset.getDatasetName()))) {
+            return FilesIndexDescription.FILES_INDEX_COMP_FACTORIES;
+        }
+        Index.ArrayIndexDetails arrayIndexDetails = (Index.ArrayIndexDetails) index.getIndexDetails();
+        int numSecondaryKeys =
+                arrayIndexDetails.getElementList().stream().map(e -> e.getProjectList().size()).reduce(0, Integer::sum);
+        int numPrimaryKeys = dataset.getPrimaryKeys().size();
+        IBinaryComparatorFactoryProvider cmpFactoryProvider =
+                metadataProvider.getStorageComponentProvider().getComparatorFactoryProvider();
+        IBinaryComparatorFactory[] secondaryCmpFactories =
+                new IBinaryComparatorFactory[numSecondaryKeys + numPrimaryKeys];
+        int secondaryCmpFactoriesPos = 0;
+        for (Index.ArrayIndexElement e : arrayIndexDetails.getElementList()) {
+            ARecordType sourceType;
+            if (e.getSourceIndicator() == 0) {
+                sourceType = recordType;
+            } else {
+                sourceType = metaType;
+            }
+            for (int i = 0; i < e.getProjectList().size(); i++) {
+                List<String> project = e.getProjectList().get(i);
+                Pair<IAType, Boolean> keyTypePair = ArrayIndexUtil.getNonNullableOpenFieldType(e.getTypeList().get(i),
+                        ArrayIndexUtil.getFlattenedKeyFieldNames(e.getUnnestList(), project), sourceType,
+                        ArrayIndexUtil.getArrayDepthIndicator(e.getUnnestList(), project));
+                IAType keyType = keyTypePair.first;
+                secondaryCmpFactories[secondaryCmpFactoriesPos++] =
+                        cmpFactoryProvider.getBinaryComparatorFactory(keyType, true);
+            }
+        }
+        // Add serializers and comparators for primary index fields.
+        System.arraycopy(primaryCmpFactories, 0, secondaryCmpFactories, numSecondaryKeys, numPrimaryKeys);
+        return secondaryCmpFactories;
+    }
+
+    private static int[] getBloomFilterFields(Dataset dataset, Index index) throws AlgebricksException {
+        // both the Primary index and the Primary Key index have bloom filters
+        if (index.isPrimaryIndex() || index.isPrimaryKeyIndex()) {
+            return dataset.getPrimaryBloomFilterFields();
+        }
+        if (dataset.getDatasetType() == DatasetType.EXTERNAL) {
+            if (index.getIndexName().equals(IndexingConstants.getFilesIndexName(dataset.getDatasetName()))) {
+                return FilesIndexDescription.BLOOM_FILTER_FIELDS;
+            } else {
+                Index.ArrayIndexDetails arrayIndexDetails = (Index.ArrayIndexDetails) index.getIndexDetails();
+                return new int[] { arrayIndexDetails.getElementList().stream().map(e -> e.getProjectList().size())
+                        .reduce(0, Integer::sum) };
+            }
+        }
+        // secondary btrees and rtrees do not have bloom filters
+        return null;
+    }
+}
diff --git a/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/declared/BTreeResourceFactoryProvider.java b/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/declared/BTreeResourceFactoryProvider.java
index 0ffdedc..7f56f2c 100644
--- a/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/declared/BTreeResourceFactoryProvider.java
+++ b/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/declared/BTreeResourceFactoryProvider.java
@@ -22,7 +22,6 @@
 import java.util.Map;
 
 import org.apache.asterix.common.config.DatasetConfig.DatasetType;
-import org.apache.asterix.common.config.DatasetConfig.IndexType;
 import org.apache.asterix.common.context.AsterixVirtualBufferCacheProvider;
 import org.apache.asterix.common.context.IStorageComponentProvider;
 import org.apache.asterix.common.exceptions.CompilationException;
@@ -129,20 +128,21 @@
                 && index.getIndexName().equals(IndexingConstants.getFilesIndexName(dataset.getDatasetName()))) {
             return FilesIndexDescription.EXTERNAL_FILE_INDEX_TYPE_TRAITS;
         }
+        Index.ValueIndexDetails indexDetails = (Index.ValueIndexDetails) index.getIndexDetails();
         int numPrimaryKeys = dataset.getPrimaryKeys().size();
-        int numSecondaryKeys = index.getKeyFieldNames().size();
+        int numSecondaryKeys = indexDetails.getKeyFieldNames().size();
         ITypeTraitProvider typeTraitProvider = metadataProvider.getStorageComponentProvider().getTypeTraitProvider();
         ITypeTraits[] secondaryTypeTraits = new ITypeTraits[numSecondaryKeys + numPrimaryKeys];
         for (int i = 0; i < numSecondaryKeys; i++) {
             ARecordType sourceType;
-            List<Integer> keySourceIndicators = index.getKeyFieldSourceIndicators();
+            List<Integer> keySourceIndicators = indexDetails.getKeyFieldSourceIndicators();
             if (keySourceIndicators == null || keySourceIndicators.get(i) == 0) {
                 sourceType = recordType;
             } else {
                 sourceType = metaType;
             }
-            Pair<IAType, Boolean> keyTypePair = Index.getNonNullableOpenFieldType(index.getKeyFieldTypes().get(i),
-                    index.getKeyFieldNames().get(i), sourceType);
+            Pair<IAType, Boolean> keyTypePair = Index.getNonNullableOpenFieldType(
+                    indexDetails.getKeyFieldTypes().get(i), indexDetails.getKeyFieldNames().get(i), sourceType);
             IAType keyType = keyTypePair.first;
             secondaryTypeTraits[i] = typeTraitProvider.getTypeTrait(keyType);
         }
@@ -163,22 +163,23 @@
                 && index.getIndexName().equals(IndexingConstants.getFilesIndexName(dataset.getDatasetName()))) {
             return FilesIndexDescription.FILES_INDEX_COMP_FACTORIES;
         }
+        Index.ValueIndexDetails indexDetails = (Index.ValueIndexDetails) index.getIndexDetails();
         int numPrimaryKeys = dataset.getPrimaryKeys().size();
-        int numSecondaryKeys = index.getKeyFieldNames().size();
+        int numSecondaryKeys = indexDetails.getKeyFieldNames().size();
         IBinaryComparatorFactoryProvider cmpFactoryProvider =
                 metadataProvider.getStorageComponentProvider().getComparatorFactoryProvider();
         IBinaryComparatorFactory[] secondaryCmpFactories =
                 new IBinaryComparatorFactory[numSecondaryKeys + numPrimaryKeys];
         for (int i = 0; i < numSecondaryKeys; i++) {
             ARecordType sourceType;
-            List<Integer> keySourceIndicators = index.getKeyFieldSourceIndicators();
+            List<Integer> keySourceIndicators = indexDetails.getKeyFieldSourceIndicators();
             if (keySourceIndicators == null || keySourceIndicators.get(i) == 0) {
                 sourceType = recordType;
             } else {
                 sourceType = metaType;
             }
-            Pair<IAType, Boolean> keyTypePair = Index.getNonNullableOpenFieldType(index.getKeyFieldTypes().get(i),
-                    index.getKeyFieldNames().get(i), sourceType);
+            Pair<IAType, Boolean> keyTypePair = Index.getNonNullableOpenFieldType(
+                    indexDetails.getKeyFieldTypes().get(i), indexDetails.getKeyFieldNames().get(i), sourceType);
             IAType keyType = keyTypePair.first;
             secondaryCmpFactories[i] = cmpFactoryProvider.getBinaryComparatorFactory(keyType, true);
         }
@@ -193,23 +194,34 @@
         // both the Primary index and the Primary Key index have bloom filters
         if (index.isPrimaryIndex() || index.isPrimaryKeyIndex()) {
             return dataset.getPrimaryBloomFilterFields();
-        } else if (dataset.getDatasetType() == DatasetType.EXTERNAL) {
+        }
+        if (dataset.getDatasetType() == DatasetType.EXTERNAL) {
             if (index.getIndexName().equals(IndexingConstants.getFilesIndexName(dataset.getDatasetName()))) {
                 return FilesIndexDescription.BLOOM_FILTER_FIELDS;
             } else {
-                return new int[] { index.getKeyFieldNames().size() };
+                Index.ValueIndexDetails indexDetails = ((Index.ValueIndexDetails) index.getIndexDetails());
+                return new int[] { indexDetails.getKeyFieldNames().size() };
             }
-        } else if (index.getIndexType() == IndexType.BTREE || index.getIndexType() == IndexType.RTREE) {
-            // secondary btrees and rtrees do not have bloom filters
-            return null;
-        } else {
-            // inverted indexes have bloom filters on deleted-key btrees
-            int numKeys = index.getKeyFieldNames().size();
-            int[] bloomFilterKeyFields = new int[numKeys];
-            for (int i = 0; i < numKeys; i++) {
-                bloomFilterKeyFields[i] = i;
-            }
-            return bloomFilterKeyFields;
+        }
+        switch (index.getIndexType()) {
+            case BTREE:
+            case RTREE:
+                // secondary btrees and rtrees do not have bloom filters
+                return null;
+            case LENGTH_PARTITIONED_NGRAM_INVIX:
+            case LENGTH_PARTITIONED_WORD_INVIX:
+            case SINGLE_PARTITION_NGRAM_INVIX:
+            case SINGLE_PARTITION_WORD_INVIX:
+                // inverted indexes have bloom filters on deleted-key btrees
+                int numKeys = ((Index.TextIndexDetails) index.getIndexDetails()).getKeyFieldNames().size();
+                int[] bloomFilterKeyFields = new int[numKeys];
+                for (int i = 0; i < numKeys; i++) {
+                    bloomFilterKeyFields[i] = i;
+                }
+                return bloomFilterKeyFields;
+            default:
+                throw new CompilationException(ErrorCode.COMPILATION_UNKNOWN_INDEX_TYPE,
+                        String.valueOf(index.getIndexType()));
         }
     }
 }
diff --git a/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/declared/MetadataProvider.java b/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/declared/MetadataProvider.java
index ef6f8d9..66e8f58 100644
--- a/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/declared/MetadataProvider.java
+++ b/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/declared/MetadataProvider.java
@@ -37,6 +37,7 @@
 import org.apache.asterix.common.dataflow.ICcApplicationContext;
 import org.apache.asterix.common.dataflow.LSMTreeInsertDeleteOperatorDescriptor;
 import org.apache.asterix.common.exceptions.AsterixException;
+import org.apache.asterix.common.exceptions.CompilationException;
 import org.apache.asterix.common.exceptions.ErrorCode;
 import org.apache.asterix.common.external.IDataSourceAdapter;
 import org.apache.asterix.common.functions.FunctionSignature;
@@ -103,7 +104,9 @@
 import org.apache.asterix.runtime.operators.LSMIndexBulkLoadOperatorDescriptor;
 import org.apache.asterix.runtime.operators.LSMIndexBulkLoadOperatorDescriptor.BulkLoadUsage;
 import org.apache.asterix.runtime.operators.LSMPrimaryInsertOperatorDescriptor;
+import org.apache.asterix.runtime.operators.LSMSecondaryInsertDeleteWithNestedPlanOperatorDescriptor;
 import org.apache.asterix.runtime.operators.LSMSecondaryUpsertOperatorDescriptor;
+import org.apache.asterix.runtime.operators.LSMSecondaryUpsertWithNestedPlanOperatorDescriptor;
 import org.apache.hyracks.algebricks.common.constraints.AlgebricksAbsolutePartitionConstraint;
 import org.apache.hyracks.algebricks.common.constraints.AlgebricksPartitionConstraint;
 import org.apache.hyracks.algebricks.common.exceptions.AlgebricksException;
@@ -128,6 +131,7 @@
 import org.apache.hyracks.algebricks.data.IPrinterFactory;
 import org.apache.hyracks.algebricks.data.IResultSerializerFactoryProvider;
 import org.apache.hyracks.algebricks.data.ISerializerDeserializerProvider;
+import org.apache.hyracks.algebricks.runtime.base.AlgebricksPipeline;
 import org.apache.hyracks.algebricks.runtime.base.IPushRuntimeFactory;
 import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluatorFactory;
 import org.apache.hyracks.algebricks.runtime.operators.std.SinkWriterRuntimeFactory;
@@ -537,6 +541,21 @@
         }
         Index theIndex = isSecondary ? MetadataManager.INSTANCE.getIndex(mdTxnCtx, dataset.getDataverseName(),
                 dataset.getDatasetName(), indexName) : primaryIndex;
+
+        int numSecondaryKeys;
+        switch (theIndex.getIndexType()) {
+            case ARRAY:
+                numSecondaryKeys = ((Index.ArrayIndexDetails) theIndex.getIndexDetails()).getElementList().stream()
+                        .map(e -> e.getProjectList().size()).reduce(0, Integer::sum);
+                break;
+            case BTREE:
+                numSecondaryKeys = ((Index.ValueIndexDetails) theIndex.getIndexDetails()).getKeyFieldNames().size();
+                break;
+            default:
+                throw new CompilationException(ErrorCode.COMPILATION_UNKNOWN_INDEX_TYPE,
+                        theIndex.getIndexType().toString());
+        }
+
         int numPrimaryKeys = dataset.getPrimaryKeys().size();
         RecordDescriptor outputRecDesc = JobGenHelper.mkRecordDescriptor(typeEnv, opSchema, context);
         Pair<IFileSplitProvider, AlgebricksPartitionConstraint> spPc =
@@ -551,7 +570,6 @@
         byte[] failValueForIndexOnlyPlan = null;
         boolean proceedIndexOnlyPlan = isIndexOnlyPlan && isSecondary;
         if (proceedIndexOnlyPlan) {
-            int numSecondaryKeys = theIndex.getKeyFieldNames().size();
             primaryKeyFieldsInSecondaryIndex = new int[numPrimaryKeys];
             for (int i = 0; i < numPrimaryKeys; i++) {
                 primaryKeyFieldsInSecondaryIndex[i] = i + numSecondaryKeys;
@@ -600,6 +618,7 @@
             throw new AlgebricksException(
                     "Code generation error: no index " + indexName + " for dataset " + dataset.getDatasetName());
         }
+        Index.ValueIndexDetails secondaryIndexDetails = (Index.ValueIndexDetails) secondaryIndex.getIndexDetails();
         RecordDescriptor outputRecDesc = JobGenHelper.mkRecordDescriptor(typeEnv, opSchema, context);
         Pair<IFileSplitProvider, AlgebricksPartitionConstraint> spPc =
                 getSplitProviderAndConstraints(dataset, secondaryIndex.getIndexName());
@@ -613,8 +632,8 @@
         byte[] failValueForIndexOnlyPlan = null;
         if (isIndexOnlyPlan) {
             ARecordType recType = (ARecordType) findType(dataset.getItemTypeDataverseName(), dataset.getItemTypeName());
-            List<List<String>> secondaryKeyFields = secondaryIndex.getKeyFieldNames();
-            List<IAType> secondaryKeyTypes = secondaryIndex.getKeyFieldTypes();
+            List<List<String>> secondaryKeyFields = secondaryIndexDetails.getKeyFieldNames();
+            List<IAType> secondaryKeyTypes = secondaryIndexDetails.getKeyFieldTypes();
             Pair<IAType, Boolean> keyTypePair =
                     Index.getNonNullableOpenFieldType(secondaryKeyTypes.get(0), secondaryKeyFields.get(0), recType);
             IAType keyType = keyTypePair.first;
@@ -752,10 +771,11 @@
             IOperatorSchema[] inputSchemas, IVariableTypeEnvironment typeEnv, List<LogicalVariable> primaryKeys,
             List<LogicalVariable> secondaryKeys, List<LogicalVariable> additionalNonKeyFields,
             ILogicalExpression filterExpr, RecordDescriptor recordDesc, JobGenContext context, JobSpecification spec,
-            boolean bulkload) throws AlgebricksException {
+            boolean bulkload, List<List<AlgebricksPipeline>> secondaryKeysPipelines, IOperatorSchema pipelineTopSchema)
+            throws AlgebricksException {
         return getIndexInsertOrDeleteOrUpsertRuntime(IndexOperation.INSERT, dataSourceIndex, propagatedSchema,
                 inputSchemas, typeEnv, primaryKeys, secondaryKeys, additionalNonKeyFields, filterExpr, recordDesc,
-                context, spec, bulkload, null, null, null);
+                context, spec, bulkload, null, null, null, secondaryKeysPipelines, pipelineTopSchema);
     }
 
     @Override
@@ -763,11 +783,12 @@
             IDataSourceIndex<String, DataSourceId> dataSourceIndex, IOperatorSchema propagatedSchema,
             IOperatorSchema[] inputSchemas, IVariableTypeEnvironment typeEnv, List<LogicalVariable> primaryKeys,
             List<LogicalVariable> secondaryKeys, List<LogicalVariable> additionalNonKeyFields,
-            ILogicalExpression filterExpr, RecordDescriptor recordDesc, JobGenContext context, JobSpecification spec)
+            ILogicalExpression filterExpr, RecordDescriptor recordDesc, JobGenContext context, JobSpecification spec,
+            List<List<AlgebricksPipeline>> secondaryKeysPipelines, IOperatorSchema pipelineTopSchema)
             throws AlgebricksException {
         return getIndexInsertOrDeleteOrUpsertRuntime(IndexOperation.DELETE, dataSourceIndex, propagatedSchema,
                 inputSchemas, typeEnv, primaryKeys, secondaryKeys, additionalNonKeyFields, filterExpr, recordDesc,
-                context, spec, false, null, null, null);
+                context, spec, false, null, null, null, secondaryKeysPipelines, pipelineTopSchema);
     }
 
     @Override
@@ -777,10 +798,11 @@
             List<LogicalVariable> secondaryKeys, List<LogicalVariable> additionalFilteringKeys,
             ILogicalExpression filterExpr, LogicalVariable upsertIndicatorVar, List<LogicalVariable> prevSecondaryKeys,
             LogicalVariable prevAdditionalFilteringKey, RecordDescriptor recordDesc, JobGenContext context,
-            JobSpecification spec) throws AlgebricksException {
+            JobSpecification spec, List<List<AlgebricksPipeline>> secondaryKeysPipelines) throws AlgebricksException {
         return getIndexInsertOrDeleteOrUpsertRuntime(IndexOperation.UPSERT, dataSourceIndex, propagatedSchema,
                 inputSchemas, typeEnv, primaryKeys, secondaryKeys, additionalFilteringKeys, filterExpr, recordDesc,
-                context, spec, false, upsertIndicatorVar, prevSecondaryKeys, prevAdditionalFilteringKey);
+                context, spec, false, upsertIndicatorVar, prevSecondaryKeys, prevAdditionalFilteringKey,
+                secondaryKeysPipelines, null);
     }
 
     @Override
@@ -1185,7 +1207,8 @@
             List<LogicalVariable> additionalNonKeyFields, ILogicalExpression filterExpr,
             RecordDescriptor inputRecordDesc, JobGenContext context, JobSpecification spec, boolean bulkload,
             LogicalVariable upsertIndicatorVar, List<LogicalVariable> prevSecondaryKeys,
-            LogicalVariable prevAdditionalFilteringKey) throws AlgebricksException {
+            LogicalVariable prevAdditionalFilteringKey, List<List<AlgebricksPipeline>> secondaryKeysPipelines,
+            IOperatorSchema pipelineTopSchema) throws AlgebricksException {
         String indexName = dataSourceIndex.getId();
         DataverseName dataverseName = dataSourceIndex.getDataSource().getId().getDataverseName();
         String datasetName = dataSourceIndex.getDataSource().getId().getDatasourceName();
@@ -1199,12 +1222,36 @@
             prevAdditionalFilteringKeys = new ArrayList<>();
             prevAdditionalFilteringKeys.add(prevAdditionalFilteringKey);
         }
-        AsterixTupleFilterFactory filterFactory = createTupleFilterFactory(inputSchemas, typeEnv, filterExpr, context);
+
+        // If we have a pipeline, then we need to pass the schema of the pipeline to the filter factory.
+        AsterixTupleFilterFactory filterFactory;
+        if (pipelineTopSchema != null) {
+            IOperatorSchema[] schemasForFilterFactory = new IOperatorSchema[inputSchemas.length + 1];
+            System.arraycopy(inputSchemas, 0, schemasForFilterFactory, 0, inputSchemas.length);
+            schemasForFilterFactory[inputSchemas.length] = pipelineTopSchema;
+            filterFactory = createTupleFilterFactory(schemasForFilterFactory, typeEnv, filterExpr, context);
+
+        } else {
+            filterFactory = createTupleFilterFactory(inputSchemas, typeEnv, filterExpr, context);
+        }
+
         switch (secondaryIndex.getIndexType()) {
             case BTREE:
                 return getBTreeRuntime(dataverseName, datasetName, indexName, propagatedSchema, primaryKeys,
                         secondaryKeys, additionalNonKeyFields, filterFactory, inputRecordDesc, context, spec, indexOp,
                         bulkload, upsertIndicatorVar, prevSecondaryKeys, prevAdditionalFilteringKeys);
+            case ARRAY:
+                if (bulkload) {
+                    // In the case of bulk-load, we do not handle any nested plans. We perform the exact same behavior
+                    // as a normal B-Tree bulk load.
+                    return getBTreeRuntime(dataverseName, datasetName, indexName, propagatedSchema, primaryKeys,
+                            secondaryKeys, additionalNonKeyFields, filterFactory, inputRecordDesc, context, spec,
+                            indexOp, bulkload, upsertIndicatorVar, prevSecondaryKeys, prevAdditionalFilteringKeys);
+                } else {
+                    return getArrayIndexRuntime(dataverseName, datasetName, indexName, propagatedSchema, primaryKeys,
+                            additionalNonKeyFields, inputRecordDesc, spec, indexOp, upsertIndicatorVar,
+                            secondaryKeysPipelines);
+                }
             case RTREE:
                 return getRTreeRuntime(dataverseName, datasetName, indexName, propagatedSchema, primaryKeys,
                         secondaryKeys, additionalNonKeyFields, filterFactory, inputRecordDesc, context, spec, indexOp,
@@ -1219,8 +1266,7 @@
                         prevAdditionalFilteringKeys);
             default:
                 throw new AlgebricksException(
-                        indexOp.name() + "Insert, upsert, and delete not implemented for index type: "
-                                + secondaryIndex.getIndexType());
+                        indexOp.name() + " not implemented for index type: " + secondaryIndex.getIndexType());
         }
     }
 
@@ -1311,6 +1357,60 @@
         }
     }
 
+    private Pair<IOperatorDescriptor, AlgebricksPartitionConstraint> getArrayIndexRuntime(DataverseName dataverseName,
+            String datasetName, String indexName, IOperatorSchema propagatedSchema, List<LogicalVariable> primaryKeys,
+            List<LogicalVariable> additionalNonKeyFields, RecordDescriptor inputRecordDesc, JobSpecification spec,
+            IndexOperation indexOp, LogicalVariable upsertIndicatorVar,
+            List<List<AlgebricksPipeline>> secondaryKeysPipelines) throws AlgebricksException {
+
+        Dataset dataset = MetadataManagerUtil.findExistingDataset(mdTxnCtx, dataverseName, datasetName);
+        int numPrimaryKeys = primaryKeys.size();
+        int numFilterFields = DatasetUtil.getFilterField(dataset) == null ? 0 : 1;
+
+        // Generate field permutations (this only includes primary keys and filter fields).
+        int[] fieldPermutation = new int[numPrimaryKeys + numFilterFields];
+        int[] modificationCallbackPrimaryKeyFields = new int[primaryKeys.size()];
+        int i = 0;
+        int j = 0;
+        for (LogicalVariable varKey : primaryKeys) {
+            int idx = propagatedSchema.findVariable(varKey);
+            fieldPermutation[i] = idx;
+            modificationCallbackPrimaryKeyFields[j] = i;
+            i++;
+            j++;
+        }
+        if (numFilterFields > 0) {
+            int idx = propagatedSchema.findVariable(additionalNonKeyFields.get(0));
+            fieldPermutation[numPrimaryKeys] = idx;
+        }
+
+        try {
+            // Index parameters.
+            Index secondaryIndex = MetadataManager.INSTANCE.getIndex(mdTxnCtx, dataset.getDataverseName(),
+                    dataset.getDatasetName(), indexName);
+            Pair<IFileSplitProvider, AlgebricksPartitionConstraint> splitsAndConstraint =
+                    getSplitProviderAndConstraints(dataset, secondaryIndex.getIndexName());
+            // Prepare callback.
+            IModificationOperationCallbackFactory modificationCallbackFactory = dataset.getModificationCallbackFactory(
+                    storageComponentProvider, secondaryIndex, indexOp, modificationCallbackPrimaryKeyFields);
+            IIndexDataflowHelperFactory idfh = new IndexDataflowHelperFactory(
+                    storageComponentProvider.getStorageManager(), splitsAndConstraint.first);
+            IOperatorDescriptor op;
+            if (indexOp == IndexOperation.UPSERT) {
+                int upsertIndicatorFieldIndex = propagatedSchema.findVariable(upsertIndicatorVar);
+                op = new LSMSecondaryUpsertWithNestedPlanOperatorDescriptor(spec, inputRecordDesc, fieldPermutation,
+                        idfh, modificationCallbackFactory, upsertIndicatorFieldIndex, BinaryBooleanInspector.FACTORY,
+                        secondaryKeysPipelines.get(0), secondaryKeysPipelines.get(1));
+            } else {
+                op = new LSMSecondaryInsertDeleteWithNestedPlanOperatorDescriptor(spec, inputRecordDesc,
+                        fieldPermutation, indexOp, idfh, modificationCallbackFactory, secondaryKeysPipelines.get(0));
+            }
+            return new Pair<>(op, splitsAndConstraint.second);
+        } catch (Exception e) {
+            throw new AlgebricksException(e);
+        }
+    }
+
     private Pair<IOperatorDescriptor, AlgebricksPartitionConstraint> getRTreeRuntime(DataverseName dataverseName,
             String datasetName, String indexName, IOperatorSchema propagatedSchema, List<LogicalVariable> primaryKeys,
             List<LogicalVariable> secondaryKeys, List<LogicalVariable> additionalNonKeyFields,
@@ -1326,8 +1426,9 @@
         ARecordType recType = (ARecordType) itemType;
         Index secondaryIndex = MetadataManager.INSTANCE.getIndex(mdTxnCtx, dataset.getDataverseName(),
                 dataset.getDatasetName(), indexName);
-        List<List<String>> secondaryKeyExprs = secondaryIndex.getKeyFieldNames();
-        List<IAType> secondaryKeyTypes = secondaryIndex.getKeyFieldTypes();
+        Index.ValueIndexDetails secondaryIndexDetails = (Index.ValueIndexDetails) secondaryIndex.getIndexDetails();
+        List<List<String>> secondaryKeyExprs = secondaryIndexDetails.getKeyFieldNames();
+        List<IAType> secondaryKeyTypes = secondaryIndexDetails.getKeyFieldTypes();
         Pair<IAType, Boolean> keyPairType =
                 Index.getNonNullableOpenFieldType(secondaryKeyTypes.get(0), secondaryKeyExprs.get(0), recType);
         IAType spatialType = keyPairType.first;
@@ -1626,9 +1727,10 @@
             // Index parameters.
             Index secondaryIndex = MetadataManager.INSTANCE.getIndex(mdTxnCtx, dataset.getDataverseName(),
                     dataset.getDatasetName(), indexName);
+            Index.TextIndexDetails secondaryIndexDetails = (Index.TextIndexDetails) secondaryIndex.getIndexDetails();
 
-            List<List<String>> secondaryKeyExprs = secondaryIndex.getKeyFieldNames();
-            List<IAType> secondaryKeyTypeEntries = secondaryIndex.getKeyFieldTypes();
+            List<List<String>> secondaryKeyExprs = secondaryIndexDetails.getKeyFieldNames();
+            List<IAType> secondaryKeyTypeEntries = secondaryIndexDetails.getKeyFieldTypes();
 
             int numTokenFields = (!isPartitioned) ? secondaryKeys.size() : secondaryKeys.size() + 1;
             ITypeTraits[] tokenTypeTraits = new ITypeTraits[numTokenFields];
@@ -1657,10 +1759,10 @@
             }
 
             IBinaryTokenizerFactory tokenizerFactory = NonTaggedFormatUtil.getBinaryTokenizerFactory(
-                    secondaryKeyType.getTypeTag(), indexType, secondaryIndex.getGramLength());
+                    secondaryKeyType.getTypeTag(), indexType, secondaryIndexDetails.getGramLength());
             IFullTextConfigEvaluatorFactory fullTextConfigEvaluatorFactory =
                     FullTextUtil.fetchFilterAndCreateConfigEvaluator(this, secondaryIndex.getDataverseName(),
-                            secondaryIndex.getFullTextConfigName());
+                            secondaryIndexDetails.getFullTextConfigName());
 
             Pair<IFileSplitProvider, AlgebricksPartitionConstraint> splitsAndConstraint =
                     getSplitProviderAndConstraints(dataset, secondaryIndex.getIndexName());
diff --git a/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/entities/Dataset.java b/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/entities/Dataset.java
index dd3f67d..f547516 100644
--- a/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/entities/Dataset.java
+++ b/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/entities/Dataset.java
@@ -54,6 +54,7 @@
 import org.apache.asterix.metadata.MetadataManager;
 import org.apache.asterix.metadata.MetadataTransactionContext;
 import org.apache.asterix.metadata.api.IMetadataEntity;
+import org.apache.asterix.metadata.declared.ArrayBTreeResourceFactoryProvider;
 import org.apache.asterix.metadata.declared.BTreeResourceFactoryProvider;
 import org.apache.asterix.metadata.declared.MetadataProvider;
 import org.apache.asterix.metadata.lock.ExternalDatasetsRegistry;
@@ -471,6 +472,11 @@
                 recordType, metaType, mdProvider.getStorageComponentProvider().getComparatorFactoryProvider());
         IResourceFactory resourceFactory;
         switch (index.getIndexType()) {
+            case ARRAY:
+                resourceFactory = ArrayBTreeResourceFactoryProvider.INSTANCE.getResourceFactory(mdProvider, this, index,
+                        recordType, metaType, mergePolicyFactory, mergePolicyProperties, filterTypeTraits,
+                        filterCmpFactories);
+                break;
             case BTREE:
                 resourceFactory = BTreeResourceFactoryProvider.INSTANCE.getResourceFactory(mdProvider, this, index,
                         recordType, metaType, mergePolicyFactory, mergePolicyProperties, filterTypeTraits,
@@ -574,7 +580,7 @@
             return new SecondaryIndexInstanctSearchOperationCallbackFactory(getDatasetId(),
                     primaryKeyFieldsInSecondaryIndex, storageComponentProvider.getTransactionSubsystemProvider(),
                     index.resourceType());
-        } else if (index.getKeyFieldNames().isEmpty()) {
+        } else if (index.isPrimaryKeyIndex()) {
             // this is the case where the index is secondary primary index and locking is required
             // since the secondary primary index replaces the dataset index (which locks)
             return new PrimaryIndexInstantSearchOperationCallbackFactory(getDatasetId(), primaryKeyFields,
diff --git a/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/entities/Index.java b/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/entities/Index.java
index c8133e0..cf09779 100644
--- a/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/entities/Index.java
+++ b/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/entities/Index.java
@@ -19,7 +19,10 @@
 
 package org.apache.asterix.metadata.entities;
 
+import java.io.Serializable;
+import java.util.Collections;
 import java.util.List;
+import java.util.Objects;
 
 import org.apache.asterix.common.config.DatasetConfig.IndexType;
 import org.apache.asterix.common.exceptions.AsterixException;
@@ -43,6 +46,7 @@
 
     private static final long serialVersionUID = 3L;
     public static final int RECORD_INDICATOR = 0;
+    public static final int META_RECORD_INDICATOR = 1;
 
     private final DataverseName dataverseName;
     // Enforced to be unique within a dataverse.
@@ -50,46 +54,38 @@
     // Enforced to be unique within a dataverse, dataset combination.
     private final String indexName;
     private final IndexType indexType;
-    private final List<List<String>> keyFieldNames;
-    private final List<Integer> keyFieldSourceIndicators;
-    private final List<IAType> keyFieldTypes;
-    private final boolean overrideKeyFieldTypes;
-    private final boolean isEnforced;
+    private final IIndexDetails indexDetails;
     private final boolean isPrimaryIndex;
-    // Specific to NGRAM indexes.
-    private final int gramLength;
-    // ToDo: to allow index to access the full-text config in another dataverse,
-    //   maybe we need to add a new field here fullTextConfigDataverseName for the dataverse name of the full-text config
-    // Specific to FullText indexes.
-    private final String fullTextConfigName;
+    private final boolean isEnforced;
     // Type of pending operations with respect to atomic DDL operation
     private int pendingOp;
 
     public Index(DataverseName dataverseName, String datasetName, String indexName, IndexType indexType,
-            List<List<String>> keyFieldNames, List<Integer> keyFieldSourceIndicators, List<IAType> keyFieldTypes,
-            int gramLength, String fullTextConfigName, boolean overrideKeyFieldTypes, boolean isEnforced,
-            boolean isPrimaryIndex, int pendingOp) {
-        this.dataverseName = dataverseName;
-        this.datasetName = datasetName;
-        this.indexName = indexName;
+            IIndexDetails indexDetails, boolean isEnforced, boolean isPrimaryIndex, int pendingOp) {
+        boolean categoryOk = (indexType == null && indexDetails == null) || (IndexCategory
+                .of(Objects.requireNonNull(indexType)) == ((AbstractIndexDetails) Objects.requireNonNull(indexDetails))
+                        .getIndexCategory());
+        if (!categoryOk) {
+            throw new IllegalArgumentException();
+        }
+        this.dataverseName = Objects.requireNonNull(dataverseName);
+        this.datasetName = Objects.requireNonNull(datasetName);
+        this.indexName = Objects.requireNonNull(indexName);
         this.indexType = indexType;
-        this.keyFieldNames = keyFieldNames;
-        this.keyFieldSourceIndicators = keyFieldSourceIndicators;
-        this.keyFieldTypes = keyFieldTypes;
-        this.gramLength = gramLength;
-        this.fullTextConfigName = fullTextConfigName;
-
-        this.overrideKeyFieldTypes = overrideKeyFieldTypes;
-        this.isEnforced = isEnforced;
+        this.indexDetails = indexDetails;
         this.isPrimaryIndex = isPrimaryIndex;
+        this.isEnforced = isEnforced;
         this.pendingOp = pendingOp;
     }
 
+    @Deprecated
     public Index(DataverseName dataverseName, String datasetName, String indexName, IndexType indexType,
             List<List<String>> keyFieldNames, List<Integer> keyFieldSourceIndicators, List<IAType> keyFieldTypes,
             boolean overrideKeyFieldTypes, boolean isEnforced, boolean isPrimaryIndex, int pendingOp) {
-        this(dataverseName, datasetName, indexName, indexType, keyFieldNames, keyFieldSourceIndicators, keyFieldTypes,
-                -1, null, overrideKeyFieldTypes, isEnforced, isPrimaryIndex, pendingOp);
+        this(dataverseName,
+                datasetName, indexName, indexType, createSimpleIndexDetails(indexType, keyFieldNames,
+                        keyFieldSourceIndicators, keyFieldTypes, overrideKeyFieldTypes),
+                isEnforced, isPrimaryIndex, pendingOp);
     }
 
     public DataverseName getDataverseName() {
@@ -104,26 +100,6 @@
         return indexName;
     }
 
-    public List<List<String>> getKeyFieldNames() {
-        return keyFieldNames;
-    }
-
-    public List<Integer> getKeyFieldSourceIndicators() {
-        return keyFieldSourceIndicators;
-    }
-
-    public List<IAType> getKeyFieldTypes() {
-        return keyFieldTypes;
-    }
-
-    public int getGramLength() {
-        return gramLength;
-    }
-
-    public String getFullTextConfigName() {
-        return fullTextConfigName;
-    }
-
     public IndexType getIndexType() {
         return indexType;
     }
@@ -132,8 +108,8 @@
         return isPrimaryIndex;
     }
 
-    public boolean isOverridingKeyFieldTypes() {
-        return overrideKeyFieldTypes;
+    public IIndexDetails getIndexDetails() {
+        return indexDetails;
     }
 
     public boolean isEnforced() {
@@ -154,7 +130,7 @@
 
     public boolean isPrimaryKeyIndex() {
         // a primary key index has no key field names
-        return keyFieldNames.isEmpty();
+        return indexType == IndexType.BTREE && ((ValueIndexDetails) indexDetails).keyFieldNames.isEmpty();
     }
 
     public static Pair<IAType, Boolean> getNonNullableType(IAType keyType) {
@@ -204,9 +180,7 @@
     }
 
     private static IAType keyFieldType(List<String> expr, ARecordType recType) throws AlgebricksException {
-        IAType fieldType = recType;
-        fieldType = recType.getSubFieldType(expr);
-        return fieldType;
+        return recType.getSubFieldType(expr);
     }
 
     @Override
@@ -280,19 +254,9 @@
         return dataverseName.compareTo(otherIndex.getDataverseName());
     }
 
-    public boolean hasMetaFields() {
-        if (keyFieldSourceIndicators != null) {
-            for (Integer indicator : keyFieldSourceIndicators) {
-                if (indicator.intValue() != 0) {
-                    return true;
-                }
-            }
-        }
-        return false;
-    }
-
     public byte resourceType() throws CompilationException {
         switch (indexType) {
+            case ARRAY:
             case BTREE:
                 return ResourceType.LSM_BTREE;
             case RTREE:
@@ -311,4 +275,225 @@
     public String toString() {
         return dataverseName + "." + datasetName + "." + indexName;
     }
+
+    public enum IndexCategory {
+        VALUE,
+        TEXT,
+        ARRAY;
+
+        public static IndexCategory of(IndexType indexType) {
+            switch (indexType) {
+                case BTREE:
+                case RTREE:
+                    return VALUE;
+                case SINGLE_PARTITION_WORD_INVIX:
+                case SINGLE_PARTITION_NGRAM_INVIX:
+                case LENGTH_PARTITIONED_WORD_INVIX:
+                case LENGTH_PARTITIONED_NGRAM_INVIX:
+                    return TEXT;
+                case ARRAY:
+                    return ARRAY;
+                default:
+                    throw new IllegalArgumentException(String.valueOf(indexType));
+            }
+        }
+    }
+
+    public interface IIndexDetails extends Serializable {
+        boolean isOverridingKeyFieldTypes();
+    }
+
+    static abstract class AbstractIndexDetails implements IIndexDetails {
+
+        private static final long serialVersionUID = 1L;
+
+        abstract IndexCategory getIndexCategory();
+    }
+
+    public static final class ValueIndexDetails extends AbstractIndexDetails {
+
+        private static final long serialVersionUID = 1L;
+
+        private final List<List<String>> keyFieldNames;
+
+        private final List<Integer> keyFieldSourceIndicators;
+
+        private final List<IAType> keyFieldTypes;
+
+        private final boolean overrideKeyFieldTypes;
+
+        public ValueIndexDetails(List<List<String>> keyFieldNames, List<Integer> keyFieldSourceIndicators,
+                List<IAType> keyFieldTypes, boolean overrideKeyFieldTypes) {
+            this.keyFieldNames = keyFieldNames;
+            this.keyFieldSourceIndicators = keyFieldSourceIndicators;
+            this.keyFieldTypes = keyFieldTypes;
+            this.overrideKeyFieldTypes = overrideKeyFieldTypes;
+        }
+
+        @Override
+        IndexCategory getIndexCategory() {
+            return IndexCategory.VALUE;
+        }
+
+        public List<List<String>> getKeyFieldNames() {
+            return keyFieldNames;
+        }
+
+        public List<Integer> getKeyFieldSourceIndicators() {
+            return keyFieldSourceIndicators;
+        }
+
+        public List<IAType> getKeyFieldTypes() {
+            return keyFieldTypes;
+        }
+
+        @Override
+        public boolean isOverridingKeyFieldTypes() {
+            return overrideKeyFieldTypes;
+        }
+    }
+
+    public static final class TextIndexDetails extends AbstractIndexDetails {
+
+        private static final long serialVersionUID = 1L;
+
+        private final List<List<String>> keyFieldNames;
+
+        private final List<Integer> keyFieldSourceIndicators;
+
+        private final List<IAType> keyFieldTypes;
+
+        private final boolean overrideKeyFieldTypes;
+
+        // ToDo: to allow index to access the full-text config in another dataverse,
+        //   maybe we need to add a new field here fullTextConfigDataverseName for dataverse name of full-text config
+        // Specific to FullText indexes.
+        private final String fullTextConfigName;
+
+        // Specific to NGRAM indexes.
+        private final int gramLength;
+
+        public TextIndexDetails(List<List<String>> keyFieldNames, List<Integer> keyFieldSourceIndicators,
+                List<IAType> keyFieldTypes, boolean overrideKeyFieldTypes, int gramLength, String fullTextConfigName) {
+            this.keyFieldNames = keyFieldNames;
+            this.keyFieldTypes = keyFieldTypes;
+            this.keyFieldSourceIndicators = keyFieldSourceIndicators;
+            this.overrideKeyFieldTypes = overrideKeyFieldTypes;
+            this.gramLength = gramLength;
+            this.fullTextConfigName = fullTextConfigName;
+        }
+
+        @Override
+        IndexCategory getIndexCategory() {
+            return IndexCategory.TEXT;
+        }
+
+        public List<List<String>> getKeyFieldNames() {
+            return keyFieldNames;
+        }
+
+        public List<Integer> getKeyFieldSourceIndicators() {
+            return keyFieldSourceIndicators;
+        }
+
+        public List<IAType> getKeyFieldTypes() {
+            return keyFieldTypes;
+        }
+
+        @Override
+        public boolean isOverridingKeyFieldTypes() {
+            return overrideKeyFieldTypes;
+        }
+
+        public int getGramLength() {
+            return gramLength;
+        }
+
+        public String getFullTextConfigName() {
+            return fullTextConfigName;
+        }
+    }
+
+    public static class ArrayIndexDetails extends AbstractIndexDetails {
+
+        private static final long serialVersionUID = 1L;
+
+        private final List<ArrayIndexElement> elementList;
+
+        private final boolean overrideKeyFieldTypes;
+
+        public ArrayIndexDetails(List<ArrayIndexElement> elementList, boolean overrideKeyFieldTypes) {
+            this.elementList = elementList;
+            this.overrideKeyFieldTypes = overrideKeyFieldTypes;
+        }
+
+        @Override
+        IndexCategory getIndexCategory() {
+            return IndexCategory.ARRAY;
+        }
+
+        public List<ArrayIndexElement> getElementList() {
+            return elementList;
+        }
+
+        @Override
+        public boolean isOverridingKeyFieldTypes() {
+            return overrideKeyFieldTypes;
+        }
+    }
+
+    public static final class ArrayIndexElement implements Serializable {
+
+        private static final long serialVersionUID = 1L;
+
+        private final List<List<String>> unnestList;
+
+        private final List<List<String>> projectList;
+
+        private final List<IAType> typeList;
+
+        private final int sourceIndicator;
+
+        public ArrayIndexElement(List<List<String>> unnestList, List<List<String>> projectList, List<IAType> typeList,
+                int sourceIndicator) {
+            this.unnestList = unnestList != null ? unnestList : Collections.emptyList();
+            this.projectList = projectList;
+            this.typeList = typeList;
+            this.sourceIndicator = sourceIndicator;
+        }
+
+        public List<List<String>> getUnnestList() {
+            return unnestList;
+        }
+
+        public List<List<String>> getProjectList() {
+            return projectList;
+        }
+
+        public List<IAType> getTypeList() {
+            return typeList;
+        }
+
+        public int getSourceIndicator() {
+            return sourceIndicator;
+        }
+    }
+
+    @Deprecated
+    private static Index.IIndexDetails createSimpleIndexDetails(IndexType indexType, List<List<String>> keyFieldNames,
+            List<Integer> keyFieldSourceIndicators, List<IAType> keyFieldTypes, boolean overrideKeyFieldTypes) {
+        if (indexType == null) {
+            return null;
+        }
+        switch (Index.IndexCategory.of(indexType)) {
+            case VALUE:
+                return new ValueIndexDetails(keyFieldNames, keyFieldSourceIndicators, keyFieldTypes,
+                        overrideKeyFieldTypes);
+            case TEXT:
+                return new TextIndexDetails(keyFieldNames, keyFieldSourceIndicators, keyFieldTypes,
+                        overrideKeyFieldTypes, -1, null);
+            default:
+                throw new IllegalArgumentException(String.valueOf(indexType));
+        }
+    }
 }
diff --git a/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/entitytupletranslators/IndexTupleTranslator.java b/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/entitytupletranslators/IndexTupleTranslator.java
index c69c0b1..15f3a8b 100644
--- a/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/entitytupletranslators/IndexTupleTranslator.java
+++ b/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/entitytupletranslators/IndexTupleTranslator.java
@@ -21,10 +21,16 @@
 
 import java.util.ArrayList;
 import java.util.Calendar;
+import java.util.Collections;
 import java.util.List;
+import java.util.stream.Collectors;
 
+import org.apache.asterix.builders.IARecordBuilder;
 import org.apache.asterix.builders.OrderedListBuilder;
+import org.apache.asterix.builders.RecordBuilder;
 import org.apache.asterix.common.config.DatasetConfig.IndexType;
+import org.apache.asterix.common.exceptions.AsterixException;
+import org.apache.asterix.common.exceptions.ErrorCode;
 import org.apache.asterix.common.metadata.DataverseName;
 import org.apache.asterix.common.transactions.TxnId;
 import org.apache.asterix.formats.nontagged.SerializerDeserializerProvider;
@@ -40,15 +46,21 @@
 import org.apache.asterix.om.base.AInt32;
 import org.apache.asterix.om.base.AInt8;
 import org.apache.asterix.om.base.AMutableInt8;
+import org.apache.asterix.om.base.ANull;
 import org.apache.asterix.om.base.AOrderedList;
 import org.apache.asterix.om.base.ARecord;
 import org.apache.asterix.om.base.AString;
 import org.apache.asterix.om.base.IACursor;
+import org.apache.asterix.om.base.IAObject;
 import org.apache.asterix.om.types.AOrderedListType;
 import org.apache.asterix.om.types.ARecordType;
+import org.apache.asterix.om.types.ATypeTag;
 import org.apache.asterix.om.types.BuiltinType;
 import org.apache.asterix.om.types.IAType;
+import org.apache.asterix.om.utils.RecordUtil;
 import org.apache.hyracks.algebricks.common.exceptions.AlgebricksException;
+import org.apache.hyracks.algebricks.common.utils.Pair;
+import org.apache.hyracks.algebricks.common.utils.Triple;
 import org.apache.hyracks.api.dataflow.value.ISerializerDeserializer;
 import org.apache.hyracks.api.exceptions.HyracksDataException;
 import org.apache.hyracks.data.std.util.ArrayBackedValueStorage;
@@ -70,18 +82,25 @@
     public static final String INDEX_SEARCHKEY_TYPE_FIELD_NAME = "SearchKeyType";
     public static final String INDEX_ISENFORCED_FIELD_NAME = "IsEnforced";
     public static final String INDEX_SEARCHKEY_SOURCE_INDICATOR_FIELD_NAME = "SearchKeySourceIndicator";
+    public static final String INDEX_SEARCHKEY_ELEMENTS_FIELD_NAME = "SearchKeyElements";
+    public static final String COMPLEXSEARCHKEY_UNNEST_FIELD_NAME = "UnnestList";
+    public static final String COMPLEXSEARCHKEY_PROJECT_FIELD_NAME = "ProjectList";
 
     protected final TxnId txnId;
     protected final MetadataNode metadataNode;
 
     protected OrderedListBuilder listBuilder;
+    protected OrderedListBuilder innerListBuilder;
     protected OrderedListBuilder primaryKeyListBuilder;
+    protected OrderedListBuilder complexSearchKeyNameListBuilder;
+    protected IARecordBuilder complexSearchKeyNameRecordBuilder;
     protected AOrderedListType stringList;
     protected AOrderedListType int8List;
     protected ArrayBackedValueStorage nameValue;
     protected ArrayBackedValueStorage itemValue;
     protected AMutableInt8 aInt8;
     protected ISerializerDeserializer<AInt8> int8Serde;
+    protected ISerializerDeserializer<ANull> nullSerde;
 
     @SuppressWarnings("unchecked")
     protected IndexTupleTranslator(TxnId txnId, MetadataNode metadataNode, boolean getTuple) {
@@ -90,13 +109,17 @@
         this.metadataNode = metadataNode;
         if (getTuple) {
             listBuilder = new OrderedListBuilder();
+            innerListBuilder = new OrderedListBuilder();
             primaryKeyListBuilder = new OrderedListBuilder();
+            complexSearchKeyNameRecordBuilder = new RecordBuilder();
+            complexSearchKeyNameListBuilder = new OrderedListBuilder();
             stringList = new AOrderedListType(BuiltinType.ASTRING, null);
             int8List = new AOrderedListType(BuiltinType.AINT8, null);
             nameValue = new ArrayBackedValueStorage();
             itemValue = new ArrayBackedValueStorage();
             aInt8 = new AMutableInt8((byte) 0);
             int8Serde = SerializerDeserializerProvider.INSTANCE.getSerializerDeserializer(BuiltinType.AINT8);
+            nullSerde = SerializerDeserializerProvider.INSTANCE.getSerializerDeserializer(BuiltinType.ANULL);
         }
     }
 
@@ -112,52 +135,107 @@
         String indexName =
                 ((AString) indexRecord.getValueByPos(MetadataRecordTypes.INDEX_ARECORD_INDEXNAME_FIELD_INDEX))
                         .getStringValue();
-        IndexType indexStructure = IndexType.valueOf(
+        IndexType indexType = IndexType.valueOf(
                 ((AString) indexRecord.getValueByPos(MetadataRecordTypes.INDEX_ARECORD_INDEXSTRUCTURE_FIELD_INDEX))
                         .getStringValue());
-        IACursor fieldNameCursor =
-                ((AOrderedList) indexRecord.getValueByPos(MetadataRecordTypes.INDEX_ARECORD_SEARCHKEY_FIELD_INDEX))
-                        .getCursor();
-        List<List<String>> searchKey = new ArrayList<>();
-        AOrderedList fieldNameList;
-        while (fieldNameCursor.next()) {
-            fieldNameList = (AOrderedList) fieldNameCursor.get();
-            IACursor nestedFieldNameCursor = (fieldNameList.getCursor());
-            List<String> nestedFieldName = new ArrayList<>();
-            while (nestedFieldNameCursor.next()) {
-                nestedFieldName.add(((AString) nestedFieldNameCursor.get()).getStringValue());
-            }
-            searchKey.add(nestedFieldName);
-        }
-        int indexKeyTypeFieldPos = indexRecord.getType().getFieldIndex(INDEX_SEARCHKEY_TYPE_FIELD_NAME);
-        IACursor fieldTypeCursor = new ACollectionCursor();
-        if (indexKeyTypeFieldPos > 0) {
-            fieldTypeCursor = ((AOrderedList) indexRecord.getValueByPos(indexKeyTypeFieldPos)).getCursor();
-        }
-        List<IAType> searchKeyType = new ArrayList<>(searchKey.size());
-        while (fieldTypeCursor.next()) {
-            String typeName = ((AString) fieldTypeCursor.get()).getStringValue();
-            IAType fieldType = BuiltinTypeMap.getTypeFromTypeName(metadataNode, txnId, dataverseName, typeName);
-            searchKeyType.add(fieldType);
-        }
-        boolean isOverridingKeyTypes = !searchKeyType.isEmpty();
 
-        int isEnforcedFieldPos = indexRecord.getType().getFieldIndex(INDEX_ISENFORCED_FIELD_NAME);
-        Boolean isEnforcingKeys = false;
-        if (isEnforcedFieldPos > 0) {
-            isEnforcingKeys = ((ABoolean) indexRecord.getValueByPos(isEnforcedFieldPos)).getBoolean();
+        // Read key names
+        List<Pair<List<List<String>>, List<List<String>>>> searchElements = new ArrayList<>();
+        switch (Index.IndexCategory.of(indexType)) {
+            case VALUE:
+            case TEXT:
+                // Read the key names from the SearchKeyName field
+                IACursor fieldNameCursor = ((AOrderedList) indexRecord
+                        .getValueByPos(MetadataRecordTypes.INDEX_ARECORD_SEARCHKEY_FIELD_INDEX)).getCursor();
+                AOrderedList fieldNameList;
+                while (fieldNameCursor.next()) {
+                    fieldNameList = (AOrderedList) fieldNameCursor.get();
+                    IACursor nestedFieldNameCursor = (fieldNameList.getCursor());
+                    List<String> nestedFieldName = new ArrayList<>();
+                    while (nestedFieldNameCursor.next()) {
+                        nestedFieldName.add(((AString) nestedFieldNameCursor.get()).getStringValue());
+                    }
+                    searchElements.add(new Pair<>(null, Collections.singletonList(nestedFieldName)));
+                }
+                break;
+            case ARRAY:
+                // Read the unnest/project from the ComplexSearchKeyName field
+                int complexSearchKeyFieldPos = indexRecord.getType().getFieldIndex(INDEX_SEARCHKEY_ELEMENTS_FIELD_NAME);
+                IACursor complexSearchKeyCursor = new ACollectionCursor();
+                if (complexSearchKeyFieldPos > 0) {
+                    complexSearchKeyCursor =
+                            ((AOrderedList) indexRecord.getValueByPos(complexSearchKeyFieldPos)).getCursor();
+                }
+                while (complexSearchKeyCursor.next()) {
+                    Pair<List<List<String>>, List<List<String>>> searchElement;
+                    IAObject complexSearchKeyItem = complexSearchKeyCursor.get();
+                    switch (complexSearchKeyItem.getType().getTypeTag()) {
+                        case ARRAY:
+                            AOrderedList complexSearchKeyArray = (AOrderedList) complexSearchKeyItem;
+                            List<String> project = new ArrayList<>(complexSearchKeyArray.size());
+                            // We only have one element.
+                            AOrderedList innerListForArray = (AOrderedList) complexSearchKeyArray.getItem(0);
+                            IACursor innerListCursorForArray = innerListForArray.getCursor();
+                            while (innerListCursorForArray.next()) {
+                                project.add(((AString) innerListCursorForArray.get()).getStringValue());
+                            }
+                            searchElement = new Pair<>(null, Collections.singletonList(project));
+                            break;
+                        case OBJECT:
+                            ARecord complexSearchKeyRecord = (ARecord) complexSearchKeyItem;
+                            ARecordType complexSearchKeyRecordType = complexSearchKeyRecord.getType();
+                            int unnestFieldPos =
+                                    complexSearchKeyRecordType.getFieldIndex(COMPLEXSEARCHKEY_UNNEST_FIELD_NAME);
+                            if (unnestFieldPos < 0) {
+                                throw new AsterixException(ErrorCode.METADATA_ERROR, complexSearchKeyRecord.toJSON());
+                            }
+                            AOrderedList unnestFieldList =
+                                    (AOrderedList) complexSearchKeyRecord.getValueByPos(unnestFieldPos);
+                            List<List<String>> unnestList = new ArrayList<>(unnestFieldList.size());
+                            IACursor unnestFieldListCursor = unnestFieldList.getCursor();
+                            while (unnestFieldListCursor.next()) {
+                                AOrderedList innerList = (AOrderedList) unnestFieldListCursor.get();
+                                List<String> unnestPath = new ArrayList<>(innerList.size());
+                                IACursor innerListCursor = innerList.getCursor();
+                                while (innerListCursor.next()) {
+                                    unnestPath.add(((AString) innerListCursor.get()).getStringValue());
+                                }
+                                unnestList.add(unnestPath);
+                            }
+                            int projectFieldPos =
+                                    complexSearchKeyRecordType.getFieldIndex(COMPLEXSEARCHKEY_PROJECT_FIELD_NAME);
+                            if (projectFieldPos < 0) {
+                                throw new AsterixException(ErrorCode.METADATA_ERROR, complexSearchKeyRecord.toJSON());
+                            }
+                            AOrderedList projectFieldList =
+                                    (AOrderedList) complexSearchKeyRecord.getValueByPos(projectFieldPos);
+                            List<List<String>> projectList = new ArrayList<>(projectFieldList.size());
+                            IACursor projectFieldListCursor = projectFieldList.getCursor();
+                            while (projectFieldListCursor.next()) {
+                                if (projectFieldListCursor.get().getType().getTypeTag().equals(ATypeTag.NULL)) {
+                                    projectList.add(null);
+                                    break;
+                                }
+                                AOrderedList innerList = (AOrderedList) projectFieldListCursor.get();
+                                List<String> projectPath = new ArrayList<>(innerList.size());
+                                IACursor innerListCursor = innerList.getCursor();
+                                while (innerListCursor.next()) {
+                                    projectPath.add(((AString) innerListCursor.get()).getStringValue());
+                                }
+                                projectList.add(projectPath);
+                            }
+                            searchElement = new Pair<>(unnestList, projectList);
+                            break;
+                        default:
+                            throw new AsterixException(ErrorCode.METADATA_ERROR, complexSearchKeyItem.toJSON());
+                    }
+                    searchElements.add(searchElement);
+                }
+                break;
+            default:
+                throw new AsterixException(ErrorCode.METADATA_ERROR, indexType.toString());
         }
-        Boolean isPrimaryIndex =
-                ((ABoolean) indexRecord.getValueByPos(MetadataRecordTypes.INDEX_ARECORD_ISPRIMARY_FIELD_INDEX))
-                        .getBoolean();
-        int pendingOp = ((AInt32) indexRecord.getValueByPos(MetadataRecordTypes.INDEX_ARECORD_PENDINGOP_FIELD_INDEX))
-                .getIntegerValue();
-        // Check if there is a gram length as well.
-        int gramLength = -1;
-        int gramLenPos = indexRecord.getType().getFieldIndex(GRAM_LENGTH_FIELD_NAME);
-        if (gramLenPos >= 0) {
-            gramLength = ((AInt32) indexRecord.getValueByPos(gramLenPos)).getIntegerValue();
-        }
+        int searchElementCount = searchElements.size();
 
         String fullTextConfig = null;
         int fullTextConfigPos = indexRecord.getType().getFieldIndex(FULL_TEXT_CONFIG_FIELD_NAME);
@@ -166,7 +244,7 @@
         }
 
         // Read a field-source-indicator field.
-        List<Integer> keyFieldSourceIndicator = new ArrayList<>();
+        List<Integer> keyFieldSourceIndicator = new ArrayList<>(searchElementCount);
         int keyFieldSourceIndicatorIndex =
                 indexRecord.getType().getFieldIndex(INDEX_SEARCHKEY_SOURCE_INDICATOR_FIELD_NAME);
         if (keyFieldSourceIndicatorIndex >= 0) {
@@ -175,14 +253,47 @@
                 keyFieldSourceIndicator.add((int) ((AInt8) cursor.get()).getByteValue());
             }
         } else {
-            for (int index = 0; index < searchKey.size(); ++index) {
-                keyFieldSourceIndicator.add(0);
+            for (int index = 0; index < searchElementCount; ++index) {
+                keyFieldSourceIndicator.add(Index.RECORD_INDICATOR);
             }
         }
 
-        // index key type information is not persisted, thus we extract type information
-        // from the record metadata
+        // Read key types
+        int indexKeyTypeFieldPos = indexRecord.getType().getFieldIndex(INDEX_SEARCHKEY_TYPE_FIELD_NAME);
+        IACursor fieldTypeCursor = new ACollectionCursor();
+        if (indexKeyTypeFieldPos > 0) {
+            fieldTypeCursor = ((AOrderedList) indexRecord.getValueByPos(indexKeyTypeFieldPos)).getCursor();
+        }
+        List<List<IAType>> searchKeyType = new ArrayList<>(searchElementCount);
+        while (fieldTypeCursor.next()) {
+            IAObject fieldTypeItem = fieldTypeCursor.get();
+            switch (fieldTypeItem.getType().getTypeTag()) {
+                case STRING:
+                    // This is a simple element, place in a single-element list.
+                    String typeName = ((AString) fieldTypeItem).getStringValue();
+                    IAType fieldType = BuiltinTypeMap.getTypeFromTypeName(metadataNode, txnId, dataverseName, typeName);
+                    searchKeyType.add(Collections.singletonList(fieldType));
+                    break;
+                case ARRAY:
+                    // This is a complex element, read all types.
+                    List<IAType> fieldTypes = new ArrayList<>();
+                    AOrderedList fieldTypeList = (AOrderedList) fieldTypeItem;
+                    IACursor fieldTypeListCursor = fieldTypeList.getCursor();
+                    while (fieldTypeListCursor.next()) {
+                        typeName = ((AString) fieldTypeListCursor.get()).getStringValue();
+                        fieldTypes
+                                .add(BuiltinTypeMap.getTypeFromTypeName(metadataNode, txnId, dataverseName, typeName));
+                    }
+                    searchKeyType.add(fieldTypes);
+                    break;
+                default:
+                    throw new AsterixException(ErrorCode.METADATA_ERROR, fieldTypeItem.toJSON());
+            }
+        }
+        boolean isOverridingKeyTypes;
         if (searchKeyType.isEmpty()) {
+            // if index key type information is not persisted, then we extract type information
+            // from the record metadata
             Dataset dataset = metadataNode.getDataset(txnId, dataverseName, datasetName);
             String datatypeName = dataset.getItemTypeName();
             DataverseName datatypeDataverseName = dataset.getItemTypeDataverseName();
@@ -195,16 +306,127 @@
                 metaDt = (ARecordType) metadataNode.getDatatype(txnId, metatypeDataverseName, metatypeName)
                         .getDatatype();
             }
-            searchKeyType = KeyFieldTypeUtil.getKeyTypes(recordDt, metaDt, searchKey, keyFieldSourceIndicator);
+            searchKeyType = new ArrayList<>(searchElementCount);
+            for (int i = 0; i < searchElementCount; i++) {
+                Pair<List<List<String>>, List<List<String>>> searchElement = searchElements.get(i);
+                List<List<String>> unnestPathList = searchElement.first;
+                List<List<String>> projectPathList = searchElement.second;
+
+                ARecordType sourceRecordType = keyFieldSourceIndicator.get(i) == 1 ? metaDt : recordDt;
+                IAType inputTypePrime;
+                boolean inputTypeNullable, inputTypeMissable;
+                if (unnestPathList == null) {
+                    inputTypePrime = sourceRecordType;
+                    inputTypeNullable = inputTypeMissable = false;
+                } else {
+                    Triple<IAType, Boolean, Boolean> unnestTypeResult =
+                            KeyFieldTypeUtil.getKeyUnnestType(sourceRecordType, unnestPathList, null);
+                    if (unnestTypeResult == null) {
+                        inputTypePrime = null; // = ANY
+                        inputTypeNullable = inputTypeMissable = true;
+                    } else {
+                        inputTypePrime = unnestTypeResult.first;
+                        inputTypeNullable = unnestTypeResult.second;
+                        inputTypeMissable = unnestTypeResult.third;
+                    }
+                }
+
+                List<IAType> projectTypeList = new ArrayList<>(projectPathList.size());
+                for (List<String> projectPath : projectPathList) {
+                    IAType projectTypePrime;
+                    boolean projectTypeNullable, projectTypeMissable;
+                    if (projectPath == null) {
+                        projectTypePrime = inputTypePrime;
+                        projectTypeNullable = inputTypeNullable;
+                        projectTypeMissable = inputTypeMissable;
+                    } else if (inputTypePrime == null ||
+                    // handle special case of the empty field name in
+                    // ExternalIndexingOperations.FILE_INDEX_FIELD_NAMES
+                            (projectPath.size() == 1 && projectPath.get(0).isEmpty())) {
+                        projectTypePrime = null; // ANY
+                        projectTypeNullable = projectTypeMissable = true;
+                    } else {
+                        if (inputTypePrime.getTypeTag() != ATypeTag.OBJECT) {
+                            throw new AsterixException(ErrorCode.METADATA_ERROR, projectPath.toString());
+                        }
+                        Triple<IAType, Boolean, Boolean> projectTypeResult =
+                                KeyFieldTypeUtil.getKeyProjectType((ARecordType) inputTypePrime, projectPath, null);
+                        if (projectTypeResult == null) {
+                            throw new AsterixException(ErrorCode.METADATA_ERROR, projectPath.toString());
+                        }
+                        projectTypePrime = projectTypeResult.first;
+                        projectTypeNullable = inputTypeNullable || projectTypeResult.second;
+                        projectTypeMissable = inputTypeMissable || projectTypeResult.third;
+                    }
+                    IAType projectType = projectTypePrime == null ? null
+                            : KeyFieldTypeUtil.makeUnknownableType(projectTypePrime, projectTypeNullable,
+                                    projectTypeMissable);
+
+                    projectTypeList.add(projectType);
+                }
+
+                searchKeyType.add(projectTypeList);
+            }
+            isOverridingKeyTypes = false;
+        } else {
+            isOverridingKeyTypes = true;
         }
 
-        return new Index(dataverseName, datasetName, indexName, indexStructure, searchKey, keyFieldSourceIndicator,
-                searchKeyType, gramLength, fullTextConfig, isOverridingKeyTypes, isEnforcingKeys, isPrimaryIndex,
-                pendingOp);
+        // create index details structure
+        Index.IIndexDetails indexDetails;
+        switch (Index.IndexCategory.of(indexType)) {
+            case VALUE:
+                List<List<String>> keyFieldNames =
+                        searchElements.stream().map(Pair::getSecond).map(l -> l.get(0)).collect(Collectors.toList());
+                List<IAType> keyFieldTypes = searchKeyType.stream().map(l -> l.get(0)).collect(Collectors.toList());
+                indexDetails = new Index.ValueIndexDetails(keyFieldNames, keyFieldSourceIndicator, keyFieldTypes,
+                        isOverridingKeyTypes);
+                break;
+            case TEXT:
+                keyFieldNames =
+                        searchElements.stream().map(Pair::getSecond).map(l -> l.get(0)).collect(Collectors.toList());
+                keyFieldTypes = searchKeyType.stream().map(l -> l.get(0)).collect(Collectors.toList());
+                // Check if there is a gram length as well.
+                int gramLength = -1;
+                int gramLenPos = indexRecord.getType().getFieldIndex(GRAM_LENGTH_FIELD_NAME);
+                if (gramLenPos >= 0) {
+                    gramLength = ((AInt32) indexRecord.getValueByPos(gramLenPos)).getIntegerValue();
+                }
+                indexDetails = new Index.TextIndexDetails(keyFieldNames, keyFieldSourceIndicator, keyFieldTypes,
+                        isOverridingKeyTypes, gramLength, fullTextConfig);
+                break;
+            case ARRAY:
+                List<Index.ArrayIndexElement> elementList = new ArrayList<>(searchElementCount);
+                for (int i = 0; i < searchElementCount; i++) {
+                    Pair<List<List<String>>, List<List<String>>> searchElement = searchElements.get(i);
+                    List<IAType> typeList = searchKeyType.get(i);
+                    int sourceIndicator = keyFieldSourceIndicator.get(i);
+                    elementList.add(new Index.ArrayIndexElement(searchElement.first, searchElement.second, typeList,
+                            sourceIndicator));
+                }
+                indexDetails = new Index.ArrayIndexDetails(elementList, isOverridingKeyTypes);
+                break;
+            default:
+                throw new AsterixException(ErrorCode.METADATA_ERROR, indexType.toString());
+        }
+
+        int isEnforcedFieldPos = indexRecord.getType().getFieldIndex(INDEX_ISENFORCED_FIELD_NAME);
+        Boolean isEnforcingKeys = false;
+        if (isEnforcedFieldPos > 0) {
+            isEnforcingKeys = ((ABoolean) indexRecord.getValueByPos(isEnforcedFieldPos)).getBoolean();
+        }
+        Boolean isPrimaryIndex =
+                ((ABoolean) indexRecord.getValueByPos(MetadataRecordTypes.INDEX_ARECORD_ISPRIMARY_FIELD_INDEX))
+                        .getBoolean();
+        int pendingOp = ((AInt32) indexRecord.getValueByPos(MetadataRecordTypes.INDEX_ARECORD_PENDINGOP_FIELD_INDEX))
+                .getIntegerValue();
+
+        return new Index(dataverseName, datasetName, indexName, indexType, indexDetails, isEnforcingKeys,
+                isPrimaryIndex, pendingOp);
     }
 
     @Override
-    public ITupleReference getTupleFromMetadataEntity(Index index) throws HyracksDataException {
+    public ITupleReference getTupleFromMetadataEntity(Index index) throws HyracksDataException, AlgebricksException {
         String dataverseCanonicalName = index.getDataverseName().getCanonicalForm();
 
         // write the key in the first 3 fields of the tuple
@@ -241,15 +463,30 @@
         recordBuilder.addField(MetadataRecordTypes.INDEX_ARECORD_INDEXNAME_FIELD_INDEX, fieldValue);
 
         // write field 3
+        IndexType indexType = index.getIndexType();
         fieldValue.reset();
-        aString.setValue(index.getIndexType().toString());
+        aString.setValue(indexType.toString());
         stringSerde.serialize(aString, fieldValue.getDataOutput());
         recordBuilder.addField(MetadataRecordTypes.INDEX_ARECORD_INDEXSTRUCTURE_FIELD_INDEX, fieldValue);
 
         // write field 4
         primaryKeyListBuilder.reset((AOrderedListType) MetadataRecordTypes.INDEX_RECORDTYPE
                 .getFieldTypes()[MetadataRecordTypes.INDEX_ARECORD_SEARCHKEY_FIELD_INDEX]);
-        List<List<String>> searchKey = index.getKeyFieldNames();
+        List<List<String>> searchKey;
+        switch (Index.IndexCategory.of(indexType)) {
+            case VALUE:
+                searchKey = ((Index.ValueIndexDetails) index.getIndexDetails()).getKeyFieldNames();
+                break;
+            case TEXT:
+                searchKey = ((Index.TextIndexDetails) index.getIndexDetails()).getKeyFieldNames();
+                break;
+            case ARRAY:
+                // If we have a complex index, we persist all of the names in the complex SK name array instead.
+                searchKey = Collections.emptyList();
+                break;
+            default:
+                throw new AsterixException(ErrorCode.METADATA_ERROR, indexType.toString());
+        }
         for (List<String> field : searchKey) {
             listBuilder.reset(stringList);
             for (String subField : field) {
@@ -300,15 +537,86 @@
     /**
      * Keep protected to allow other extensions to add additional fields
      */
-    protected void writeOpenFields(Index index) throws HyracksDataException {
-        writeGramLength(index);
-        writeFullTextConfig(index);
+    protected void writeOpenFields(Index index) throws HyracksDataException, AlgebricksException {
+        switch (Index.IndexCategory.of(index.getIndexType())) {
+            case TEXT:
+                Index.TextIndexDetails textIndexDetails = (Index.TextIndexDetails) index.getIndexDetails();
+                writeGramLength(textIndexDetails);
+                writeFullTextConfig(textIndexDetails);
+                break;
+            case ARRAY:
+                writeComplexSearchKeys((Index.ArrayIndexDetails) index.getIndexDetails());
+                break;
+        }
         writeSearchKeyType(index);
         writeEnforced(index);
         writeSearchKeySourceIndicator(index);
     }
 
-    private void writeGramLength(Index index) throws HyracksDataException {
+    private void writeComplexSearchKeys(Index.ArrayIndexDetails indexDetails) throws HyracksDataException {
+        complexSearchKeyNameListBuilder.reset(AOrderedListType.FULL_OPEN_ORDEREDLIST_TYPE);
+        for (Index.ArrayIndexElement element : indexDetails.getElementList()) {
+            if (element.getUnnestList().isEmpty()) {
+                // If this is not a complex search key, write the field names as before.
+                buildSearchKeyNameList(element.getProjectList());
+                itemValue.reset();
+                listBuilder.write(itemValue.getDataOutput(), true);
+            } else {
+                // Otherwise, we create a complex searchkey name record.
+                complexSearchKeyNameRecordBuilder.reset(RecordUtil.FULLY_OPEN_RECORD_TYPE);
+
+                nameValue.reset();
+                aString.setValue(COMPLEXSEARCHKEY_UNNEST_FIELD_NAME);
+                stringSerde.serialize(aString, nameValue.getDataOutput());
+                buildSearchKeyNameList(element.getUnnestList());
+                itemValue.reset();
+                listBuilder.write(itemValue.getDataOutput(), true);
+                complexSearchKeyNameRecordBuilder.addField(nameValue, itemValue);
+
+                nameValue.reset();
+                aString.setValue(COMPLEXSEARCHKEY_PROJECT_FIELD_NAME);
+                stringSerde.serialize(aString, nameValue.getDataOutput());
+                buildSearchKeyNameList(element.getProjectList());
+                itemValue.reset();
+                listBuilder.write(itemValue.getDataOutput(), true);
+                complexSearchKeyNameRecordBuilder.addField(nameValue, itemValue);
+
+                itemValue.reset();
+                complexSearchKeyNameRecordBuilder.write(itemValue.getDataOutput(), true);
+            }
+            complexSearchKeyNameListBuilder.addItem(itemValue);
+        }
+
+        nameValue.reset();
+        fieldValue.reset();
+        aString.setValue(INDEX_SEARCHKEY_ELEMENTS_FIELD_NAME);
+        stringSerde.serialize(aString, nameValue.getDataOutput());
+        complexSearchKeyNameListBuilder.write(fieldValue.getDataOutput(), true);
+        recordBuilder.addField(nameValue, fieldValue);
+    }
+
+    private void buildSearchKeyNameList(List<List<String>> fieldList) throws HyracksDataException {
+        listBuilder.reset(AOrderedListType.FULL_OPEN_ORDEREDLIST_TYPE);
+        for (List<String> nestedField : fieldList) {
+            if (nestedField == null) {
+                itemValue.reset();
+                nullSerde.serialize(ANull.NULL, itemValue.getDataOutput());
+            } else {
+                innerListBuilder.reset(AOrderedListType.FULL_OPEN_ORDEREDLIST_TYPE);
+                for (String subField : nestedField) {
+                    itemValue.reset();
+                    aString.setValue(subField);
+                    stringSerde.serialize(aString, itemValue.getDataOutput());
+                    innerListBuilder.addItem(itemValue);
+                }
+                itemValue.reset();
+                innerListBuilder.write(itemValue.getDataOutput(), true);
+            }
+            listBuilder.addItem(itemValue);
+        }
+    }
+
+    private void writeGramLength(Index.TextIndexDetails index) throws HyracksDataException {
         if (index.getGramLength() > 0) {
             fieldValue.reset();
             nameValue.reset();
@@ -319,7 +627,7 @@
         }
     }
 
-    private void writeFullTextConfig(Index index) throws HyracksDataException {
+    private void writeFullTextConfig(Index.TextIndexDetails index) throws HyracksDataException {
         if (!Strings.isNullOrEmpty(index.getFullTextConfigName())) {
             nameValue.reset();
             aString.setValue(FULL_TEXT_CONFIG_FIELD_NAME);
@@ -333,26 +641,59 @@
         }
     }
 
-    private void writeSearchKeyType(Index index) throws HyracksDataException {
-        if (index.isOverridingKeyFieldTypes()) {
-            OrderedListBuilder typeListBuilder = new OrderedListBuilder();
-            typeListBuilder.reset(new AOrderedListType(BuiltinType.ANY, null));
-            nameValue.reset();
-            aString.setValue(INDEX_SEARCHKEY_TYPE_FIELD_NAME);
-
-            stringSerde.serialize(aString, nameValue.getDataOutput());
-
-            List<IAType> searchKeyType = index.getKeyFieldTypes();
-            for (IAType type : searchKeyType) {
-                itemValue.reset();
-                aString.setValue(type.getTypeName());
-                stringSerde.serialize(aString, itemValue.getDataOutput());
-                typeListBuilder.addItem(itemValue);
-            }
-            fieldValue.reset();
-            typeListBuilder.write(fieldValue.getDataOutput(), true);
-            recordBuilder.addField(nameValue, fieldValue);
+    private void writeSearchKeyType(Index index) throws HyracksDataException, AlgebricksException {
+        if (!index.getIndexDetails().isOverridingKeyFieldTypes()) {
+            return;
         }
+
+        OrderedListBuilder typeListBuilder = new OrderedListBuilder();
+        typeListBuilder.reset(AOrderedListType.FULL_OPEN_ORDEREDLIST_TYPE);
+
+        nameValue.reset();
+        aString.setValue(INDEX_SEARCHKEY_TYPE_FIELD_NAME);
+
+        stringSerde.serialize(aString, nameValue.getDataOutput());
+
+        switch (Index.IndexCategory.of(index.getIndexType())) {
+            // For value and text indexes, we persist the type as a single string (backwards compatibility).
+            case VALUE:
+                for (IAType type : ((Index.ValueIndexDetails) index.getIndexDetails()).getKeyFieldTypes()) {
+                    itemValue.reset();
+                    aString.setValue(type.getTypeName());
+                    stringSerde.serialize(aString, itemValue.getDataOutput());
+                    typeListBuilder.addItem(itemValue);
+                }
+                break;
+            case TEXT:
+                for (IAType type : ((Index.TextIndexDetails) index.getIndexDetails()).getKeyFieldTypes()) {
+                    itemValue.reset();
+                    aString.setValue(type.getTypeName());
+                    stringSerde.serialize(aString, itemValue.getDataOutput());
+                    typeListBuilder.addItem(itemValue);
+                }
+                break;
+            case ARRAY:
+                // For array indexes we persist the type as a list of strings.
+                for (Index.ArrayIndexElement element : ((Index.ArrayIndexDetails) index.getIndexDetails())
+                        .getElementList()) {
+                    listBuilder.reset(AOrderedListType.FULL_OPEN_ORDEREDLIST_TYPE);
+                    for (IAType type : element.getTypeList()) {
+                        itemValue.reset();
+                        aString.setValue(type.getTypeName());
+                        stringSerde.serialize(aString, itemValue.getDataOutput());
+                        listBuilder.addItem(itemValue);
+                    }
+                    itemValue.reset();
+                    listBuilder.write(itemValue.getDataOutput(), true);
+                    typeListBuilder.addItem(itemValue);
+                }
+                break;
+            default:
+                throw new AsterixException(ErrorCode.METADATA_ERROR, index.getIndexType().toString());
+        }
+        fieldValue.reset();
+        typeListBuilder.write(fieldValue.getDataOutput(), true);
+        recordBuilder.addField(nameValue, fieldValue);
     }
 
     private void writeEnforced(Index index) throws HyracksDataException {
@@ -366,8 +707,22 @@
         }
     }
 
-    private void writeSearchKeySourceIndicator(Index index) throws HyracksDataException {
-        List<Integer> keySourceIndicator = index.getKeyFieldSourceIndicators();
+    private void writeSearchKeySourceIndicator(Index index) throws HyracksDataException, AlgebricksException {
+        List<Integer> keySourceIndicator;
+        switch (Index.IndexCategory.of(index.getIndexType())) {
+            case VALUE:
+                keySourceIndicator = ((Index.ValueIndexDetails) index.getIndexDetails()).getKeyFieldSourceIndicators();
+                break;
+            case TEXT:
+                keySourceIndicator = ((Index.TextIndexDetails) index.getIndexDetails()).getKeyFieldSourceIndicators();
+                break;
+            case ARRAY:
+                keySourceIndicator = ((Index.ArrayIndexDetails) index.getIndexDetails()).getElementList().stream()
+                        .map(Index.ArrayIndexElement::getSourceIndicator).collect(Collectors.toList());
+                break;
+            default:
+                throw new AsterixException(ErrorCode.METADATA_ERROR, index.getIndexType().toString());
+        }
         boolean needSerialization = false;
         if (keySourceIndicator != null) {
             for (int source : keySourceIndicator) {
@@ -393,4 +748,4 @@
             recordBuilder.addField(nameValue, fieldValue);
         }
     }
-}
+}
\ No newline at end of file
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
new file mode 100644
index 0000000..af97101
--- /dev/null
+++ b/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/utils/ArrayIndexUtil.java
@@ -0,0 +1,380 @@
+/*
+ * 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.metadata.utils;
+
+import java.util.ArrayList;
+import java.util.List;
+
+import org.apache.asterix.common.exceptions.AsterixException;
+import org.apache.asterix.common.exceptions.CompilationException;
+import org.apache.asterix.common.exceptions.ErrorCode;
+import org.apache.asterix.metadata.entities.Index;
+import org.apache.asterix.om.typecomputer.impl.TypeComputeUtils;
+import org.apache.asterix.om.types.AOrderedListType;
+import org.apache.asterix.om.types.ARecordType;
+import org.apache.asterix.om.types.ATypeTag;
+import org.apache.asterix.om.types.AUnionType;
+import org.apache.asterix.om.types.AUnorderedListType;
+import org.apache.asterix.om.types.AbstractCollectionType;
+import org.apache.asterix.om.types.IAType;
+import org.apache.asterix.om.utils.NonTaggedFormatUtil;
+import org.apache.hyracks.algebricks.common.exceptions.AlgebricksException;
+import org.apache.hyracks.algebricks.common.utils.Pair;
+
+public class ArrayIndexUtil {
+    /**
+     * @deprecated Use the project + unnest scheme instead of array indicators.
+     */
+    public static IAType getSubFieldInArrayType(ARecordType recordType, List<String> subFieldName,
+            List<Integer> arrayDepthIndicators) throws AlgebricksException {
+        IAType subType = recordType.getFieldType(subFieldName.get(0));
+        for (int i = 1; i < subFieldName.size(); i++) {
+            if (subType == null) {
+                return null;
+            } else if (subType.getTypeTag().equals(ATypeTag.UNION)) {
+                // Support enforced types here.
+                subType = ((AUnionType) subType).getActualType();
+                if (!subType.getTypeTag().equals(ATypeTag.OBJECT) && !subType.getTypeTag().equals(ATypeTag.ARRAY)
+                        && !subType.getTypeTag().equals(ATypeTag.MULTISET)) {
+                    throw new AsterixException(ErrorCode.COMPILATION_ERROR,
+                            "Field accessor is not defined for values of type " + subType.getTypeTag());
+                }
+            }
+            if (subType.getTypeTag().equals(ATypeTag.OBJECT) && arrayDepthIndicators.get(i - 1) == 0) {
+                subType = ((ARecordType) subType).getFieldType(subFieldName.get(i));
+            } else if ((subType.getTypeTag().equals(ATypeTag.ARRAY) || subType.getTypeTag().equals(ATypeTag.MULTISET))
+                    && arrayDepthIndicators.get(i - 1) > 0) {
+                for (int j = 0; j < arrayDepthIndicators.get(i - 1); j++) {
+                    subType = TypeComputeUtils.extractListItemType(subType);
+                }
+                subType = (subType != null) ? ((ARecordType) subType).getFieldType(subFieldName.get(i)) : null;
+            } else {
+                throw new AsterixException(ErrorCode.COMPILATION_ERROR,
+                        (arrayDepthIndicators.get(i - 1) > 0)
+                                ? "Object type given, but array depth indicator is " + "non-zero."
+                                : "Array/multiset type given, but array depth indicator is zero.");
+            }
+        }
+        if (subType != null && arrayDepthIndicators.get(arrayDepthIndicators.size() - 1) > 0) {
+            // If the end field is an array, we must extract the list item here as well.
+            for (int j = 0; j < arrayDepthIndicators.get(arrayDepthIndicators.size() - 1); j++) {
+                if (subType instanceof AbstractCollectionType) {
+                    subType = TypeComputeUtils.extractListItemType(subType);
+                } else {
+                    throw new AsterixException(ErrorCode.COMPILATION_ERROR,
+                            "Array type expected for last term, but given: "
+                                    + ((subType != null) ? subType.getTypeTag() : "null"));
+                }
+            }
+        }
+        return subType;
+    }
+
+    /**
+     * Given a path of complex types (i.e. lists + records), determine the nullability of the field.
+     * @deprecated Use the project + unnest scheme instead of array indicators.
+     */
+    public static boolean isSubFieldNullable(ARecordType recordType, List<String> subFieldName,
+            List<Integer> arrayIndicators) throws AlgebricksException {
+        IAType subType = recordType.getFieldType(subFieldName.get(0));
+        for (int i = 1; i < subFieldName.size(); i++) {
+            if (subType == null) {
+                return true;
+            }
+            if (subType.getTypeTag().equals(ATypeTag.UNION)) {
+                if (NonTaggedFormatUtil.isOptional(subType)) {
+                    return true;
+                }
+                subType = ((AUnionType) subType).getActualType();
+                if (subType.getTypeTag() != ATypeTag.OBJECT) {
+                    throw new AsterixException(
+                            "Field accessor is not defined for values of type " + subType.getTypeTag());
+                }
+            }
+
+            if (subType instanceof ARecordType) {
+                subType = ((ARecordType) subType).getFieldType(subFieldName.get(i));
+            } else if (subType instanceof AbstractCollectionType && arrayIndicators.get(i - 1) > 0) {
+                for (int j = 0; j < arrayIndicators.get(i - 1); j++) {
+                    subType = TypeComputeUtils.extractListItemType(subType);
+                }
+                subType = (subType != null) ? ((ARecordType) subType).getFieldType(subFieldName.get(i)) : null;
+            } else {
+                throw CompilationException.create(ErrorCode.COMPILATION_ILLEGAL_STATE,
+                        "Illegal field type " + subType.getTypeTag() + " when checking field nullability");
+            }
+        }
+        return subType == null || NonTaggedFormatUtil.isOptional(subType);
+    }
+
+    /**
+     * Similar function to Index's "getNonNullableOpenFieldType", but accounts for array fields as well.
+     * @deprecated Use the project + unnest scheme instead of array indicators.
+     */
+    public static Pair<IAType, Boolean> getNonNullableOpenFieldType(IAType fieldType, List<String> fieldName,
+            ARecordType recType, List<Integer> arrayIndicators) throws AlgebricksException {
+        Pair<IAType, Boolean> keyPairType = null;
+        IAType subType = recType;
+        boolean nullable = false;
+        for (int i = 0; i < fieldName.size(); i++) {
+            if (subType instanceof AUnionType) {
+                nullable = nullable || ((AUnionType) subType).isUnknownableType();
+                subType = ((AUnionType) subType).getActualType();
+            }
+            if (subType instanceof ARecordType) {
+                subType = ((ARecordType) subType).getFieldType(fieldName.get(i));
+            } else if ((subType instanceof AOrderedListType || subType instanceof AUnorderedListType)
+                    && arrayIndicators.get(i - 1) > 0) {
+                for (int j = 0; j < arrayIndicators.get(i - 1); j++) {
+                    subType = TypeComputeUtils.extractListItemType(subType);
+                }
+                if (subType instanceof ARecordType) {
+                    subType = ((ARecordType) subType).getFieldType(fieldName.get(i));
+                } else {
+                    throw AsterixException.create(ErrorCode.COMPILATION_ILLEGAL_STATE, "Unexpected type " + fieldType);
+                }
+            } else {
+                throw AsterixException.create(ErrorCode.COMPILATION_ILLEGAL_STATE, "Unexpected type " + fieldType);
+            }
+
+            if (subType == null) {
+                keyPairType = Index.getNonNullableType(fieldType);
+                break;
+            }
+        }
+        if (subType != null) {
+            IAType keyType = ArrayIndexUtil.getSubFieldInArrayType(recType, fieldName, arrayIndicators);
+            Pair<IAType, Boolean> pair = Index.getNonNullableType(keyType);
+            pair.second = pair.second || ArrayIndexUtil.isSubFieldNullable(recType, fieldName, arrayIndicators);
+            keyPairType = pair;
+        }
+        keyPairType.second = keyPairType.second || nullable;
+        return keyPairType;
+    }
+
+    /**
+     * @deprecated Use new unnestList and projectList scheme.
+     */
+    public static List<String> getFlattenedKeyFieldNames(List<List<String>> unnestList, List<String> projectList) {
+        if (unnestList == null) {
+            return projectList;
+
+        } else {
+            List<String> flattenedKeyNameList = new ArrayList<>();
+            for (List<String> unnestField : unnestList) {
+                flattenedKeyNameList.addAll(unnestField);
+            }
+            if (projectList != null) {
+                flattenedKeyNameList.addAll(projectList);
+            }
+            return flattenedKeyNameList;
+        }
+    }
+
+    /**
+     * @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 || projectList.isEmpty()) {
+                // 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);
+    }
+
+    /**
+     * Given the {@code Index}'s representation of an array path (i.e. a concatenation of record paths, with array
+     * steps specified in depths corresponding to an index in the aforementioned record path array), traverse each
+     * distinct record path and invoke the appropriate commands for each scenario.
+     * <p>
+     * Here, we keep track of the record/list type at each step and give this to each command.
+     */
+    public static void walkArrayPath(ARecordType baseRecordType, List<String> flattenedFieldName,
+            List<Integer> flattenedDepthIndicators, TypeTrackerCommandExecutor commandExecutor)
+            throws AlgebricksException {
+        ArrayPath arrayPath = new ArrayPath(flattenedFieldName, flattenedDepthIndicators).invoke();
+        List<List<String>> fieldNamesPerArray = arrayPath.fieldNamesPerArray;
+        List<Integer> depthOfArraySteps = arrayPath.depthOfArraySteps;
+
+        // 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;
+            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;
+            }
+
+            for (int j = 0; j < depthOfArraySteps.get(i); j++) {
+                if (isTrackingType) {
+                    workingType = TypeComputeUtils.extractListItemType(workingType);
+                    if (workingType == null) {
+                        throw new AsterixException(ErrorCode.COMPILATION_ILLEGAL_STATE,
+                                "Expected list type inside record: " + startingStepRecordType);
+                    }
+                }
+                boolean isFirstArrayStep = i == 0;
+                boolean isFirstUnnestInStep = j == 0;
+                boolean isLastUnnestInIntermediateStep =
+                        j == depthOfArraySteps.get(i) - 1 && i < fieldNamesPerArray.size() - 1;
+                commandExecutor.executeActionOnEachArrayStep(startingStepRecordType, workingType,
+                        fieldNamesPerArray.get(i), isFirstArrayStep, isFirstUnnestInStep,
+                        isLastUnnestInIntermediateStep);
+            }
+
+            if (i == fieldNamesPerArray.size() - 1) {
+                boolean requiresOnlyOneUnnest = depthOfArraySteps.stream().reduce(0, Integer::sum).equals(1);
+                boolean isNonArrayStep = depthOfArraySteps.get(i) == 0;
+                commandExecutor.executeActionOnFinalArrayStep(startingStepRecordType, fieldNamesPerArray.get(i),
+                        isNonArrayStep, requiresOnlyOneUnnest);
+            }
+        }
+    }
+
+    /**
+     * Given the {@code Index}'s representation of an array path (i.e. a concatenation of record paths, with array
+     * steps specified in depths corresponding to an index in the aforementioned record path array), traverse each
+     * distinct record path and invoke the appropriate commands for each scenario.
+     * <p>
+     * Here, we keep track of the total number of actions performed and give this to each command.
+     */
+    public static void walkArrayPath(List<String> flattenedFieldName, List<Integer> flattenedDepthIndicators,
+            ActionCounterCommandExecutor commandExecutor) throws AlgebricksException {
+        ArrayPath arrayPath = new ArrayPath(flattenedFieldName, flattenedDepthIndicators).invoke();
+        List<List<String>> fieldNamesPerArray = arrayPath.fieldNamesPerArray;
+        List<Integer> depthOfArraySteps = arrayPath.depthOfArraySteps;
+
+        int numberOfActionsPerformed = 0;
+        for (int i = 0; i < fieldNamesPerArray.size(); i++) {
+            int unnestLevel = depthOfArraySteps.get(i);
+            if (i == 0) {
+                commandExecutor.executeActionOnFirstArrayStep();
+                numberOfActionsPerformed++;
+                unnestLevel--;
+            }
+
+            for (int j = 0; j < unnestLevel; j++) {
+                commandExecutor.executeActionOnIntermediateArrayStep(numberOfActionsPerformed++);
+            }
+
+            if (i == fieldNamesPerArray.size() - 1) {
+                commandExecutor.executeActionOnFinalArrayStep(numberOfActionsPerformed++);
+            }
+        }
+    }
+
+    public interface ActionCounterCommandExecutor {
+        void executeActionOnFirstArrayStep() throws AlgebricksException;
+
+        void executeActionOnIntermediateArrayStep(int numberOfActionsAlreadyPerformed) throws AlgebricksException;
+
+        void executeActionOnFinalArrayStep(int numberOfActionsAlreadyPerformed) throws AlgebricksException;
+    }
+
+    public interface TypeTrackerCommandExecutor {
+        void executeActionOnEachArrayStep(ARecordType startingStepRecordType, IAType workingType,
+                List<String> fieldName, boolean isFirstArrayStep, boolean isFirstUnnestInStep,
+                boolean isLastUnnestInIntermediateStep) throws AlgebricksException;
+
+        void executeActionOnFinalArrayStep(ARecordType startingStepRecordType, List<String> fieldName,
+                boolean isNonArrayStep, boolean requiresOnlyOneUnnest) throws AlgebricksException;
+    }
+
+    private static class ArrayPath {
+        private final List<String> flattenedFieldName;
+        private final List<Integer> flattenedDepthIndicators;
+        private List<List<String>> fieldNamesPerArray;
+        private List<Integer> depthOfArraySteps;
+
+        public ArrayPath(List<String> flattenedFieldName, List<Integer> flattenedDepthIndicators) {
+            this.flattenedFieldName = flattenedFieldName;
+            this.flattenedDepthIndicators = flattenedDepthIndicators;
+        }
+
+        public ArrayPath invoke() {
+            fieldNamesPerArray = new ArrayList<>();
+            depthOfArraySteps = new ArrayList<>();
+            List<String> workingRecordPath = new ArrayList<>();
+            for (int i = 0; i < flattenedDepthIndicators.size(); i++) {
+                workingRecordPath.add(flattenedFieldName.get(i));
+
+                if (i == flattenedDepthIndicators.size() - 1 || flattenedDepthIndicators.get(i) > 0) {
+                    depthOfArraySteps.add(flattenedDepthIndicators.get(i));
+                    fieldNamesPerArray.add(workingRecordPath);
+                    workingRecordPath = new ArrayList<>();
+                }
+            }
+            return this;
+        }
+    }
+}
diff --git a/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/utils/ExternalIndexingOperations.java b/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/utils/ExternalIndexingOperations.java
index 9847fe4..b899e16 100644
--- a/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/utils/ExternalIndexingOperations.java
+++ b/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/utils/ExternalIndexingOperations.java
@@ -80,6 +80,11 @@
 
 public class ExternalIndexingOperations {
     private static final Logger LOGGER = LogManager.getLogger();
+    /**
+     * Note: there's a special handling of this empty indexing field name in
+     * {@link org.apache.asterix.metadata.entitytupletranslators.IndexTupleTranslator
+     * IndexTupleTranslator.createMetadataEntityFromARecord()}
+     */
     public static final List<List<String>> FILE_INDEX_FIELD_NAMES =
             Collections.unmodifiableList(Collections.singletonList(Collections.singletonList("")));
     public static final List<IAType> FILE_INDEX_FIELD_TYPES =
diff --git a/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/utils/IndexUtil.java b/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/utils/IndexUtil.java
index 311e4b6..f5b2697 100644
--- a/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/utils/IndexUtil.java
+++ b/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/utils/IndexUtil.java
@@ -67,8 +67,16 @@
         if (index.isPrimaryIndex()) {
             return DatasetUtil.createBTreeFieldsWhenThereisAFilter(dataset);
         }
+        int numSecondaryKeys;
+        if (index.getIndexType() == DatasetConfig.IndexType.BTREE) {
+            numSecondaryKeys = ((Index.ValueIndexDetails) index.getIndexDetails()).getKeyFieldNames().size();
+        } else if (index.getIndexType() == DatasetConfig.IndexType.ARRAY) {
+            numSecondaryKeys = ((Index.ArrayIndexDetails) index.getIndexDetails()).getElementList().stream()
+                    .map(e -> e.getProjectList().size()).reduce(0, Integer::sum);
+        } else {
+            throw new CompilationException(ErrorCode.COMPILATION_UNKNOWN_INDEX_TYPE, index.getIndexType().toString());
+        }
         int numPrimaryKeys = dataset.getPrimaryKeys().size();
-        int numSecondaryKeys = index.getKeyFieldNames().size();
         int[] btreeFields = new int[numSecondaryKeys + numPrimaryKeys];
         for (int k = 0; k < btreeFields.length; k++) {
             btreeFields[k] = k;
@@ -82,9 +90,14 @@
             return empty;
         }
         int numPrimaryKeys = dataset.getPrimaryKeys().size();
-        int numSecondaryKeys = index.getKeyFieldNames().size();
+        int numSecondaryKeys;
         switch (index.getIndexType()) {
+            case ARRAY:
+                numSecondaryKeys = ((Index.ArrayIndexDetails) index.getIndexDetails()).getElementList().stream()
+                        .map(e -> e.getProjectList().size()).reduce(0, Integer::sum);
+                return new int[] { numPrimaryKeys + numSecondaryKeys };
             case BTREE:
+                numSecondaryKeys = ((Index.ValueIndexDetails) index.getIndexDetails()).getKeyFieldNames().size();
                 return new int[] { numPrimaryKeys + numSecondaryKeys };
             case RTREE:
             case LENGTH_PARTITIONED_NGRAM_INVIX:
diff --git a/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/utils/InvertedIndexResourceFactoryProvider.java b/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/utils/InvertedIndexResourceFactoryProvider.java
index 43798b2..b6510b9 100644
--- a/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/utils/InvertedIndexResourceFactoryProvider.java
+++ b/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/utils/InvertedIndexResourceFactoryProvider.java
@@ -66,7 +66,8 @@
             IBinaryComparatorFactory[] filterCmpFactories) throws AlgebricksException {
         // Get basic info
         List<List<String>> primaryKeys = dataset.getPrimaryKeys();
-        List<List<String>> secondaryKeys = index.getKeyFieldNames();
+        Index.TextIndexDetails indexDetails = (Index.TextIndexDetails) index.getIndexDetails();
+        List<List<String>> secondaryKeys = indexDetails.getKeyFieldNames();
         List<String> filterFieldName = DatasetUtil.getFilterField(dataset);
         int numPrimaryKeys = primaryKeys.size();
         int numSecondaryKeys = secondaryKeys.size();
@@ -122,7 +123,7 @@
         IBinaryTokenizerFactory tokenizerFactory = getTokenizerFactory(dataset, index, recordType, metaType);
         IFullTextConfigEvaluatorFactory fullTextConfigEvaluatorFactory =
                 FullTextUtil.fetchFilterAndCreateConfigEvaluator(mdProvider, index.getDataverseName(),
-                        index.getFullTextConfigName());
+                        indexDetails.getFullTextConfigName());
 
         return new LSMInvertedIndexLocalResourceFactory(storageManager, typeTraits, cmpFactories, filterTypeTraits,
                 filterCmpFactories, secondaryFilterFields, opTrackerFactory, ioOpCallbackFactory,
@@ -154,7 +155,8 @@
     private static ITypeTraits[] getTokenTypeTraits(Dataset dataset, Index index, ARecordType recordType,
             ARecordType metaType) throws AlgebricksException {
         int numPrimaryKeys = dataset.getPrimaryKeys().size();
-        int numSecondaryKeys = index.getKeyFieldNames().size();
+        Index.TextIndexDetails indexDetails = (Index.TextIndexDetails) index.getIndexDetails();
+        int numSecondaryKeys = indexDetails.getKeyFieldNames().size();
         IndexType indexType = index.getIndexType();
         // Sanity checks.
         if (numPrimaryKeys > 1) {
@@ -168,14 +170,14 @@
         boolean isPartitioned = indexType == IndexType.LENGTH_PARTITIONED_WORD_INVIX
                 || indexType == IndexType.LENGTH_PARTITIONED_NGRAM_INVIX;
         ARecordType sourceType;
-        List<Integer> keySourceIndicators = index.getKeyFieldSourceIndicators();
+        List<Integer> keySourceIndicators = indexDetails.getKeyFieldSourceIndicators();
         if (keySourceIndicators == null || keySourceIndicators.get(0) == 0) {
             sourceType = recordType;
         } else {
             sourceType = metaType;
         }
-        Pair<IAType, Boolean> keyTypePair = Index.getNonNullableOpenFieldType(index.getKeyFieldTypes().get(0),
-                index.getKeyFieldNames().get(0), sourceType);
+        Pair<IAType, Boolean> keyTypePair = Index.getNonNullableOpenFieldType(indexDetails.getKeyFieldTypes().get(0),
+                indexDetails.getKeyFieldNames().get(0), sourceType);
         IAType secondaryKeyType = keyTypePair.first;
         int numTokenFields = (!isPartitioned) ? numSecondaryKeys : numSecondaryKeys + 1;
         ITypeTraits[] tokenTypeTraits = new ITypeTraits[numTokenFields];
@@ -190,7 +192,8 @@
     private static IBinaryComparatorFactory[] getTokenComparatorFactories(Dataset dataset, Index index,
             ARecordType recordType, ARecordType metaType) throws AlgebricksException {
         int numPrimaryKeys = dataset.getPrimaryKeys().size();
-        int numSecondaryKeys = index.getKeyFieldNames().size();
+        Index.TextIndexDetails indexDetails = (Index.TextIndexDetails) index.getIndexDetails();
+        int numSecondaryKeys = indexDetails.getKeyFieldNames().size();
         IndexType indexType = index.getIndexType();
         // Sanity checks.
         if (numPrimaryKeys > 1) {
@@ -203,15 +206,15 @@
         }
         boolean isPartitioned = indexType == IndexType.LENGTH_PARTITIONED_WORD_INVIX
                 || indexType == IndexType.LENGTH_PARTITIONED_NGRAM_INVIX;
-        List<Integer> keySourceIndicators = index.getKeyFieldSourceIndicators();
+        List<Integer> keySourceIndicators = indexDetails.getKeyFieldSourceIndicators();
         ARecordType sourceType;
         if (keySourceIndicators == null || keySourceIndicators.get(0) == 0) {
             sourceType = recordType;
         } else {
             sourceType = metaType;
         }
-        Pair<IAType, Boolean> keyTypePair = Index.getNonNullableOpenFieldType(index.getKeyFieldTypes().get(0),
-                index.getKeyFieldNames().get(0), sourceType);
+        Pair<IAType, Boolean> keyTypePair = Index.getNonNullableOpenFieldType(indexDetails.getKeyFieldTypes().get(0),
+                indexDetails.getKeyFieldNames().get(0), sourceType);
         IAType secondaryKeyType = keyTypePair.first;
         // Comparators and type traits for tokens.
         int numTokenFields = (!isPartitioned) ? numSecondaryKeys : numSecondaryKeys + 1;
@@ -227,7 +230,8 @@
     private static IBinaryTokenizerFactory getTokenizerFactory(Dataset dataset, Index index, ARecordType recordType,
             ARecordType metaType) throws AlgebricksException {
         int numPrimaryKeys = dataset.getPrimaryKeys().size();
-        int numSecondaryKeys = index.getKeyFieldNames().size();
+        Index.TextIndexDetails indexDetails = (Index.TextIndexDetails) index.getIndexDetails();
+        int numSecondaryKeys = indexDetails.getKeyFieldNames().size();
         IndexType indexType = index.getIndexType();
         // Sanity checks.
         if (numPrimaryKeys > 1) {
@@ -239,19 +243,19 @@
                     indexType, 1);
         }
         ARecordType sourceType;
-        List<Integer> keySourceIndicators = index.getKeyFieldSourceIndicators();
+        List<Integer> keySourceIndicators = indexDetails.getKeyFieldSourceIndicators();
         if (keySourceIndicators == null || keySourceIndicators.get(0) == 0) {
             sourceType = recordType;
         } else {
             sourceType = metaType;
         }
-        Pair<IAType, Boolean> keyTypePair = Index.getNonNullableOpenFieldType(index.getKeyFieldTypes().get(0),
-                index.getKeyFieldNames().get(0), sourceType);
+        Pair<IAType, Boolean> keyTypePair = Index.getNonNullableOpenFieldType(indexDetails.getKeyFieldTypes().get(0),
+                indexDetails.getKeyFieldNames().get(0), sourceType);
         IAType secondaryKeyType = keyTypePair.first;
         // Set tokenizer factory.
         // TODO: We might want to expose the hashing option at the AQL level,
         // and add the choice to the index metadata.
         return NonTaggedFormatUtil.getBinaryTokenizerFactory(secondaryKeyType.getTypeTag(), indexType,
-                index.getGramLength());
+                indexDetails.getGramLength());
     }
 }
diff --git a/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/utils/KeyFieldTypeUtil.java b/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/utils/KeyFieldTypeUtil.java
index e074241..9458f61 100644
--- a/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/utils/KeyFieldTypeUtil.java
+++ b/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/utils/KeyFieldTypeUtil.java
@@ -20,6 +20,7 @@
 
 import java.util.ArrayList;
 import java.util.List;
+import java.util.Objects;
 
 import org.apache.asterix.common.config.DatasetConfig.DatasetType;
 import org.apache.asterix.common.exceptions.CompilationException;
@@ -27,11 +28,18 @@
 import org.apache.asterix.metadata.entities.Dataset;
 import org.apache.asterix.metadata.entities.Index;
 import org.apache.asterix.metadata.entities.InternalDatasetDetails;
+import org.apache.asterix.om.exceptions.ExceptionUtil;
 import org.apache.asterix.om.types.ARecordType;
+import org.apache.asterix.om.types.ATypeTag;
+import org.apache.asterix.om.types.AUnionType;
+import org.apache.asterix.om.types.AbstractCollectionType;
 import org.apache.asterix.om.types.IAType;
 import org.apache.asterix.om.utils.NonTaggedFormatUtil;
+import org.apache.asterix.om.utils.RecordUtil;
 import org.apache.hyracks.algebricks.common.exceptions.AlgebricksException;
 import org.apache.hyracks.algebricks.common.utils.Pair;
+import org.apache.hyracks.algebricks.common.utils.Triple;
+import org.apache.hyracks.api.exceptions.SourceLocation;
 
 public class KeyFieldTypeUtil {
 
@@ -92,6 +100,8 @@
      *            record.
      * @return a list of IATypes, one for each corresponding index key field.
      * @throws AlgebricksException
+     *
+     * @deprecated use {@link #getKeyProjectType(ARecordType, List, SourceLocation)}
      */
     public static List<IAType> getKeyTypes(ARecordType recordType, ARecordType metaRecordType, List<List<String>> keys,
             List<Integer> keySourceIndicators) throws AlgebricksException {
@@ -119,17 +129,39 @@
      */
     public static List<IAType> getBTreeIndexKeyTypes(Index index, ARecordType recordType, ARecordType metaRecordType)
             throws AlgebricksException {
-        List<Integer> keySourceIndicators = index.getKeyFieldSourceIndicators();
+        Index.ValueIndexDetails indexDetails = (Index.ValueIndexDetails) index.getIndexDetails();
+        List<Integer> keySourceIndicators = indexDetails.getKeyFieldSourceIndicators();
         List<IAType> indexKeyTypes = new ArrayList<>();
-        for (int i = 0; i < index.getKeyFieldNames().size(); i++) {
-            Pair<IAType, Boolean> keyPairType = Index.getNonNullableOpenFieldType(index.getKeyFieldTypes().get(i),
-                    index.getKeyFieldNames().get(i), chooseSource(keySourceIndicators, i, recordType, metaRecordType));
+        for (int i = 0; i < indexDetails.getKeyFieldNames().size(); i++) {
+            Pair<IAType, Boolean> keyPairType = Index.getNonNullableOpenFieldType(
+                    indexDetails.getKeyFieldTypes().get(i), indexDetails.getKeyFieldNames().get(i),
+                    chooseSource(keySourceIndicators, i, recordType, metaRecordType));
             indexKeyTypes.add(keyPairType.first);
         }
         return indexKeyTypes;
     }
 
     /**
+     * @see KeyFieldTypeUtil#getBTreeIndexKeyTypes(Index, ARecordType, ARecordType)
+     */
+    public static List<IAType> getArrayBTreeIndexKeyTypes(Index index, ARecordType recordType,
+            ARecordType metaRecordType) throws AlgebricksException {
+        Index.ArrayIndexDetails indexDetails = (Index.ArrayIndexDetails) index.getIndexDetails();
+        List<IAType> indexKeyTypes = new ArrayList<>();
+        for (Index.ArrayIndexElement e : indexDetails.getElementList()) {
+            for (int i = 0; i < e.getProjectList().size(); i++) {
+                ARecordType sourceType = (e.getSourceIndicator() == 0) ? recordType : metaRecordType;
+                Pair<IAType, Boolean> keyPairType = ArrayIndexUtil.getNonNullableOpenFieldType(e.getTypeList().get(i),
+                        ArrayIndexUtil.getFlattenedKeyFieldNames(e.getUnnestList(), e.getProjectList().get(i)),
+                        sourceType,
+                        ArrayIndexUtil.getArrayDepthIndicator(e.getUnnestList(), e.getProjectList().get(i)));
+                indexKeyTypes.add(keyPairType.first);
+            }
+        }
+        return indexKeyTypes;
+    }
+
+    /**
      * Get the types of RTree index key fields
      *
      * @param index,
@@ -143,11 +175,12 @@
      */
     public static List<IAType> getRTreeIndexKeyTypes(Index index, ARecordType recordType, ARecordType metaRecordType)
             throws AlgebricksException {
-        List<Integer> keySourceIndicators = index.getKeyFieldSourceIndicators();
+        Index.ValueIndexDetails indexDetails = (Index.ValueIndexDetails) index.getIndexDetails();
+        List<Integer> keySourceIndicators = indexDetails.getKeyFieldSourceIndicators();
         List<IAType> indexKeyTypes = new ArrayList<>();
         ARecordType targetRecType = chooseSource(keySourceIndicators, 0, recordType, metaRecordType);
-        Pair<IAType, Boolean> keyPairType = Index.getNonNullableOpenFieldType(index.getKeyFieldTypes().get(0),
-                index.getKeyFieldNames().get(0), targetRecType);
+        Pair<IAType, Boolean> keyPairType = Index.getNonNullableOpenFieldType(indexDetails.getKeyFieldTypes().get(0),
+                indexDetails.getKeyFieldNames().get(0), targetRecType);
         IAType keyType = keyPairType.first;
         IAType nestedKeyType = NonTaggedFormatUtil.getNestedSpatialType(keyType.getTypeTag());
         int numKeys = KeyFieldTypeUtil.getNumSecondaryKeys(index, targetRecType, metaRecordType);
@@ -171,17 +204,22 @@
      */
     public static int getNumSecondaryKeys(Index index, ARecordType recordType, ARecordType metaRecordType)
             throws AlgebricksException {
-        List<Integer> keySourceIndicators = index.getKeyFieldSourceIndicators();
         switch (index.getIndexType()) {
+            case ARRAY:
+                return ((Index.ArrayIndexDetails) index.getIndexDetails()).getElementList().stream()
+                        .map(e -> e.getProjectList().size()).reduce(0, Integer::sum);
             case BTREE:
+                return ((Index.ValueIndexDetails) index.getIndexDetails()).getKeyFieldNames().size();
             case SINGLE_PARTITION_WORD_INVIX:
             case SINGLE_PARTITION_NGRAM_INVIX:
             case LENGTH_PARTITIONED_WORD_INVIX:
             case LENGTH_PARTITIONED_NGRAM_INVIX:
-                return index.getKeyFieldNames().size();
+                return ((Index.TextIndexDetails) index.getIndexDetails()).getKeyFieldNames().size();
             case RTREE:
-                Pair<IAType, Boolean> keyPairType = Index.getNonNullableOpenFieldType(index.getKeyFieldTypes().get(0),
-                        index.getKeyFieldNames().get(0),
+                Index.ValueIndexDetails indexDetails = (Index.ValueIndexDetails) index.getIndexDetails();
+                List<Integer> keySourceIndicators = indexDetails.getKeyFieldSourceIndicators();
+                Pair<IAType, Boolean> keyPairType = Index.getNonNullableOpenFieldType(
+                        indexDetails.getKeyFieldTypes().get(0), indexDetails.getKeyFieldNames().get(0),
                         chooseSource(keySourceIndicators, 0, recordType, metaRecordType));
                 IAType keyType = keyPairType.first;
                 return NonTaggedFormatUtil.getNumDimensions(keyType.getTypeTag()) * 2;
@@ -208,4 +246,107 @@
             ARecordType metaRecordType) {
         return keySourceIndicators.get(index) == 0 ? recordType : metaRecordType;
     }
+
+    /**
+     * Returns type after applying UNNEST steps defined by an index element.
+     *
+     * @return { primeType, nullable, missable } or {@code null} if the path is not found in an open record
+     * @throws CompilationException
+     *             if path is not found in a closed record
+     */
+    public static Triple<IAType, Boolean, Boolean> getKeyUnnestType(final ARecordType inputType,
+            List<List<String>> unnestPathList, SourceLocation sourceLoc) throws CompilationException {
+        if (unnestPathList.isEmpty()) {
+            return new Triple<>(inputType, false, false);
+        }
+        IAType itemType = inputType;
+        boolean itemTypeNullable = false, itemTypeMissable = false;
+        for (List<String> unnestPath : unnestPathList) {
+            // check that the type is a record at this point
+            if (itemType.getTypeTag() != ATypeTag.OBJECT) {
+                throw new CompilationException(ErrorCode.TYPE_MISMATCH_GENERIC, sourceLoc, ATypeTag.OBJECT,
+                        itemType.getTypeTag());
+            }
+            ARecordType itemRecordType = (ARecordType) itemType;
+            Triple<IAType, Boolean, Boolean> fieldTypeResult = getKeyProjectType(itemRecordType, unnestPath, sourceLoc);
+            if (fieldTypeResult == null) {
+                return null;
+            }
+            IAType fieldType = fieldTypeResult.first;
+            boolean fieldTypeNullable = fieldTypeResult.second;
+            boolean fieldTypeMissable = fieldTypeResult.third;
+            // check that we've arrived to a collection type
+            if (!fieldType.getTypeTag().isListType()) {
+                throw new CompilationException(ErrorCode.TYPE_MISMATCH_GENERIC,
+                        sourceLoc, ExceptionUtil.toExpectedTypeString(new byte[] {
+                                ATypeTag.SERIALIZED_ORDEREDLIST_TYPE_TAG, ATypeTag.SERIALIZED_UNORDEREDLIST_TYPE_TAG }),
+                        fieldType);
+            }
+            AbstractCollectionType fieldListType = (AbstractCollectionType) fieldType;
+            IAType fieldListItemType = fieldListType.getItemType();
+            boolean fieldListItemTypeNullable = false, fieldListItemTypeMissable = false;
+            if (fieldListItemType.getTypeTag() == ATypeTag.UNION) {
+                AUnionType fieldListItemTypeUnion = (AUnionType) fieldListItemType;
+                fieldListItemType = fieldListItemTypeUnion.getActualType();
+                fieldListItemTypeNullable = fieldListItemTypeUnion.isNullableType();
+                fieldListItemTypeMissable = fieldListItemTypeUnion.isMissableType();
+            }
+            itemType = fieldListItemType;
+            itemTypeNullable = itemTypeNullable || fieldTypeNullable || fieldListItemTypeNullable;
+            itemTypeMissable = itemTypeMissable || fieldTypeMissable || fieldListItemTypeMissable;
+        }
+        return new Triple<>(itemType, itemTypeNullable, itemTypeMissable);
+    }
+
+    /**
+     * Returns type after applying SELECT steps defined by an index element.
+     *
+     * @return { primeType, nullable, missable } or {@code null} if the path is not found in an open record
+     * @throws CompilationException
+     *             if path is not found in a closed record
+     */
+    public static Triple<IAType, Boolean, Boolean> getKeyProjectType(final ARecordType inputType, List<String> path,
+            SourceLocation sourceLoc) throws CompilationException {
+        IAType itemType = inputType;
+        boolean itemTypeNullable = false, itemTypeMissalbe = false;
+        for (String step : path) {
+            // check that the type is a record at this point
+            if (itemType.getTypeTag() != ATypeTag.OBJECT) {
+                throw new CompilationException(ErrorCode.TYPE_MISMATCH_GENERIC, sourceLoc, ATypeTag.OBJECT,
+                        itemType.getTypeTag());
+            }
+            ARecordType itemRecordType = (ARecordType) itemType;
+            IAType fieldType = itemRecordType.getFieldType(step);
+            if (fieldType == null) {
+                if (itemRecordType.isOpen()) {
+                    // open record type and we couldn't find the field -> ok.
+                    return null;
+                } else {
+                    // closed record type and we couldn't find the field -> error.
+                    throw new CompilationException(ErrorCode.COMPILATION_FIELD_NOT_FOUND, sourceLoc,
+                            RecordUtil.toFullyQualifiedName(path));
+                }
+            }
+            if (fieldType.getTypeTag() == ATypeTag.UNION) {
+                AUnionType fieldTypeUnion = (AUnionType) fieldType;
+                itemType = fieldTypeUnion.getActualType();
+                itemTypeNullable = itemTypeNullable || fieldTypeUnion.isNullableType();
+                itemTypeMissalbe = itemTypeMissalbe || fieldTypeUnion.isMissableType();
+            } else {
+                itemType = fieldType;
+            }
+        }
+        return new Triple<>(itemType, itemTypeNullable, itemTypeMissalbe);
+    }
+
+    public static IAType makeUnknownableType(IAType primeType, boolean nullable, boolean missable) {
+        IAType type = Objects.requireNonNull(primeType);
+        if (nullable) {
+            type = AUnionType.createNullableType(type);
+        }
+        if (missable) {
+            type = AUnionType.createMissableType(type);
+        }
+        return type;
+    }
 }
diff --git a/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/utils/RTreeResourceFactoryProvider.java b/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/utils/RTreeResourceFactoryProvider.java
index 2108a4c..e00d7b6 100644
--- a/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/utils/RTreeResourceFactoryProvider.java
+++ b/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/utils/RTreeResourceFactoryProvider.java
@@ -71,15 +71,16 @@
             ARecordType recordType, ARecordType metaType, ILSMMergePolicyFactory mergePolicyFactory,
             Map<String, String> mergePolicyProperties, ITypeTraits[] filterTypeTraits,
             IBinaryComparatorFactory[] filterCmpFactories) throws AlgebricksException {
-        if (index.getKeyFieldNames().size() != 1) {
+        Index.ValueIndexDetails indexDetails = (Index.ValueIndexDetails) index.getIndexDetails();
+        if (indexDetails.getKeyFieldNames().size() != 1) {
             throw new CompilationException(ErrorCode.COMPILATION_ILLEGAL_INDEX_NUM_OF_FIELD,
-                    index.getKeyFieldNames().size(), index.getIndexType(), 1);
+                    indexDetails.getKeyFieldNames().size(), index.getIndexType(), 1);
         }
-        IAType spatialType = Index.getNonNullableOpenFieldType(index.getKeyFieldTypes().get(0),
-                index.getKeyFieldNames().get(0), recordType).first;
+        IAType spatialType = Index.getNonNullableOpenFieldType(indexDetails.getKeyFieldTypes().get(0),
+                indexDetails.getKeyFieldNames().get(0), recordType).first;
         if (spatialType == null) {
             throw new CompilationException(ErrorCode.COMPILATION_FIELD_NOT_FOUND,
-                    StringUtils.join(index.getKeyFieldNames().get(0), '.'));
+                    StringUtils.join(indexDetails.getKeyFieldNames().get(0), '.'));
         }
         List<List<String>> primaryKeyFields = dataset.getPrimaryKeys();
         int numPrimaryKeys = primaryKeyFields.size();
@@ -189,7 +190,8 @@
     private static ITypeTraits[] getTypeTraits(MetadataProvider metadataProvider, Dataset dataset, Index index,
             ARecordType recordType, ARecordType metaType) throws AlgebricksException {
         ITypeTraitProvider ttProvider = metadataProvider.getStorageComponentProvider().getTypeTraitProvider();
-        List<List<String>> secondaryKeyFields = index.getKeyFieldNames();
+        Index.ValueIndexDetails indexDetails = (Index.ValueIndexDetails) index.getIndexDetails();
+        List<List<String>> secondaryKeyFields = indexDetails.getKeyFieldNames();
         int numSecondaryKeys = secondaryKeyFields.size();
         int numPrimaryKeys = dataset.getPrimaryKeys().size();
         ITypeTraits[] primaryTypeTraits = dataset.getPrimaryTypeTraits(metadataProvider, recordType, metaType);
@@ -198,14 +200,14 @@
                     + "There can be only one field as a key for the R-tree index.");
         }
         ARecordType sourceType;
-        List<Integer> keySourceIndicators = index.getKeyFieldSourceIndicators();
+        List<Integer> keySourceIndicators = indexDetails.getKeyFieldSourceIndicators();
         if (keySourceIndicators == null || keySourceIndicators.get(0) == 0) {
             sourceType = recordType;
         } else {
             sourceType = metaType;
         }
-        Pair<IAType, Boolean> spatialTypePair = Index.getNonNullableOpenFieldType(index.getKeyFieldTypes().get(0),
-                secondaryKeyFields.get(0), sourceType);
+        Pair<IAType, Boolean> spatialTypePair = Index.getNonNullableOpenFieldType(
+                indexDetails.getKeyFieldTypes().get(0), secondaryKeyFields.get(0), sourceType);
         IAType spatialType = spatialTypePair.first;
         if (spatialType == null) {
             throw new AsterixException("Could not find field " + secondaryKeyFields.get(0) + " in the schema.");
@@ -227,21 +229,22 @@
             ARecordType recordType, ARecordType metaType) throws AlgebricksException {
         IBinaryComparatorFactoryProvider cmpFactoryProvider =
                 metadataProvider.getStorageComponentProvider().getComparatorFactoryProvider();
-        List<List<String>> secondaryKeyFields = index.getKeyFieldNames();
+        Index.ValueIndexDetails indexDetails = (Index.ValueIndexDetails) index.getIndexDetails();
+        List<List<String>> secondaryKeyFields = indexDetails.getKeyFieldNames();
         int numSecondaryKeys = secondaryKeyFields.size();
         if (numSecondaryKeys != 1) {
             throw new AsterixException("Cannot use " + numSecondaryKeys + " fields as a key for the R-tree index. "
                     + "There can be only one field as a key for the R-tree index.");
         }
-        List<Integer> keySourceIndicators = index.getKeyFieldSourceIndicators();
+        List<Integer> keySourceIndicators = indexDetails.getKeyFieldSourceIndicators();
         ARecordType sourceType;
         if (keySourceIndicators == null || keySourceIndicators.get(0) == 0) {
             sourceType = recordType;
         } else {
             sourceType = metaType;
         }
-        Pair<IAType, Boolean> spatialTypePair = Index.getNonNullableOpenFieldType(index.getKeyFieldTypes().get(0),
-                secondaryKeyFields.get(0), sourceType);
+        Pair<IAType, Boolean> spatialTypePair = Index.getNonNullableOpenFieldType(
+                indexDetails.getKeyFieldTypes().get(0), secondaryKeyFields.get(0), sourceType);
         IAType spatialType = spatialTypePair.first;
         if (spatialType == null) {
             throw new AsterixException("Could not find field " + secondaryKeyFields.get(0) + " in the schema.");
diff --git a/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/utils/SecondaryArrayIndexBTreeOperationsHelper.java b/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/utils/SecondaryArrayIndexBTreeOperationsHelper.java
new file mode 100644
index 0000000..fdb21a2
--- /dev/null
+++ b/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/utils/SecondaryArrayIndexBTreeOperationsHelper.java
@@ -0,0 +1,757 @@
+/*
+ * 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.metadata.utils;
+
+import java.util.ArrayDeque;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.Deque;
+import java.util.LinkedList;
+import java.util.List;
+import java.util.Queue;
+import java.util.Stack;
+import java.util.stream.IntStream;
+import java.util.stream.Stream;
+
+import org.apache.asterix.common.config.DatasetConfig.DatasetType;
+import org.apache.asterix.common.exceptions.AsterixException;
+import org.apache.asterix.common.exceptions.ErrorCode;
+import org.apache.asterix.common.utils.StorageConstants;
+import org.apache.asterix.external.indexing.IndexingConstants;
+import org.apache.asterix.formats.nontagged.TypeTraitProvider;
+import org.apache.asterix.metadata.declared.MetadataProvider;
+import org.apache.asterix.metadata.entities.Dataset;
+import org.apache.asterix.metadata.entities.Index;
+import org.apache.asterix.metadata.entities.InternalDatasetDetails;
+import org.apache.asterix.om.functions.BuiltinFunctions;
+import org.apache.asterix.om.types.ARecordType;
+import org.apache.asterix.om.types.IAType;
+import org.apache.asterix.runtime.utils.RuntimeUtils;
+import org.apache.commons.lang3.mutable.Mutable;
+import org.apache.commons.lang3.mutable.MutableObject;
+import org.apache.hyracks.algebricks.common.constraints.AlgebricksPartitionConstraintHelper;
+import org.apache.hyracks.algebricks.common.exceptions.AlgebricksException;
+import org.apache.hyracks.algebricks.common.utils.Pair;
+import org.apache.hyracks.algebricks.core.jobgen.impl.ConnectorPolicyAssignmentPolicy;
+import org.apache.hyracks.algebricks.data.IBinaryComparatorFactoryProvider;
+import org.apache.hyracks.algebricks.data.ISerializerDeserializerProvider;
+import org.apache.hyracks.algebricks.data.ITypeTraitProvider;
+import org.apache.hyracks.algebricks.runtime.base.IAggregateEvaluatorFactory;
+import org.apache.hyracks.algebricks.runtime.base.IPushRuntimeFactory;
+import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluatorFactory;
+import org.apache.hyracks.algebricks.runtime.base.IUnnestingEvaluatorFactory;
+import org.apache.hyracks.algebricks.runtime.evaluators.ColumnAccessEvalFactory;
+import org.apache.hyracks.algebricks.runtime.operators.aggreg.SimpleAlgebricksAccumulatingAggregatorFactory;
+import org.apache.hyracks.algebricks.runtime.operators.base.SinkRuntimeFactory;
+import org.apache.hyracks.algebricks.runtime.operators.meta.AlgebricksMetaOperatorDescriptor;
+import org.apache.hyracks.algebricks.runtime.operators.std.AssignRuntimeFactory;
+import org.apache.hyracks.algebricks.runtime.operators.std.UnnestRuntimeFactory;
+import org.apache.hyracks.api.dataflow.IOperatorDescriptor;
+import org.apache.hyracks.api.dataflow.value.IBinaryComparatorFactory;
+import org.apache.hyracks.api.dataflow.value.ISerializerDeserializer;
+import org.apache.hyracks.api.dataflow.value.ITypeTraits;
+import org.apache.hyracks.api.dataflow.value.RecordDescriptor;
+import org.apache.hyracks.api.exceptions.SourceLocation;
+import org.apache.hyracks.api.job.JobSpecification;
+import org.apache.hyracks.dataflow.std.connectors.OneToOneConnectorDescriptor;
+import org.apache.hyracks.dataflow.std.group.AbstractAggregatorDescriptorFactory;
+import org.apache.hyracks.dataflow.std.group.preclustered.PreclusteredGroupOperatorDescriptor;
+import org.apache.hyracks.storage.am.common.dataflow.IIndexDataflowHelperFactory;
+import org.apache.hyracks.storage.am.common.dataflow.IndexDataflowHelperFactory;
+
+public class SecondaryArrayIndexBTreeOperationsHelper extends SecondaryTreeIndexOperationsHelper {
+    private final int numAtomicSecondaryKeys, numArraySecondaryKeys, numTotalSecondaryKeys;
+    private final Index.ArrayIndexDetails arrayIndexDetails;
+    private final EvalFactoryAndRecDescStackBuilder evalFactoryAndRecDescStackBuilder =
+            new EvalFactoryAndRecDescStackBuilder();
+
+    // TODO (GLENN): Phase these out and use the UNNEST / PROJECT scheme instead.
+    private final List<List<String>> flattenedFieldNames;
+    private final List<IAType> flattenedKeyTypes;
+    private final List<List<Integer>> depthIndicators;
+
+    protected SecondaryArrayIndexBTreeOperationsHelper(Dataset dataset, Index index, MetadataProvider metadataProvider,
+            SourceLocation sourceLoc) throws AlgebricksException {
+        super(dataset, index, metadataProvider, sourceLoc);
+        arrayIndexDetails = (Index.ArrayIndexDetails) index.getIndexDetails();
+
+        flattenedFieldNames = new ArrayList<>();
+        flattenedKeyTypes = new ArrayList<>();
+        depthIndicators = new ArrayList<>();
+        for (Index.ArrayIndexElement e : arrayIndexDetails.getElementList()) {
+            if (e.getUnnestList().isEmpty()) {
+                flattenedFieldNames.add(e.getProjectList().get(0));
+                flattenedKeyTypes.add(e.getTypeList().get(0));
+                depthIndicators
+                        .add(ArrayIndexUtil.getArrayDepthIndicator(e.getUnnestList(), e.getProjectList().get(0)));
+            } else {
+                for (int i = 0; i < e.getProjectList().size(); i++) {
+                    List<String> project = e.getProjectList().get(i);
+                    flattenedFieldNames.add(ArrayIndexUtil.getFlattenedKeyFieldNames(e.getUnnestList(), project));
+                    depthIndicators.add(ArrayIndexUtil.getArrayDepthIndicator(e.getUnnestList(), project));
+                    flattenedKeyTypes.add(e.getTypeList().get(i));
+                }
+            }
+        }
+
+        int totalSecondaryKeyCount = 0;
+        int atomicSecondaryKeyCount = 0;
+        for (Index.ArrayIndexElement e : arrayIndexDetails.getElementList()) {
+            if (e.getUnnestList().isEmpty()) {
+                atomicSecondaryKeyCount++;
+                totalSecondaryKeyCount++;
+            } else {
+                totalSecondaryKeyCount += e.getProjectList().size();
+            }
+        }
+
+        numTotalSecondaryKeys = totalSecondaryKeyCount;
+        numAtomicSecondaryKeys = atomicSecondaryKeyCount;
+        numArraySecondaryKeys = numTotalSecondaryKeys - numAtomicSecondaryKeys;
+    }
+
+    private int findPosOfArrayIndex() throws AsterixException {
+        for (int i = 0; i < arrayIndexDetails.getElementList().size(); i++) {
+            if (!arrayIndexDetails.getElementList().get(i).getUnnestList().isEmpty()) {
+                return i;
+            }
+        }
+        throw new AsterixException(ErrorCode.COMPILATION_ILLEGAL_STATE, sourceLoc, "No array index found.");
+    }
+
+    @Override
+    protected void setSecondaryRecDescAndComparators() throws AlgebricksException {
+        Index.ArrayIndexDetails arrayIndexDetails = (Index.ArrayIndexDetails) index.getIndexDetails();
+        int numSecondaryKeys = this.getNumSecondaryKeys();
+        secondaryFieldAccessEvalFactories = new IScalarEvaluatorFactory[numSecondaryKeys + numFilterFields];
+        secondaryComparatorFactories = new IBinaryComparatorFactory[numSecondaryKeys + numPrimaryKeys];
+        secondaryBloomFilterKeyFields = new int[numSecondaryKeys];
+        ISerializerDeserializer[] secondaryRecFields =
+                new ISerializerDeserializer[numPrimaryKeys + numSecondaryKeys + numFilterFields];
+        ISerializerDeserializer[] enforcedRecFields =
+                new ISerializerDeserializer[1 + numPrimaryKeys + (dataset.hasMetaPart() ? 1 : 0) + numFilterFields];
+        ITypeTraits[] enforcedTypeTraits =
+                new ITypeTraits[1 + numPrimaryKeys + (dataset.hasMetaPart() ? 1 : 0) + numFilterFields];
+        secondaryTypeTraits = new ITypeTraits[numSecondaryKeys + numPrimaryKeys];
+        ISerializerDeserializerProvider serdeProvider = metadataProvider.getDataFormat().getSerdeProvider();
+        ITypeTraitProvider typeTraitProvider = metadataProvider.getDataFormat().getTypeTraitProvider();
+        IBinaryComparatorFactoryProvider comparatorFactoryProvider =
+                metadataProvider.getDataFormat().getBinaryComparatorFactoryProvider();
+        // Record column is 0 for external datasets, numPrimaryKeys for internal ones
+        int recordColumn = dataset.getDatasetType() == DatasetType.INTERNAL ? numPrimaryKeys : 0;
+        boolean isOverridingKeyFieldTypes = arrayIndexDetails.isOverridingKeyFieldTypes();
+        int flattenedListPos = 0;
+        for (Index.ArrayIndexElement e : arrayIndexDetails.getElementList()) {
+            for (int i = 0; i < e.getProjectList().size(); i++) {
+                ARecordType sourceType = (e.getSourceIndicator() == 0) ? itemType : metaType;
+                addSKEvalFactories(isOverridingKeyFieldTypes ? enforcedItemType : sourceType, flattenedListPos, false);
+                Pair<IAType, Boolean> keyTypePair = ArrayIndexUtil.getNonNullableOpenFieldType(e.getTypeList().get(i),
+                        ArrayIndexUtil.getFlattenedKeyFieldNames(e.getUnnestList(), e.getProjectList().get(i)),
+                        sourceType,
+                        ArrayIndexUtil.getArrayDepthIndicator(e.getUnnestList(), e.getProjectList().get(i)));
+                IAType keyType = keyTypePair.first;
+                anySecondaryKeyIsNullable = anySecondaryKeyIsNullable || keyTypePair.second;
+                ISerializerDeserializer keySerde = serdeProvider.getSerializerDeserializer(keyType);
+                secondaryRecFields[flattenedListPos] = keySerde;
+                secondaryComparatorFactories[flattenedListPos] =
+                        comparatorFactoryProvider.getBinaryComparatorFactory(keyType, true);
+                secondaryTypeTraits[flattenedListPos] = typeTraitProvider.getTypeTrait(keyType);
+                secondaryBloomFilterKeyFields[flattenedListPos] = flattenedListPos;
+
+                flattenedListPos++;
+            }
+        }
+        if (dataset.getDatasetType() == DatasetType.INTERNAL) {
+            // Add serializers and comparators for primary index fields.
+            for (int i = 0; i < numPrimaryKeys; i++) {
+                secondaryRecFields[numSecondaryKeys + i] = primaryRecDesc.getFields()[i];
+                enforcedRecFields[i] = primaryRecDesc.getFields()[i];
+                secondaryTypeTraits[numSecondaryKeys + i] = primaryRecDesc.getTypeTraits()[i];
+                enforcedTypeTraits[i] = primaryRecDesc.getTypeTraits()[i];
+                secondaryComparatorFactories[numSecondaryKeys + i] = primaryComparatorFactories[i];
+            }
+        } else {
+            // Add serializers and comparators for RID fields.
+            for (int i = 0; i < numPrimaryKeys; i++) {
+                secondaryRecFields[numSecondaryKeys + i] = IndexingConstants.getSerializerDeserializer(i);
+                enforcedRecFields[i] = IndexingConstants.getSerializerDeserializer(i);
+                secondaryTypeTraits[numSecondaryKeys + i] = IndexingConstants.getTypeTraits(i);
+                enforcedTypeTraits[i] = IndexingConstants.getTypeTraits(i);
+                secondaryComparatorFactories[numSecondaryKeys + i] = IndexingConstants.getComparatorFactory(i);
+            }
+        }
+        enforcedRecFields[numPrimaryKeys] = serdeProvider.getSerializerDeserializer(itemType);
+        enforcedTypeTraits[numPrimaryKeys] = typeTraitProvider.getTypeTrait(itemType);
+        if (dataset.hasMetaPart()) {
+            enforcedRecFields[numPrimaryKeys + 1] = serdeProvider.getSerializerDeserializer(metaType);
+            enforcedTypeTraits[numPrimaryKeys + 1] = typeTraitProvider.getTypeTrait(metaType);
+        }
+
+        if (numFilterFields > 0) {
+            ARecordType filterItemType =
+                    ((InternalDatasetDetails) dataset.getDatasetDetails()).getFilterSourceIndicator() == 0 ? itemType
+                            : metaType;
+            addSKEvalFactories(itemType, numSecondaryKeys, true);
+            Pair<IAType, Boolean> keyTypePair;
+            keyTypePair = Index.getNonNullableKeyFieldType(filterFieldName, filterItemType);
+            IAType type = keyTypePair.first;
+            ISerializerDeserializer serde = serdeProvider.getSerializerDeserializer(type);
+            secondaryRecFields[numPrimaryKeys + numSecondaryKeys] = serde;
+            enforcedRecFields[numPrimaryKeys + 1 + (dataset.hasMetaPart() ? 1 : 0)] = serde;
+            enforcedTypeTraits[numPrimaryKeys + 1 + (dataset.hasMetaPart() ? 1 : 0)] =
+                    typeTraitProvider.getTypeTrait(type);
+        }
+        secondaryRecDesc = new RecordDescriptor(secondaryRecFields, secondaryTypeTraits);
+        enforcedRecDesc = new RecordDescriptor(enforcedRecFields, enforcedTypeTraits);
+    }
+
+    @Override
+    protected int getNumSecondaryKeys() {
+        return arrayIndexDetails.getElementList().stream().map(e -> e.getProjectList().size()).reduce(0, Integer::sum);
+    }
+
+    protected int[] createFieldPermutationForBulkLoadOp(int numSecondaryKeyFields) {
+        int[] fieldPermutation = new int[numSecondaryKeyFields + numPrimaryKeys + numFilterFields];
+        for (int i = 0; i < fieldPermutation.length; i++) {
+            fieldPermutation[i] = i;
+        }
+        return fieldPermutation;
+    }
+
+    protected void addSKEvalFactories(ARecordType recordType, int fieldPos, boolean isFilterField)
+            throws AlgebricksException {
+        if (isFilterField) {
+            addFilterFieldToBuilder(recordType);
+            return;
+        }
+
+        List<Integer> arrayDepthIndicators = depthIndicators.get(fieldPos);
+        List<String> fieldNames = flattenedFieldNames.get(fieldPos);
+        if (arrayDepthIndicators.stream().noneMatch(b -> b > 0)) {
+            addAtomicFieldToBuilder(recordType, fieldPos);
+        } else {
+            EvalFactoryAndRecDescInvoker commandExecutor =
+                    new EvalFactoryAndRecDescInvoker(!evalFactoryAndRecDescStackBuilder.isUnnestEvalPopulated());
+            ArrayIndexUtil.walkArrayPath(recordType, fieldNames, arrayDepthIndicators, commandExecutor);
+        }
+    }
+
+    /**
+     * The following job spec is produced: (key provider) -> (PIDX scan) -> (cast)? -> ((unnest) -> (assign))* ->
+     * (select)? -> (sort)? -> (bulk load) -> (sink)
+     */
+    @Override
+    public JobSpecification buildLoadingJobSpec() throws AlgebricksException {
+        JobSpecification spec = RuntimeUtils.createJobSpecification(metadataProvider.getApplicationContext());
+        if (dataset.getDatasetType() == DatasetType.EXTERNAL) {
+            throw new UnsupportedOperationException("Array indexes on external datasets not currently supported.");
+        } else {
+            IndexUtil.bindJobEventListener(spec, metadataProvider);
+
+            // Start the job spec. Create a key provider and connect this to a primary index scan.
+            IOperatorDescriptor sourceOp = DatasetUtil.createDummyKeyProviderOp(spec, dataset, metadataProvider);
+            IOperatorDescriptor targetOp = DatasetUtil.createPrimaryIndexScanOp(spec, metadataProvider, dataset);
+            spec.connect(new OneToOneConnectorDescriptor(spec), sourceOp, 0, targetOp, 0);
+
+            sourceOp = targetOp;
+            if (arrayIndexDetails.isOverridingKeyFieldTypes() && !enforcedItemType.equals(itemType)) {
+                // If we have an enforced type, insert a "cast" after the primary index scan.
+                targetOp = createCastOp(spec, dataset.getDatasetType(), index.isEnforced());
+                spec.connect(new OneToOneConnectorDescriptor(spec), sourceOp, 0, targetOp, 0);
+                sourceOp = targetOp;
+            }
+
+            // TODO (GLENN): Refactor to use UNNEST + PROJECT scheme.
+            // Perform the unnest work.
+            final Mutable<IOperatorDescriptor> sourceOpRef = new MutableObject<>(sourceOp);
+            final Mutable<IOperatorDescriptor> targetOpRef = new MutableObject<>(targetOp);
+            LoadingJobBuilder jobBuilder = new LoadingJobBuilder(spec, sourceOpRef, targetOpRef);
+            int posOfArrayIndex = findPosOfArrayIndex();
+            ArrayIndexUtil.walkArrayPath(flattenedFieldNames.get(posOfArrayIndex), depthIndicators.get(posOfArrayIndex),
+                    jobBuilder);
+            sourceOp = sourceOpRef.getValue();
+
+            if (anySecondaryKeyIsNullable || arrayIndexDetails.isOverridingKeyFieldTypes()) {
+                // If any of the secondary fields are nullable, then we need to filter out the nulls.
+                targetOp = createFilterNullsSelectOp(spec, numTotalSecondaryKeys, secondaryRecDesc);
+                spec.connect(new OneToOneConnectorDescriptor(spec), sourceOp, 0, targetOp, 0);
+                sourceOp = targetOp;
+            }
+
+            // Sort by secondary keys, then primary keys.
+            IBinaryComparatorFactory[] comparatorFactories = getComparatorFactoriesForOrder();
+            targetOp = createSortOp(spec, comparatorFactories, secondaryRecDesc);
+            spec.connect(new OneToOneConnectorDescriptor(spec), sourceOp, 0, targetOp, 0);
+            sourceOp = targetOp;
+
+            // Only insert unique <SK, PK> pairs into our index,
+            targetOp = createPreSortedDistinctOp(spec, comparatorFactories, secondaryRecDesc);
+            spec.connect(new OneToOneConnectorDescriptor(spec), sourceOp, 0, targetOp, 0);
+            sourceOp = targetOp;
+
+            // Apply the bulk loading operator.
+            IIndexDataflowHelperFactory dataflowHelperFactory = new IndexDataflowHelperFactory(
+                    metadataProvider.getStorageComponentProvider().getStorageManager(), secondaryFileSplitProvider);
+            targetOp = createTreeIndexBulkLoadOp(spec, createFieldPermutationForBulkLoadOp(numTotalSecondaryKeys),
+                    dataflowHelperFactory, StorageConstants.DEFAULT_TREE_FILL_FACTOR);
+            spec.connect(new OneToOneConnectorDescriptor(spec), sourceOp, 0, targetOp, 0);
+
+            // Apply the sink.
+            sourceOp = targetOp;
+            SinkRuntimeFactory sinkRuntimeFactory = new SinkRuntimeFactory();
+            sinkRuntimeFactory.setSourceLocation(sourceLoc);
+            targetOp = new AlgebricksMetaOperatorDescriptor(spec, 1, 0,
+                    new IPushRuntimeFactory[] { sinkRuntimeFactory }, new RecordDescriptor[] { secondaryRecDesc });
+            spec.connect(new OneToOneConnectorDescriptor(spec), sourceOp, 0, targetOp, 0);
+            spec.addRoot(targetOp);
+            spec.setConnectorPolicyAssignmentPolicy(new ConnectorPolicyAssignmentPolicy());
+            return spec;
+        }
+    }
+
+    private IBinaryComparatorFactory[] getComparatorFactoriesForOrder() {
+        IBinaryComparatorFactory[] comparatorFactories =
+                new IBinaryComparatorFactory[numPrimaryKeys + numTotalSecondaryKeys + numFilterFields];
+        if (numTotalSecondaryKeys >= 0) {
+            System.arraycopy(secondaryComparatorFactories, 0, comparatorFactories, 0, numTotalSecondaryKeys);
+        }
+        if (numPrimaryKeys >= 0) {
+            System.arraycopy(primaryComparatorFactories, 0, comparatorFactories, numTotalSecondaryKeys, numPrimaryKeys);
+        }
+        if (numFilterFields > 0) {
+            comparatorFactories[numTotalSecondaryKeys + numPrimaryKeys] =
+                    secondaryComparatorFactories[numTotalSecondaryKeys];
+        }
+        return comparatorFactories;
+    }
+
+    private IOperatorDescriptor createPreSortedDistinctOp(JobSpecification spec,
+            IBinaryComparatorFactory[] secondaryComparatorFactories, RecordDescriptor secondaryRecDesc) {
+        int[] distinctFields = new int[secondaryComparatorFactories.length];
+        for (int i = 0; i < secondaryComparatorFactories.length; i++) {
+            distinctFields[i] = i;
+        }
+
+        IAggregateEvaluatorFactory[] aggFactories = new IAggregateEvaluatorFactory[] {};
+        AbstractAggregatorDescriptorFactory aggregatorFactory =
+                new SimpleAlgebricksAccumulatingAggregatorFactory(aggFactories, distinctFields);
+        aggregatorFactory.setSourceLocation(sourceLoc);
+
+        PreclusteredGroupOperatorDescriptor distinctOp = new PreclusteredGroupOperatorDescriptor(spec, distinctFields,
+                secondaryComparatorFactories, aggregatorFactory, secondaryRecDesc, false, -1);
+        distinctOp.setSourceLocation(sourceLoc);
+
+        return distinctOp;
+    }
+
+    /**
+     * Create an UNNEST operator for use with array indexes, which will perform the unnest and append the new field to
+     * the end of the input tuple. We expect three types of inputs to this operator:
+     * <p>
+     * <ol>
+     * <li>Tuples from a PIDX scan, which are in the format [PKs, record].
+     * <li>Tuples from an UNNEST op, which are in the format [PKs, (filter)?, intermediate-record].
+     * <li>Tuples from an UNNEST op that has already assigned a composite key, which are in the format:
+     * [PKs, (atomic SKs)?, (filter)?, intermediate-record].
+     * </ol>
+     * <p>
+     * In all cases here, the field(s) we want to unnest are located at the end of the input tuple.
+     */
+    private AlgebricksMetaOperatorDescriptor createUnnestOp(JobSpecification spec, int inputWidth,
+            IScalarEvaluatorFactory sef, RecordDescriptor unnestRecDesc) throws AlgebricksException {
+        int[] projectionList = IntStream.range(0, inputWidth + 1).toArray();
+        IUnnestingEvaluatorFactory unnestingEvaluatorFactory =
+                metadataProvider.getFunctionManager().lookupFunction(BuiltinFunctions.SCAN_COLLECTION, sourceLoc)
+                        .createUnnestingEvaluatorFactory(new IScalarEvaluatorFactory[] { sef });
+        UnnestRuntimeFactory unnest = new UnnestRuntimeFactory(projectionList.length - 1, unnestingEvaluatorFactory,
+                projectionList, false, null);
+        unnest.setSourceLocation(sourceLoc);
+        AlgebricksMetaOperatorDescriptor algebricksMetaOperatorDescriptor = new AlgebricksMetaOperatorDescriptor(spec,
+                1, 1, new IPushRuntimeFactory[] { unnest }, new RecordDescriptor[] { unnestRecDesc });
+        AlgebricksPartitionConstraintHelper.setPartitionConstraintInJobSpec(spec, algebricksMetaOperatorDescriptor,
+                primaryPartitionConstraint);
+        return algebricksMetaOperatorDescriptor;
+    }
+
+    /**
+     * Create an ASSIGN operator for use in-between UNNEST operators. This means that the projected fields will be in
+     * the order of [PKs, (atomic SKs)?, (filter)?, intermediate record], from the following expected inputs:
+     * <p>
+     * <ol>
+     * <li>Tuples from an PIDX scan -> UNNEST op, which are in the format [PKs, record, intermediate record].
+     * <li>Tuples from an ASSIGN op -> UNNEST op, which are in the format [PKs, (atomic SKs)?, (filter)?, record,
+     * intermediate record].
+     * <p>
+     * </ol>
+     * In addition to removing the record filter for the first case, we must also retrieve the filter field and the
+     * top-level atomic SK if they exist.
+     */
+    private AlgebricksMetaOperatorDescriptor createIntermediateAssignOp(JobSpecification spec, boolean isFirstAssign,
+            int inputWidth, List<IScalarEvaluatorFactory> sefs, RecordDescriptor assignRecDesc) {
+        int[] outColumns, projectionList;
+        if (isFirstAssign) {
+            projectionList = new int[numPrimaryKeys + numAtomicSecondaryKeys + numFilterFields + 1];
+            outColumns = IntStream.range(inputWidth, (numAtomicSecondaryKeys + numFilterFields == 1) ? (inputWidth + 1)
+                    : inputWidth + numAtomicSecondaryKeys + numFilterFields).toArray();
+            for (int i = 0; i < numPrimaryKeys; i++) {
+                projectionList[i] = i;
+            }
+            System.arraycopy(outColumns, 0, projectionList, numPrimaryKeys, numAtomicSecondaryKeys);
+            if (numFilterFields > 0) {
+                projectionList[numPrimaryKeys + numAtomicSecondaryKeys] = outColumns[outColumns.length - 1];
+            }
+        } else {
+            outColumns = new int[] { inputWidth };
+            projectionList = new int[inputWidth - 1];
+            for (int i = 0; i < projectionList.length - 1; i++) {
+                projectionList[i] = i;
+            }
+        }
+        projectionList[projectionList.length - 1] = inputWidth - 1;
+        return createGenericAssignOp(spec, sefs, assignRecDesc, outColumns, projectionList);
+    }
+
+    /**
+     * Create an ASSIGN operator for use after the final UNNEST operator for an array index bulk-loading job. This means
+     * that the projected fields will be in the order of [SKs, PKs, filter], from the following expected inputs:
+     * <p>
+     * <ol>
+     * <li>Tuples from an PIDX scan -> UNNEST op, which are in the format [PKs, record, intermediate record].
+     * <li>Tuples from an ASSIGN op -> UNNEST op, which are in the format [PKs, (atomic SKs)?, (filter)?, record,
+     * intermediate record].
+     * <p>
+     * </ol>
+     * For the first case, we must also retrieve the filter field and the top-level atomic SK if they exist.
+     */
+    private AlgebricksMetaOperatorDescriptor createFinalAssignOp(JobSpecification spec, boolean isFirstAssign,
+            int inputWidth, List<IScalarEvaluatorFactory> sefs, RecordDescriptor assignRecDesc) {
+        int[] outColumns, projectionList;
+        if (isFirstAssign) {
+            int outColumnsCursor = 0;
+            projectionList = new int[numPrimaryKeys + numTotalSecondaryKeys + numFilterFields];
+            outColumns = IntStream.range(inputWidth, (numTotalSecondaryKeys + numFilterFields == 1) ? (inputWidth + 1)
+                    : inputWidth + numTotalSecondaryKeys + numFilterFields).toArray();
+            for (int i = 0; i < numTotalSecondaryKeys; i++) {
+                int sizeOfFieldNamesForI = flattenedFieldNames.get(i).size();
+                if (depthIndicators.get(i).get(sizeOfFieldNamesForI - 1) != 0
+                        && (depthIndicators.get(i).stream().anyMatch(b -> b > 0))) {
+                    projectionList[i] = numPrimaryKeys + 1;
+                } else {
+                    projectionList[i] = outColumns[outColumnsCursor++];
+                }
+            }
+            for (int i = numTotalSecondaryKeys; i < numPrimaryKeys + numTotalSecondaryKeys; i++) {
+                projectionList[i] = i - numTotalSecondaryKeys;
+            }
+            if (numFilterFields > 0) {
+                projectionList[projectionList.length - 1] = outColumns[outColumnsCursor];
+            }
+        } else {
+            int atomicSKCursor = 0, arraySKCursor = 0;
+            projectionList = new int[numPrimaryKeys + numTotalSecondaryKeys + numFilterFields];
+            outColumns = IntStream.range(inputWidth, inputWidth + numArraySecondaryKeys).toArray();
+            for (int i = 0; i < numTotalSecondaryKeys; i++) {
+                int sizeOfFieldNamesForI = flattenedFieldNames.get(i).size();
+                if (depthIndicators.get(i).stream().noneMatch(b -> b > 0)) {
+                    projectionList[i] = numPrimaryKeys + atomicSKCursor++;
+                } else if (depthIndicators.get(i).get(sizeOfFieldNamesForI - 1) == 0) {
+                    projectionList[i] = outColumns[arraySKCursor++];
+                } else {
+                    projectionList[i] = numPrimaryKeys + numAtomicSecondaryKeys + numFilterFields + 1;
+                }
+            }
+            for (int i = 0; i < numPrimaryKeys; i++) {
+                projectionList[i + numTotalSecondaryKeys] = i;
+            }
+            if (numFilterFields > 0) {
+                projectionList[numPrimaryKeys + numTotalSecondaryKeys] = numPrimaryKeys + numAtomicSecondaryKeys;
+            }
+        }
+        return createGenericAssignOp(spec, sefs, assignRecDesc, outColumns, projectionList);
+    }
+
+    private AlgebricksMetaOperatorDescriptor createGenericAssignOp(JobSpecification spec,
+            List<IScalarEvaluatorFactory> sefs, RecordDescriptor assignRecDesc, int[] outColumns,
+            int[] projectionList) {
+        AssignRuntimeFactory assign =
+                new AssignRuntimeFactory(outColumns, sefs.toArray(new IScalarEvaluatorFactory[0]), projectionList);
+        assign.setSourceLocation(sourceLoc);
+        AlgebricksMetaOperatorDescriptor algebricksMetaOperatorDescriptor = new AlgebricksMetaOperatorDescriptor(spec,
+                1, 1, new IPushRuntimeFactory[] { assign }, new RecordDescriptor[] { assignRecDesc });
+        AlgebricksPartitionConstraintHelper.setPartitionConstraintInJobSpec(spec, algebricksMetaOperatorDescriptor,
+                primaryPartitionConstraint);
+        return algebricksMetaOperatorDescriptor;
+    }
+
+    private void addAtomicFieldToBuilder(ARecordType recordType, int indexPos) throws AlgebricksException {
+        IAType workingType = Index.getNonNullableOpenFieldType(flattenedKeyTypes.get(indexPos),
+                flattenedFieldNames.get(indexPos), recordType).first;
+        IScalarEvaluatorFactory sef =
+                metadataProvider.getDataFormat().getFieldAccessEvaluatorFactory(metadataProvider.getFunctionManager(),
+                        recordType, flattenedFieldNames.get(indexPos), numPrimaryKeys, sourceLoc);
+        evalFactoryAndRecDescStackBuilder.addAtomicSK(sef, workingType);
+    }
+
+    private void addFilterFieldToBuilder(ARecordType recordType) throws AlgebricksException {
+        IScalarEvaluatorFactory sef = metadataProvider.getDataFormat().getFieldAccessEvaluatorFactory(
+                metadataProvider.getFunctionManager(), recordType, filterFieldName, numPrimaryKeys, sourceLoc);
+        evalFactoryAndRecDescStackBuilder.addFilter(sef,
+                Index.getNonNullableKeyFieldType(filterFieldName, itemType).first);
+    }
+
+    class EvalFactoryAndRecDescInvoker implements ArrayIndexUtil.TypeTrackerCommandExecutor {
+        private final boolean isFirstWalk;
+
+        public EvalFactoryAndRecDescInvoker(boolean isFirstWalk) {
+            this.isFirstWalk = isFirstWalk;
+        }
+
+        @Override
+        public void executeActionOnEachArrayStep(ARecordType startingStepRecordType, IAType workingType,
+                List<String> fieldName, boolean isFirstArrayStep, boolean isFirstUnnestInStep,
+                boolean isLastUnnestInIntermediateStep) throws AlgebricksException {
+            if (!this.isFirstWalk) {
+                // We have already added the appropriate UNNESTs.
+                return;
+            }
+
+            int sourceColumnForNestedArrays = numPrimaryKeys + numAtomicSecondaryKeys + numFilterFields;
+            if (isFirstUnnestInStep) {
+                int sourceColumnForFirstUnnestInAtomicPath =
+                        isFirstArrayStep ? numPrimaryKeys : sourceColumnForNestedArrays;
+                IScalarEvaluatorFactory sef = metadataProvider.getDataFormat().getFieldAccessEvaluatorFactory(
+                        metadataProvider.getFunctionManager(), startingStepRecordType, fieldName,
+                        sourceColumnForFirstUnnestInAtomicPath, sourceLoc);
+                evalFactoryAndRecDescStackBuilder.addUnnest(sef, workingType);
+            } else {
+                IScalarEvaluatorFactory sef = new ColumnAccessEvalFactory(sourceColumnForNestedArrays);
+                evalFactoryAndRecDescStackBuilder.addUnnest(sef, workingType);
+            }
+        }
+
+        @Override
+        public void executeActionOnFinalArrayStep(ARecordType startingStepRecordType, List<String> fieldName,
+                boolean isNonArrayStep, boolean requiresOnlyOneUnnest) throws AlgebricksException {
+            // If the final value is nested inside a record, add this SEF.
+            if (!isNonArrayStep) {
+                return;
+            }
+
+            int sourceColumnForFinalEvaluator = 1 + ((requiresOnlyOneUnnest) ? numPrimaryKeys
+                    : (numPrimaryKeys + numAtomicSecondaryKeys + numFilterFields));
+            IScalarEvaluatorFactory sef = metadataProvider.getDataFormat().getFieldAccessEvaluatorFactory(
+                    metadataProvider.getFunctionManager(), startingStepRecordType, fieldName,
+                    sourceColumnForFinalEvaluator, sourceLoc);
+            evalFactoryAndRecDescStackBuilder.addFinalArraySK(sef);
+        }
+    }
+
+    class LoadingJobBuilder implements ArrayIndexUtil.ActionCounterCommandExecutor {
+        private final Stack<RecordDescriptor> recDescStack = evalFactoryAndRecDescStackBuilder.buildRecDescStack();
+        private final Stack<List<IScalarEvaluatorFactory>> sefStack =
+                evalFactoryAndRecDescStackBuilder.buildEvalFactoryStack();
+
+        private final JobSpecification spec;
+        private final Mutable<IOperatorDescriptor> sourceOpRef;
+        private final Mutable<IOperatorDescriptor> targetOpRef;
+        private RecordDescriptor workingRecDesc = recDescStack.pop(), nextRecDesc;
+
+        LoadingJobBuilder(JobSpecification spec, Mutable<IOperatorDescriptor> sourceOpRef,
+                Mutable<IOperatorDescriptor> targetOpRef) throws AlgebricksException {
+            this.spec = spec;
+            this.sourceOpRef = sourceOpRef;
+            this.targetOpRef = targetOpRef;
+        }
+
+        private void connectAndMoveToNextOp() {
+            spec.connect(new OneToOneConnectorDescriptor(spec), sourceOpRef.getValue(), 0, targetOpRef.getValue(), 0);
+            sourceOpRef.setValue(targetOpRef.getValue());
+            workingRecDesc = nextRecDesc;
+        }
+
+        @Override
+        public void executeActionOnFirstArrayStep() throws AlgebricksException {
+            nextRecDesc = recDescStack.pop();
+            targetOpRef
+                    .setValue(createUnnestOp(spec, workingRecDesc.getFieldCount(), sefStack.pop().get(0), nextRecDesc));
+            connectAndMoveToNextOp();
+        }
+
+        @Override
+        public void executeActionOnIntermediateArrayStep(int numberOfActionsAlreadyPerformed)
+                throws AlgebricksException {
+            // The purpose of the ASSIGN added here is twofold: 1) is to remove the unnecessary record/list we
+            // just unnested, and 2) is to extract the appropriate record fields, if we expect a record next.
+            nextRecDesc = recDescStack.pop();
+            targetOpRef.setValue(createIntermediateAssignOp(spec, numberOfActionsAlreadyPerformed < 2,
+                    workingRecDesc.getFieldCount(), sefStack.pop(), nextRecDesc));
+            connectAndMoveToNextOp();
+
+            nextRecDesc = recDescStack.pop();
+            targetOpRef
+                    .setValue(createUnnestOp(spec, workingRecDesc.getFieldCount(), sefStack.pop().get(0), nextRecDesc));
+            connectAndMoveToNextOp();
+        }
+
+        @Override
+        public void executeActionOnFinalArrayStep(int numberOfActionsAlreadyPerformed) {
+            targetOpRef.setValue(createFinalAssignOp(spec, numberOfActionsAlreadyPerformed < 2,
+                    workingRecDesc.getFieldCount(), sefStack.pop(), recDescStack.pop()));
+            connectAndMoveToNextOp();
+        }
+    }
+
+    class EvalFactoryAndRecDescStackBuilder {
+        private final Stack<IScalarEvaluatorFactory> unnestEvalFactories = new Stack<>();
+        private final List<IScalarEvaluatorFactory> atomicSKEvalFactories = new ArrayList<>();
+        private final List<IScalarEvaluatorFactory> finalArraySKEvalFactories = new ArrayList<>();
+        private final Queue<IAType> unnestEvalTypes = new LinkedList<>();
+        private final List<IAType> atomicSKEvalTypes = new ArrayList<>();
+        private IScalarEvaluatorFactory filterEvalFactory = null;
+        private IAType filterEvalType = null;
+
+        public void addAtomicSK(IScalarEvaluatorFactory sef, IAType type) {
+            atomicSKEvalFactories.add(sef);
+            atomicSKEvalTypes.add(type);
+        }
+
+        public void addFilter(IScalarEvaluatorFactory sef, IAType type) {
+            filterEvalFactory = sef;
+            filterEvalType = type;
+        }
+
+        public void addFinalArraySK(IScalarEvaluatorFactory sef) {
+            finalArraySKEvalFactories.add(sef);
+        }
+
+        public void addUnnest(IScalarEvaluatorFactory sef, IAType type) {
+            unnestEvalFactories.push(sef);
+            unnestEvalTypes.add(type);
+        }
+
+        public boolean isUnnestEvalPopulated() {
+            return !unnestEvalFactories.isEmpty();
+        }
+
+        /**
+         * Order our scalar evaluator factory stack in the order each UNNEST and ASSIGN op will be performed.
+         * <p>
+         * Visually, our stack looks like:
+         *
+         * <pre>
+         *  [ first UNNEST SEF ------------------------------------------------- ]
+         *  [ first ASSIGN SEFs -- atomic SKs and filter  ---------------------- ]
+         * *[ any intermediate UNNEST SEFs --column accessors / record accessors ]
+         *  [ final ASSIGN SEFs -- array SKs (record accessors) ---------------- ]
+         * </pre>
+         */
+        public Stack<List<IScalarEvaluatorFactory>> buildEvalFactoryStack() {
+            Stack<List<IScalarEvaluatorFactory>> resultant = new Stack<>();
+            resultant.push(finalArraySKEvalFactories);
+            int initialUnnestEvalFactorySize = unnestEvalFactories.size();
+            for (int i = 0; i < initialUnnestEvalFactorySize - 1; i++) {
+                if (i != 0) {
+                    resultant.push(new ArrayList<>());
+                }
+                resultant.push(Collections.singletonList(unnestEvalFactories.pop()));
+                if (i == initialUnnestEvalFactorySize - 2) {
+                    resultant.push(new ArrayList<>());
+                }
+            }
+            resultant.peek().addAll(atomicSKEvalFactories);
+            if (filterEvalFactory != null) {
+                resultant.peek().add(filterEvalFactory);
+            }
+            resultant.push(Collections.singletonList(unnestEvalFactories.pop()));
+            return resultant;
+        }
+
+        /**
+         * Order our record descriptor stack in the same order as our SEF stack.
+         * <p>
+         * Visually, our stack looks like:
+         *
+         * <pre>
+         *  [ primary record descriptor --------------------------------------- ]
+         *  [ primary record descriptor w/ first UNNESTed field at the end ---- ]
+         *  [ record descriptor w/ atomic fields, w/o record, w/ UNNESTed field ]
+         * *[ same record descriptor as above, w/ new unnested field ---------- ]
+         * *[ same record descriptor as above, w/o record field --------------- ]
+         *  [ secondary record descriptor ------------------------------------- ]
+         * </pre>
+         */
+        public Stack<RecordDescriptor> buildRecDescStack() throws AlgebricksException {
+            int initialUnnestEvalTypesSize = unnestEvalTypes.size();
+            Deque<RecordDescriptor> resultantAsDeque = new ArrayDeque<>();
+            resultantAsDeque.addFirst(primaryRecDesc);
+            resultantAsDeque.addFirst(createUnnestRecDesc(primaryRecDesc, unnestEvalTypes.remove()));
+            for (int i = 0; i < initialUnnestEvalTypesSize - 1; i++) {
+                resultantAsDeque.addFirst(createAssignRecDesc(resultantAsDeque.getFirst(), i == 0));
+                resultantAsDeque.addFirst(createUnnestRecDesc(resultantAsDeque.getFirst(), unnestEvalTypes.remove()));
+            }
+            resultantAsDeque.addFirst(secondaryRecDesc);
+            Stack<RecordDescriptor> resultant = new Stack<>();
+            resultant.addAll(resultantAsDeque);
+            return resultant;
+        }
+
+        private RecordDescriptor createUnnestRecDesc(RecordDescriptor priorRecDesc, IAType type)
+                throws AlgebricksException {
+            ISerializerDeserializerProvider serdeProvider = metadataProvider.getDataFormat().getSerdeProvider();
+            ISerializerDeserializer[] unnestFields = Stream
+                    .concat(Stream.of(priorRecDesc.getFields()),
+                            Stream.of(serdeProvider.getSerializerDeserializer(type)))
+                    .toArray(ISerializerDeserializer[]::new);
+            ITypeTraits[] unnestTypes = Stream.concat(Stream.of(priorRecDesc.getTypeTraits()),
+                    Stream.of(TypeTraitProvider.INSTANCE.getTypeTrait(type))).toArray(ITypeTraits[]::new);
+            return new RecordDescriptor(unnestFields, unnestTypes);
+        }
+
+        private RecordDescriptor createAssignRecDesc(RecordDescriptor priorRecDesc, boolean isFirstAssign)
+                throws AlgebricksException {
+            ArrayList<ISerializerDeserializer> assignFields = new ArrayList<>();
+            ArrayList<ITypeTraits> assignTypes = new ArrayList<>();
+            if (isFirstAssign) {
+                ISerializerDeserializerProvider serdeProvider = metadataProvider.getDataFormat().getSerdeProvider();
+                for (int i = 0; i < numPrimaryKeys; i++) {
+                    assignFields.add(priorRecDesc.getFields()[i]);
+                    assignTypes.add(priorRecDesc.getTypeTraits()[i]);
+                }
+                for (IAType s : atomicSKEvalTypes) {
+                    assignFields.add(serdeProvider.getSerializerDeserializer(s));
+                    assignTypes.add(TypeTraitProvider.INSTANCE.getTypeTrait(s));
+                }
+                if (filterEvalType != null) {
+                    assignFields.add(serdeProvider.getSerializerDeserializer(filterEvalType));
+                    assignTypes.add(TypeTraitProvider.INSTANCE.getTypeTrait(filterEvalType));
+                }
+                assignFields.add(priorRecDesc.getFields()[priorRecDesc.getFieldCount() - 1]);
+                assignTypes.add(priorRecDesc.getTypeTraits()[priorRecDesc.getFieldCount() - 1]);
+            } else {
+                assignFields = new ArrayList<>(Arrays.asList(priorRecDesc.getFields()));
+                assignTypes = new ArrayList<>(Arrays.asList(priorRecDesc.getTypeTraits()));
+                assignFields.remove(priorRecDesc.getFieldCount() - 2);
+                assignTypes.remove(priorRecDesc.getFieldCount() - 2);
+            }
+            return new RecordDescriptor(assignFields.toArray(new ISerializerDeserializer[0]),
+                    assignTypes.toArray(new ITypeTraits[0]));
+        }
+    }
+}
diff --git a/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/utils/SecondaryBTreeOperationsHelper.java b/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/utils/SecondaryBTreeOperationsHelper.java
index bc5ac8b..33f5b62 100644
--- a/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/utils/SecondaryBTreeOperationsHelper.java
+++ b/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/utils/SecondaryBTreeOperationsHelper.java
@@ -65,8 +65,9 @@
     @Override
     public JobSpecification buildLoadingJobSpec() throws AlgebricksException {
         JobSpecification spec = RuntimeUtils.createJobSpecification(metadataProvider.getApplicationContext());
-        boolean isOverridingKeyFieldTypes = index.isOverridingKeyFieldTypes();
-        int[] fieldPermutation = createFieldPermutationForBulkLoadOp(index.getKeyFieldNames().size());
+        Index.ValueIndexDetails indexDetails = (Index.ValueIndexDetails) index.getIndexDetails();
+        boolean isOverridingKeyFieldTypes = indexDetails.isOverridingKeyFieldTypes();
+        int[] fieldPermutation = createFieldPermutationForBulkLoadOp(indexDetails.getKeyFieldNames().size());
         IIndexDataflowHelperFactory dataflowHelperFactory = new IndexDataflowHelperFactory(
                 metadataProvider.getStorageComponentProvider().getStorageManager(), secondaryFileSplitProvider);
         if (dataset.getDatasetType() == DatasetType.EXTERNAL) {
@@ -86,12 +87,12 @@
                 spec.connect(new OneToOneConnectorDescriptor(spec), primaryScanOp, 0, sourceOp, 0);
             }
             AlgebricksMetaOperatorDescriptor asterixAssignOp =
-                    createExternalAssignOp(spec, index.getKeyFieldNames().size(), secondaryRecDesc);
+                    createExternalAssignOp(spec, indexDetails.getKeyFieldNames().size(), secondaryRecDesc);
 
             // If any of the secondary fields are nullable, then add a select op that filters nulls.
             AlgebricksMetaOperatorDescriptor selectOp = null;
             if (anySecondaryKeyIsNullable || isOverridingKeyFieldTypes) {
-                selectOp = createFilterNullsSelectOp(spec, index.getKeyFieldNames().size(), secondaryRecDesc);
+                selectOp = createFilterNullsSelectOp(spec, indexDetails.getKeyFieldNames().size(), secondaryRecDesc);
             }
 
             // Sort by secondary keys.
@@ -144,20 +145,20 @@
                 sourceOp = targetOp;
             }
             // primary index OR cast assign ----> assign op
-            targetOp = createAssignOp(spec, index.getKeyFieldNames().size(), secondaryRecDesc);
+            targetOp = createAssignOp(spec, indexDetails.getKeyFieldNames().size(), secondaryRecDesc);
             spec.connect(new OneToOneConnectorDescriptor(spec), sourceOp, 0, targetOp, 0);
 
             sourceOp = targetOp;
             if (anySecondaryKeyIsNullable || isOverridingKeyFieldTypes) {
                 // if any of the secondary fields are nullable, then add a select op that filters nulls.
                 // assign op ----> select op
-                targetOp = createFilterNullsSelectOp(spec, index.getKeyFieldNames().size(), secondaryRecDesc);
+                targetOp = createFilterNullsSelectOp(spec, indexDetails.getKeyFieldNames().size(), secondaryRecDesc);
                 spec.connect(new OneToOneConnectorDescriptor(spec), sourceOp, 0, targetOp, 0);
                 sourceOp = targetOp;
             }
 
             // no need to sort if the index is secondary primary index
-            if (!index.getKeyFieldNames().isEmpty()) {
+            if (!indexDetails.getKeyFieldNames().isEmpty()) {
                 // sort by secondary keys.
                 // assign op OR select op ----> sort op
                 targetOp = createSortOp(spec, secondaryComparatorFactories, secondaryRecDesc);
@@ -186,7 +187,7 @@
 
     @Override
     protected int getNumSecondaryKeys() {
-        return index.getKeyFieldNames().size();
+        return ((Index.ValueIndexDetails) index.getIndexDetails()).getKeyFieldNames().size();
     }
 
     /**
@@ -208,7 +209,8 @@
      */
     @Override
     protected void setSecondaryRecDescAndComparators() throws AlgebricksException {
-        int numSecondaryKeys = index.getKeyFieldNames().size();
+        Index.ValueIndexDetails indexDetails = (Index.ValueIndexDetails) index.getIndexDetails();
+        int numSecondaryKeys = indexDetails.getKeyFieldNames().size();
         secondaryFieldAccessEvalFactories = new IScalarEvaluatorFactory[numSecondaryKeys + numFilterFields];
         secondaryComparatorFactories = new IBinaryComparatorFactory[numSecondaryKeys + numPrimaryKeys];
         secondaryBloomFilterKeyFields = new int[numSecondaryKeys];
@@ -225,11 +227,11 @@
                 metadataProvider.getDataFormat().getBinaryComparatorFactoryProvider();
         // Record column is 0 for external datasets, numPrimaryKeys for internal ones
         int recordColumn = dataset.getDatasetType() == DatasetType.INTERNAL ? numPrimaryKeys : 0;
-        boolean isOverridingKeyFieldTypes = index.isOverridingKeyFieldTypes();
+        boolean isOverridingKeyFieldTypes = indexDetails.isOverridingKeyFieldTypes();
         for (int i = 0; i < numSecondaryKeys; i++) {
             ARecordType sourceType;
             int sourceColumn;
-            List<Integer> keySourceIndicators = index.getKeyFieldSourceIndicators();
+            List<Integer> keySourceIndicators = indexDetails.getKeyFieldSourceIndicators();
             if (keySourceIndicators == null || keySourceIndicators.get(i) == 0) {
                 sourceType = itemType;
                 sourceColumn = recordColumn;
@@ -239,9 +241,9 @@
             }
             secondaryFieldAccessEvalFactories[i] = metadataProvider.getDataFormat().getFieldAccessEvaluatorFactory(
                     metadataProvider.getFunctionManager(), isOverridingKeyFieldTypes ? enforcedItemType : sourceType,
-                    index.getKeyFieldNames().get(i), sourceColumn, sourceLoc);
-            Pair<IAType, Boolean> keyTypePair = Index.getNonNullableOpenFieldType(index.getKeyFieldTypes().get(i),
-                    index.getKeyFieldNames().get(i), sourceType);
+                    indexDetails.getKeyFieldNames().get(i), sourceColumn, sourceLoc);
+            Pair<IAType, Boolean> keyTypePair = Index.getNonNullableOpenFieldType(
+                    indexDetails.getKeyFieldTypes().get(i), indexDetails.getKeyFieldNames().get(i), sourceType);
             IAType keyType = keyTypePair.first;
             anySecondaryKeyIsNullable = anySecondaryKeyIsNullable || keyTypePair.second;
             ISerializerDeserializer keySerde = serdeProvider.getSerializerDeserializer(keyType);
diff --git a/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/utils/SecondaryCorrelatedBTreeOperationsHelper.java b/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/utils/SecondaryCorrelatedBTreeOperationsHelper.java
index 40f2610..aca953b 100644
--- a/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/utils/SecondaryCorrelatedBTreeOperationsHelper.java
+++ b/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/utils/SecondaryCorrelatedBTreeOperationsHelper.java
@@ -66,7 +66,8 @@
     public JobSpecification buildLoadingJobSpec() throws AlgebricksException {
         JobSpecification spec = RuntimeUtils.createJobSpecification(metadataProvider.getApplicationContext());
 
-        boolean isOverridingKeyFieldTypes = index.isOverridingKeyFieldTypes();
+        Index.ValueIndexDetails indexDetails = (Index.ValueIndexDetails) index.getIndexDetails();
+        boolean isOverridingKeyFieldTypes = indexDetails.isOverridingKeyFieldTypes();
 
         assert dataset.getDatasetType() == DatasetType.INTERNAL;
 
@@ -89,7 +90,7 @@
         }
         RecordDescriptor taggedSecondaryRecDesc = getTaggedRecordDescriptor(secondaryRecDesc);
         AlgebricksMetaOperatorDescriptor asterixAssignOp =
-                createAssignOp(spec, index.getKeyFieldNames().size(), taggedSecondaryRecDesc);
+                createAssignOp(spec, indexDetails.getKeyFieldNames().size(), taggedSecondaryRecDesc);
 
         // Generate compensate tuples for upsert
         IOperatorDescriptor processorOp =
@@ -122,12 +123,13 @@
 
     @Override
     protected int getNumSecondaryKeys() {
-        return index.getKeyFieldNames().size();
+        return ((Index.ValueIndexDetails) index.getIndexDetails()).getKeyFieldNames().size();
     }
 
     @Override
     protected void setSecondaryRecDescAndComparators() throws AlgebricksException {
-        int numSecondaryKeys = index.getKeyFieldNames().size();
+        Index.ValueIndexDetails indexDetails = (Index.ValueIndexDetails) index.getIndexDetails();
+        int numSecondaryKeys = indexDetails.getKeyFieldNames().size();
         secondaryFieldAccessEvalFactories = new IScalarEvaluatorFactory[numSecondaryKeys + numFilterFields];
         secondaryComparatorFactories = new IBinaryComparatorFactory[numSecondaryKeys + numPrimaryKeys];
         secondaryBloomFilterKeyFields = new int[numSecondaryKeys];
@@ -144,11 +146,11 @@
                 metadataProvider.getDataFormat().getBinaryComparatorFactoryProvider();
         // Record column is 0 for external datasets, numPrimaryKeys for internal ones
         int recordColumn = NUM_TAG_FIELDS + numPrimaryKeys;
-        boolean isOverridingKeyTypes = index.isOverridingKeyFieldTypes();
+        boolean isOverridingKeyTypes = indexDetails.isOverridingKeyFieldTypes();
         for (int i = 0; i < numSecondaryKeys; i++) {
             ARecordType sourceType;
             int sourceColumn;
-            List<Integer> keySourceIndicators = index.getKeyFieldSourceIndicators();
+            List<Integer> keySourceIndicators = indexDetails.getKeyFieldSourceIndicators();
             if (keySourceIndicators == null || keySourceIndicators.get(i) == 0) {
                 sourceType = itemType;
                 sourceColumn = recordColumn;
@@ -158,9 +160,9 @@
             }
             secondaryFieldAccessEvalFactories[i] = metadataProvider.getDataFormat().getFieldAccessEvaluatorFactory(
                     metadataProvider.getFunctionManager(), isOverridingKeyTypes ? enforcedItemType : sourceType,
-                    index.getKeyFieldNames().get(i), sourceColumn, sourceLoc);
-            Pair<IAType, Boolean> keyTypePair = Index.getNonNullableOpenFieldType(index.getKeyFieldTypes().get(i),
-                    index.getKeyFieldNames().get(i), sourceType);
+                    indexDetails.getKeyFieldNames().get(i), sourceColumn, sourceLoc);
+            Pair<IAType, Boolean> keyTypePair = Index.getNonNullableOpenFieldType(
+                    indexDetails.getKeyFieldTypes().get(i), indexDetails.getKeyFieldNames().get(i), sourceType);
             IAType keyType = keyTypePair.first;
             anySecondaryKeyIsNullable = anySecondaryKeyIsNullable || keyTypePair.second;
             ISerializerDeserializer keySerde = serdeProvider.getSerializerDeserializer(keyType);
diff --git a/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/utils/SecondaryCorrelatedInvertedIndexOperationsHelper.java b/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/utils/SecondaryCorrelatedInvertedIndexOperationsHelper.java
index 0b6821c..c111f0e 100644
--- a/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/utils/SecondaryCorrelatedInvertedIndexOperationsHelper.java
+++ b/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/utils/SecondaryCorrelatedInvertedIndexOperationsHelper.java
@@ -82,9 +82,10 @@
 
     @Override
     protected void setSecondaryRecDescAndComparators() throws AlgebricksException {
-        int numSecondaryKeys = index.getKeyFieldNames().size();
+        Index.TextIndexDetails indexDetails = (Index.TextIndexDetails) index.getIndexDetails();
+        int numSecondaryKeys = indexDetails.getKeyFieldNames().size();
         IndexType indexType = index.getIndexType();
-        boolean isOverridingKeyFieldTypes = index.isOverridingKeyFieldTypes();
+        boolean isOverridingKeyFieldTypes = indexDetails.isOverridingKeyFieldTypes();
         // Sanity checks.
         if (numPrimaryKeys > 1) {
             throw new CompilationException(ErrorCode.COMPILATION_ILLEGAL_INDEX_FOR_DATASET_WITH_COMPOSITE_PRIMARY_INDEX,
@@ -114,9 +115,9 @@
         if (numSecondaryKeys > 0) {
             secondaryFieldAccessEvalFactories[0] = metadataProvider.getDataFormat().getFieldAccessEvaluatorFactory(
                     metadataProvider.getFunctionManager(), isOverridingKeyFieldTypes ? enforcedItemType : itemType,
-                    index.getKeyFieldNames().get(0), recordColumn, sourceLoc);
-            Pair<IAType, Boolean> keyTypePair = Index.getNonNullableOpenFieldType(index.getKeyFieldTypes().get(0),
-                    index.getKeyFieldNames().get(0), itemType);
+                    indexDetails.getKeyFieldNames().get(0), recordColumn, sourceLoc);
+            Pair<IAType, Boolean> keyTypePair = Index.getNonNullableOpenFieldType(
+                    indexDetails.getKeyFieldTypes().get(0), indexDetails.getKeyFieldNames().get(0), itemType);
             secondaryKeyType = keyTypePair.first;
             anySecondaryKeyIsNullable = anySecondaryKeyIsNullable || keyTypePair.second;
             ISerializerDeserializer keySerde = serdeProvider.getSerializerDeserializer(secondaryKeyType);
@@ -151,9 +152,9 @@
         // TODO: We might want to expose the hashing option at the AQL level,
         // and add the choice to the index metadata.
         tokenizerFactory = NonTaggedFormatUtil.getBinaryTokenizerFactory(secondaryKeyType.getTypeTag(), indexType,
-                index.getGramLength());
+                indexDetails.getGramLength());
         fullTextConfigEvaluatorFactory = FullTextUtil.fetchFilterAndCreateConfigEvaluator(metadataProvider,
-                index.getDataverseName(), index.getFullTextConfigName());
+                index.getDataverseName(), indexDetails.getFullTextConfigName());
         // Type traits for inverted-list elements. Inverted lists contain
         // primary keys.
         invListsTypeTraits = new ITypeTraits[numPrimaryKeys];
@@ -222,8 +223,9 @@
                 getTaggedRecordDescriptor(dataset.getPrimaryRecordDescriptor(metadataProvider)));
 
         IOperatorDescriptor sourceOp = primaryScanOp;
-        boolean isOverridingKeyFieldTypes = index.isOverridingKeyFieldTypes();
-        int numSecondaryKeys = index.getKeyFieldNames().size();
+        Index.TextIndexDetails indexDetails = (Index.TextIndexDetails) index.getIndexDetails();
+        boolean isOverridingKeyFieldTypes = indexDetails.isOverridingKeyFieldTypes();
+        int numSecondaryKeys = indexDetails.getKeyFieldNames().size();
         if (isOverridingKeyFieldTypes && !enforcedItemType.equals(itemType)) {
             sourceOp = createCastOp(spec, dataset.getDatasetType(), index.isEnforced());
             spec.connect(new OneToOneConnectorDescriptor(spec), primaryScanOp, 0, sourceOp, 0);
@@ -272,7 +274,7 @@
 
     private AbstractOperatorDescriptor createTokenizerOp(JobSpecification spec) throws AlgebricksException {
         int docField = NUM_TAG_FIELDS;
-        int numSecondaryKeys = index.getKeyFieldNames().size();
+        int numSecondaryKeys = ((Index.TextIndexDetails) index.getIndexDetails()).getKeyFieldNames().size();
         int[] keyFields = new int[NUM_TAG_FIELDS + numPrimaryKeys + numFilterFields];
         // set tag fields
         for (int i = 0; i < NUM_TAG_FIELDS; i++) {
diff --git a/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/utils/SecondaryCorrelatedRTreeOperationsHelper.java b/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/utils/SecondaryCorrelatedRTreeOperationsHelper.java
index d10c093..82c67d7 100644
--- a/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/utils/SecondaryCorrelatedRTreeOperationsHelper.java
+++ b/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/utils/SecondaryCorrelatedRTreeOperationsHelper.java
@@ -75,15 +75,16 @@
 
     @Override
     protected void setSecondaryRecDescAndComparators() throws AlgebricksException {
-        List<List<String>> secondaryKeyFields = index.getKeyFieldNames();
+        Index.ValueIndexDetails indexDetails = (Index.ValueIndexDetails) index.getIndexDetails();
+        List<List<String>> secondaryKeyFields = indexDetails.getKeyFieldNames();
         int numSecondaryKeys = secondaryKeyFields.size();
-        boolean isOverridingKeyFieldTypes = index.isOverridingKeyFieldTypes();
+        boolean isOverridingKeyFieldTypes = indexDetails.isOverridingKeyFieldTypes();
         if (numSecondaryKeys != 1) {
             throw AsterixException.create(ErrorCode.INDEX_RTREE_MULTIPLE_FIELDS_NOT_ALLOWED, sourceLoc,
                     numSecondaryKeys);
         }
-        Pair<IAType, Boolean> spatialTypePair =
-                Index.getNonNullableOpenFieldType(index.getKeyFieldTypes().get(0), secondaryKeyFields.get(0), itemType);
+        Pair<IAType, Boolean> spatialTypePair = Index.getNonNullableOpenFieldType(
+                indexDetails.getKeyFieldTypes().get(0), secondaryKeyFields.get(0), itemType);
         IAType spatialType = spatialTypePair.first;
         anySecondaryKeyIsNullable = spatialTypePair.second;
         isPointMBR = spatialType.getTypeTag() == ATypeTag.POINT || spatialType.getTypeTag() == ATypeTag.POINT3D;
@@ -182,7 +183,7 @@
         RecordDescriptor secondaryRecDescConsideringPointMBR = isPointMBR
                 ? getTaggedRecordDescriptor(secondaryRecDescForPointMBR) : getTaggedRecordDescriptor(secondaryRecDesc);
 
-        boolean isOverridingKeyFieldTypes = index.isOverridingKeyFieldTypes();
+        boolean isOverridingKeyFieldTypes = index.getIndexDetails().isOverridingKeyFieldTypes();
 
         assert dataset.getDatasetType() == DatasetType.INTERNAL;
 
diff --git a/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/utils/SecondaryIndexOperationsHelper.java b/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/utils/SecondaryIndexOperationsHelper.java
index fd45ff4..0eced16 100644
--- a/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/utils/SecondaryIndexOperationsHelper.java
+++ b/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/utils/SecondaryIndexOperationsHelper.java
@@ -139,7 +139,7 @@
 
     private static Pair<ARecordType, ARecordType> getEnforcedType(Index index, ARecordType aRecordType,
             ARecordType metaRecordType) throws AlgebricksException {
-        return index.isOverridingKeyFieldTypes()
+        return index.getIndexDetails().isOverridingKeyFieldTypes()
                 ? TypeUtil.createEnforcedType(aRecordType, metaRecordType, Collections.singletonList(index))
                 : new Pair<>(null, null);
     }
@@ -155,6 +155,10 @@
 
         SecondaryIndexOperationsHelper indexOperationsHelper;
         switch (index.getIndexType()) {
+            case ARRAY:
+                indexOperationsHelper =
+                        new SecondaryArrayIndexBTreeOperationsHelper(dataset, index, metadataProvider, sourceLoc);
+                break;
             case BTREE:
                 indexOperationsHelper = new SecondaryBTreeOperationsHelper(dataset, index, metadataProvider, sourceLoc);
                 break;
@@ -227,7 +231,7 @@
         secondaryFilterFields = new int[numFilterFields];
         primaryFilterFields = new int[numFilterFields];
         primaryBTreeFields = new int[numPrimaryKeys + 1];
-        secondaryBTreeFields = new int[index.getKeyFieldNames().size() + numPrimaryKeys];
+        secondaryBTreeFields = new int[getNumSecondaryKeys() + numPrimaryKeys];
         for (int i = 0; i < primaryBTreeFields.length; i++) {
             primaryBTreeFields[i] = i;
         }
diff --git a/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/utils/SecondaryInvertedIndexOperationsHelper.java b/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/utils/SecondaryInvertedIndexOperationsHelper.java
index 6329d0d..2d5a4f8 100644
--- a/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/utils/SecondaryInvertedIndexOperationsHelper.java
+++ b/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/utils/SecondaryInvertedIndexOperationsHelper.java
@@ -82,14 +82,15 @@
             SourceLocation sourceLoc) throws AlgebricksException {
         super(dataset, index, metadataProvider, sourceLoc);
         this.fullTextConfigEvaluatorFactory = FullTextUtil.fetchFilterAndCreateConfigEvaluator(metadataProvider,
-                index.getDataverseName(), index.getFullTextConfigName());
+                index.getDataverseName(), ((Index.TextIndexDetails) index.getIndexDetails()).getFullTextConfigName());
     }
 
     @Override
     protected void setSecondaryRecDescAndComparators() throws AlgebricksException {
-        int numSecondaryKeys = index.getKeyFieldNames().size();
         IndexType indexType = index.getIndexType();
-        boolean isOverridingKeyFieldTypes = index.isOverridingKeyFieldTypes();
+        Index.TextIndexDetails indexDetails = (Index.TextIndexDetails) index.getIndexDetails();
+        int numSecondaryKeys = indexDetails.getKeyFieldNames().size();
+        boolean isOverridingKeyFieldTypes = indexDetails.isOverridingKeyFieldTypes();
         // Sanity checks.
         if (numPrimaryKeys > 1) {
             throw new CompilationException(ErrorCode.COMPILATION_ILLEGAL_INDEX_FOR_DATASET_WITH_COMPOSITE_PRIMARY_INDEX,
@@ -118,9 +119,9 @@
         if (numSecondaryKeys > 0) {
             secondaryFieldAccessEvalFactories[0] = metadataProvider.getDataFormat().getFieldAccessEvaluatorFactory(
                     metadataProvider.getFunctionManager(), isOverridingKeyFieldTypes ? enforcedItemType : itemType,
-                    index.getKeyFieldNames().get(0), numPrimaryKeys, sourceLoc);
-            Pair<IAType, Boolean> keyTypePair = Index.getNonNullableOpenFieldType(index.getKeyFieldTypes().get(0),
-                    index.getKeyFieldNames().get(0), itemType);
+                    indexDetails.getKeyFieldNames().get(0), numPrimaryKeys, sourceLoc);
+            Pair<IAType, Boolean> keyTypePair = Index.getNonNullableOpenFieldType(
+                    indexDetails.getKeyFieldTypes().get(0), indexDetails.getKeyFieldNames().get(0), itemType);
             secondaryKeyType = keyTypePair.first;
             anySecondaryKeyIsNullable = anySecondaryKeyIsNullable || keyTypePair.second;
             ISerializerDeserializer keySerde = serdeProvider.getSerializerDeserializer(secondaryKeyType);
@@ -155,7 +156,7 @@
         // TODO: We might want to expose the hashing option at the AQL level,
         // and add the choice to the index metadata.
         tokenizerFactory = NonTaggedFormatUtil.getBinaryTokenizerFactory(secondaryKeyType.getTypeTag(), indexType,
-                index.getGramLength());
+                ((Index.TextIndexDetails) indexDetails).getGramLength());
         // Type traits for inverted-list elements. Inverted lists contain
         // primary keys.
         invListsTypeTraits = new ITypeTraits[numPrimaryKeys];
@@ -223,8 +224,8 @@
         IOperatorDescriptor primaryScanOp = DatasetUtil.createPrimaryIndexScanOp(spec, metadataProvider, dataset);
 
         IOperatorDescriptor sourceOp = primaryScanOp;
-        boolean isOverridingKeyFieldTypes = index.isOverridingKeyFieldTypes();
-        int numSecondaryKeys = index.getKeyFieldNames().size();
+        boolean isOverridingKeyFieldTypes = index.getIndexDetails().isOverridingKeyFieldTypes();
+        int numSecondaryKeys = ((Index.TextIndexDetails) index.getIndexDetails()).getKeyFieldNames().size();
         if (isOverridingKeyFieldTypes && !enforcedItemType.equals(itemType)) {
             sourceOp = createCastOp(spec, dataset.getDatasetType(), index.isEnforced());
             spec.connect(new OneToOneConnectorDescriptor(spec), primaryScanOp, 0, sourceOp, 0);
@@ -274,7 +275,7 @@
 
     private AbstractOperatorDescriptor createTokenizerOp(JobSpecification spec) {
         int docField = 0;
-        int numSecondaryKeys = index.getKeyFieldNames().size();
+        int numSecondaryKeys = ((Index.TextIndexDetails) index.getIndexDetails()).getKeyFieldNames().size();
         int[] primaryKeyFields = new int[numPrimaryKeys + numFilterFields];
         for (int i = 0; i < primaryKeyFields.length; i++) {
             primaryKeyFields[i] = numSecondaryKeys + i;
diff --git a/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/utils/SecondaryRTreeOperationsHelper.java b/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/utils/SecondaryRTreeOperationsHelper.java
index 1d60772..dbbd723 100644
--- a/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/utils/SecondaryRTreeOperationsHelper.java
+++ b/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/utils/SecondaryRTreeOperationsHelper.java
@@ -81,15 +81,16 @@
 
     @Override
     protected void setSecondaryRecDescAndComparators() throws AlgebricksException {
-        List<List<String>> secondaryKeyFields = index.getKeyFieldNames();
+        Index.ValueIndexDetails indexDetails = (Index.ValueIndexDetails) index.getIndexDetails();
+        List<List<String>> secondaryKeyFields = indexDetails.getKeyFieldNames();
         int numSecondaryKeys = secondaryKeyFields.size();
-        boolean isOverridingKeyFieldTypes = index.isOverridingKeyFieldTypes();
+        boolean isOverridingKeyFieldTypes = indexDetails.isOverridingKeyFieldTypes();
         if (numSecondaryKeys != 1) {
             throw new AsterixException("Cannot use " + numSecondaryKeys + " fields as a key for the R-tree index. "
                     + "There can be only one field as a key for the R-tree index.");
         }
-        Pair<IAType, Boolean> spatialTypePair =
-                Index.getNonNullableOpenFieldType(index.getKeyFieldTypes().get(0), secondaryKeyFields.get(0), itemType);
+        Pair<IAType, Boolean> spatialTypePair = Index.getNonNullableOpenFieldType(
+                indexDetails.getKeyFieldTypes().get(0), secondaryKeyFields.get(0), itemType);
         IAType spatialType = spatialTypePair.first;
         anySecondaryKeyIsNullable = spatialTypePair.second;
         if (spatialType == null) {
@@ -198,7 +199,7 @@
                 isPointMBR ? numNestedSecondaryKeyFields / 2 : numNestedSecondaryKeyFields;
         RecordDescriptor secondaryRecDescConsideringPointMBR =
                 isPointMBR ? secondaryRecDescForPointMBR : secondaryRecDesc;
-        boolean isOverridingKeyFieldTypes = index.isOverridingKeyFieldTypes();
+        boolean isOverridingKeyFieldTypes = index.getIndexDetails().isOverridingKeyFieldTypes();
         IIndexDataflowHelperFactory indexDataflowHelperFactory = new IndexDataflowHelperFactory(
                 metadataProvider.getStorageComponentProvider().getStorageManager(), secondaryFileSplitProvider);
         if (dataset.getDatasetType() == DatasetType.INTERNAL) {
diff --git a/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/utils/TypeUtil.java b/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/utils/TypeUtil.java
index 7660909..c32290f 100644
--- a/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/utils/TypeUtil.java
+++ b/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/utils/TypeUtil.java
@@ -27,12 +27,14 @@
 import java.util.Map;
 
 import org.apache.asterix.common.exceptions.AsterixException;
+import org.apache.asterix.common.exceptions.CompilationException;
 import org.apache.asterix.common.exceptions.ErrorCode;
 import org.apache.asterix.common.metadata.DataverseName;
 import org.apache.asterix.metadata.entities.Dataset;
 import org.apache.asterix.metadata.entities.Function;
 import org.apache.asterix.metadata.entities.Index;
 import org.apache.asterix.om.typecomputer.impl.TypeComputeUtils;
+import org.apache.asterix.om.types.AOrderedListType;
 import org.apache.asterix.om.types.ARecordType;
 import org.apache.asterix.om.types.ATypeTag;
 import org.apache.asterix.om.types.AUnionType;
@@ -42,6 +44,7 @@
 import org.apache.commons.lang3.ArrayUtils;
 import org.apache.hyracks.algebricks.common.exceptions.AlgebricksException;
 import org.apache.hyracks.algebricks.common.utils.Pair;
+import org.apache.hyracks.algebricks.common.utils.Triple;
 
 /**
  * Provider utility methods for data types
@@ -57,6 +60,195 @@
     private TypeUtil() {
     }
 
+    private static class EnforcedTypeBuilder {
+        private final Deque<Triple<IAType, String, Integer>> typeStack = new ArrayDeque<>();
+        private List<Integer> keyDepthIndicators;
+        private List<String> keyFieldNames;
+        private ARecordType baseRecordType;
+        private IAType keyFieldType;
+
+        // Output from nested-type-stack construction.
+        private String bridgeNameFoundFromOpenTypeBuild;
+        private IAType endOfOpenTypeBuild;
+        private int indexOfOpenPart;
+
+        public void reset(ARecordType baseRecordType, List<String> keyFieldNames, List<Integer> keyDepthIndicators,
+                IAType keyFieldType) {
+            this.baseRecordType = baseRecordType;
+            this.keyFieldNames = keyFieldNames;
+            this.keyDepthIndicators = keyDepthIndicators;
+            this.keyFieldType = keyFieldType;
+        }
+
+        public ARecordType build() throws AlgebricksException {
+            boolean isOpen = constructNestedTypeStack();
+            IAType newTypeToAdd = (isOpen) ? buildNewForOpenType() : buildNewForFullyClosedType();
+            return buildRestOfRecord(newTypeToAdd);
+        }
+
+        private boolean constructNestedTypeStack() throws AlgebricksException {
+            IAType typeIntermediate = baseRecordType;
+            List<String> subFieldName = new ArrayList<>();
+            for (int i = 0; i < keyFieldNames.size() - 1; i++) {
+                typeStack.push(new Triple<>(typeIntermediate, keyFieldNames.get(i),
+                        (i == 0) ? 0 : keyDepthIndicators.get(i - 1)));
+                bridgeNameFoundFromOpenTypeBuild = typeIntermediate.getTypeName();
+
+                if (i == 0 || keyDepthIndicators.get(i - 1) == 0) {
+                    subFieldName.add(keyFieldNames.get(i));
+                } else {
+                    // We have a multi-valued intermediate. Traverse the array first, then add our field name.
+                    for (int j = 0; j < keyDepthIndicators.get(i - 1); j++) {
+                        typeIntermediate = TypeComputeUtils.extractListItemType(typeIntermediate);
+                        if (typeIntermediate == null) {
+                            String fName = String.join(".", subFieldName);
+                            throw new AsterixException(ErrorCode.COMPILATION_ERROR,
+                                    "Wrong level of array nesting for field: " + fName);
+                        }
+                    }
+                    subFieldName.add(keyFieldNames.get(i));
+                }
+
+                // Attempt to resolve the type of our working subfield.
+                typeIntermediate = TypeComputeUtils.getActualType(typeIntermediate);
+                typeIntermediate =
+                        ((ARecordType) typeIntermediate).getSubFieldType(subFieldName.subList(i, subFieldName.size()));
+
+                if (typeIntermediate == null) {
+                    endOfOpenTypeBuild = null;
+                    indexOfOpenPart = i;
+                    return true;
+                }
+                ATypeTag tt = TypeComputeUtils.getActualType(typeIntermediate).getTypeTag();
+                if (tt != ATypeTag.OBJECT && tt != ATypeTag.ARRAY && tt != ATypeTag.MULTISET) {
+                    String fName = String.join(".", subFieldName);
+                    throw new AsterixException(ErrorCode.COMPILATION_ERROR,
+                            "Field accessor is not defined for \"" + fName + "\" of type " + tt);
+                }
+            }
+
+            endOfOpenTypeBuild = typeIntermediate;
+            indexOfOpenPart = keyFieldNames.size() - 1;
+            return false;
+        }
+
+        private IAType buildNewForOpenType() {
+            int depthOfOpenType = keyDepthIndicators.subList(indexOfOpenPart + 1, keyDepthIndicators.size()).stream()
+                    .filter(i -> i != 0).findFirst().orElse(0);
+            IAType resultant = nestArrayType(keyFieldType, depthOfOpenType);
+
+            // Build the type (list or record) that holds the type (list or record) above.
+            resultant = nestArrayType(
+                    new ARecordType(keyFieldNames.get(keyFieldNames.size() - 2),
+                            new String[] { keyFieldNames.get(keyFieldNames.size() - 1) },
+                            new IAType[] { AUnionType.createUnknownableType(resultant) }, true),
+                    keyDepthIndicators.get(indexOfOpenPart));
+
+            // Create open part of the nested field.
+            for (int i = keyFieldNames.size() - 3; i > (indexOfOpenPart - 1); i--) {
+                resultant = nestArrayType(
+                        new ARecordType(keyFieldNames.get(i), new String[] { keyFieldNames.get(i + 1) },
+                                new IAType[] { AUnionType.createUnknownableType(resultant) }, true),
+                        keyDepthIndicators.get(i));
+            }
+
+            // Now update the parent to include this optional field, accounting for intermediate arrays.
+            Triple<IAType, String, Integer> gapTriple = this.typeStack.pop();
+            ARecordType parentRecord =
+                    (ARecordType) unnestArrayType(TypeComputeUtils.getActualType(gapTriple.first), gapTriple.third);
+            IAType[] parentFieldTypes = ArrayUtils.addAll(parentRecord.getFieldTypes().clone(),
+                    AUnionType.createUnknownableType(resultant));
+            resultant = new ARecordType(bridgeNameFoundFromOpenTypeBuild,
+                    ArrayUtils.addAll(parentRecord.getFieldNames(), resultant.getTypeName()), parentFieldTypes, true);
+            resultant = keepUnknown(gapTriple.first, nestArrayType(resultant, gapTriple.third));
+
+            return resultant;
+        }
+
+        private IAType buildNewForFullyClosedType() throws AsterixException {
+            // The schema is closed all the way to the field itself.
+            IAType typeIntermediate = TypeComputeUtils.getActualType(endOfOpenTypeBuild);
+            int depthOfOpenType = (indexOfOpenPart == 0) ? 0 : keyDepthIndicators.get(indexOfOpenPart - 1);
+            int depthOfKeyType = keyDepthIndicators.get(indexOfOpenPart);
+            ARecordType lastNestedRecord = (ARecordType) unnestArrayType(typeIntermediate, depthOfOpenType);
+            Map<String, IAType> recordNameTypesMap = createRecordNameTypeMap(lastNestedRecord);
+
+            // If an enforced field already exists, verify that the type is correct.
+            IAType enforcedFieldType = recordNameTypesMap.get(keyFieldNames.get(keyFieldNames.size() - 1));
+            if (enforcedFieldType != null && enforcedFieldType.getTypeTag() == ATypeTag.UNION
+                    && ((AUnionType) enforcedFieldType).isUnknownableType()) {
+                enforcedFieldType = ((AUnionType) enforcedFieldType).getActualType();
+            }
+            if (enforcedFieldType != null
+                    && !ATypeHierarchy.canPromote(enforcedFieldType.getTypeTag(), this.keyFieldType.getTypeTag())) {
+                throw new AsterixException(ErrorCode.COMPILATION_ERROR, "Cannot enforce field \""
+                        + String.join(".", this.keyFieldNames) + "\" to have type " + this.keyFieldType);
+            }
+            if (enforcedFieldType == null) {
+                recordNameTypesMap.put(keyFieldNames.get(keyFieldNames.size() - 1),
+                        AUnionType.createUnknownableType(nestArrayType(keyFieldType, depthOfKeyType)));
+            }
+
+            // Build the nested record, and account for the wrapping array.
+            IAType resultant = nestArrayType(
+                    new ARecordType(lastNestedRecord.getTypeName(), recordNameTypesMap.keySet().toArray(new String[0]),
+                            recordNameTypesMap.values().toArray(new IAType[0]), lastNestedRecord.isOpen()),
+                    depthOfOpenType);
+            return keepUnknown(endOfOpenTypeBuild, resultant);
+        }
+
+        private ARecordType buildRestOfRecord(IAType newTypeToAdd) {
+            IAType resultant = TypeComputeUtils.getActualType(newTypeToAdd);
+            while (!typeStack.isEmpty()) {
+                Triple<IAType, String, Integer> typeFromStack = typeStack.pop();
+                IAType typeIntermediate = unnestArrayType(typeFromStack.first, typeFromStack.third);
+                ARecordType recordType = (ARecordType) typeIntermediate;
+                IAType[] fieldTypes = recordType.getFieldTypes().clone();
+                fieldTypes[recordType.getFieldIndex(typeFromStack.second)] = resultant;
+
+                typeIntermediate = nestArrayType(new ARecordType(recordType.getTypeName() + "_enforced",
+                        recordType.getFieldNames(), fieldTypes, recordType.isOpen()), typeFromStack.third);
+                resultant = keepUnknown(typeFromStack.first, typeIntermediate);
+            }
+            return (ARecordType) resultant;
+        }
+
+        private static Map<String, IAType> createRecordNameTypeMap(ARecordType recordType) {
+            LinkedHashMap<String, IAType> recordNameTypesMap = new LinkedHashMap<>();
+            for (int j = 0; j < recordType.getFieldNames().length; j++) {
+                recordNameTypesMap.put(recordType.getFieldNames()[j], recordType.getFieldTypes()[j]);
+            }
+            return recordNameTypesMap;
+        }
+
+        private static IAType keepUnknown(IAType originalRecordType, IAType updatedRecordType) {
+            if (originalRecordType.getTypeTag() == ATypeTag.UNION) {
+                return AUnionType.createUnknownableType(updatedRecordType, updatedRecordType.getTypeName());
+            }
+            return updatedRecordType;
+        }
+
+        private static IAType nestArrayType(IAType originalType, int depthOfArrays) {
+            IAType resultant = originalType;
+            for (int i = 0; i < depthOfArrays; i++) {
+                resultant =
+                        new AOrderedListType(resultant, (i == depthOfArrays - 1) ? originalType.getTypeName() : null);
+            }
+            return resultant;
+        }
+
+        private static IAType unnestArrayType(IAType originalType, int depthOfArrays) {
+            IAType resultant = originalType;
+            for (int i = 0; i < depthOfArrays; i++) {
+                resultant = TypeComputeUtils.extractListItemType(resultant);
+                if (resultant != null) {
+                    resultant = TypeComputeUtils.getActualType(resultant);
+                }
+            }
+            return resultant;
+        }
+    }
+
     /**
      * Merges typed index fields with specified recordType, allowing indexed fields to be optional.
      * I.e. the type { "personId":int32, "name": string, "address" : { "street": string } } with typed indexes
@@ -66,109 +258,97 @@
      */
     public static Pair<ARecordType, ARecordType> createEnforcedType(ARecordType recordType, ARecordType metaType,
             List<Index> indexes) throws AlgebricksException {
-        IAType enforcedRecordType = recordType;
-        ARecordType enforcedMetaType = metaType;
-        List<String> subFieldName;
+        EnforcedTypeBuilder enforcedTypeBuilder = new EnforcedTypeBuilder();
+        ARecordType enforcedRecordType = recordType;
         for (Index index : indexes) {
-            if (!index.isSecondaryIndex() || !index.isOverridingKeyFieldTypes()) {
+            if (!index.isSecondaryIndex() || !index.getIndexDetails().isOverridingKeyFieldTypes()) {
                 continue;
             }
-            if (index.hasMetaFields()) {
-                throw new AlgebricksException("Indexing an open field is only supported on the record part");
-            }
-            for (int i = 0; i < index.getKeyFieldNames().size(); i++) {
-                // keeps track of a record type and a field name in that record type
-                Deque<Pair<IAType, String>> nestedTypeStack = new ArrayDeque<>();
-                List<String> splits = index.getKeyFieldNames().get(i);
-                IAType nestedFieldType = enforcedRecordType;
-                boolean openRecords = false;
-                String bridgeName = nestedFieldType.getTypeName();
-                int j;
-                // enforcedRecordType must always be/stay as ARecordType
-                validateRecord(enforcedRecordType);
-                // build the stack for the enforced type, stack of a mixture of ARecord and AUnion(ARecord) types
-                // try to build up to the last record field, e.g. for a.b.c.d.e, build up to and including "d"
-                for (j = 1; j < splits.size(); j++) {
-                    nestedTypeStack.push(new Pair<>(nestedFieldType, splits.get(j - 1)));
-                    bridgeName = nestedFieldType.getTypeName();
-                    subFieldName = splits.subList(0, j);
-                    nestedFieldType = ((ARecordType) enforcedRecordType).getSubFieldType(subFieldName);
-                    if (nestedFieldType == null) {
-                        openRecords = true;
-                        break;
-                    }
-                    // nestedFieldType (i.e. nested record field) must be either ARecordType or AUnion(ARecordType)
-                    validateNestedRecord(nestedFieldType, subFieldName);
-                }
-                if (openRecords) {
-                    // create the smallest record
-                    enforcedRecordType = new ARecordType(splits.get(splits.size() - 2),
-                            new String[] { splits.get(splits.size() - 1) },
-                            new IAType[] { AUnionType.createUnknownableType(index.getKeyFieldTypes().get(i)) }, true);
-                    // create the open part of the nested field
-                    for (int k = splits.size() - 3; k > (j - 2); k--) {
-                        enforcedRecordType = new ARecordType(splits.get(k), new String[] { splits.get(k + 1) },
-                                new IAType[] { AUnionType.createUnknownableType(enforcedRecordType) }, true);
-                    }
-                    // bridge the gap. Update the parent type to include the new optional field, e.g. c.d.e
-                    Pair<IAType, String> gapPair = nestedTypeStack.pop();
-                    ARecordType parent = (ARecordType) TypeComputeUtils.getActualType(gapPair.first);
-
-                    // parent type must be "open" to allow inclusion of the non-declared field
-                    IAType[] parentFieldTypes = ArrayUtils.addAll(parent.getFieldTypes().clone(),
-                            new IAType[] { AUnionType.createUnknownableType(enforcedRecordType) });
-                    enforcedRecordType = new ARecordType(bridgeName,
-                            ArrayUtils.addAll(parent.getFieldNames(), enforcedRecordType.getTypeName()),
-                            parentFieldTypes, true);
-                    // make nullable/missable if the original parent was nullable/missable
-                    enforcedRecordType = keepUnknown(gapPair.first, (ARecordType) enforcedRecordType);
-                } else {
-                    // schema is closed all the way to the field. Enforced fields are either null or strongly typed
-                    // e.g. nestedFieldType = a.b.c.d
-                    ARecordType lastNestedRecord = (ARecordType) TypeComputeUtils.getActualType(nestedFieldType);
-                    Map<String, IAType> recordNameTypesMap = TypeUtil.createRecordNameTypeMap(lastNestedRecord);
-                    // if a an enforced field already exists and the type is correct
-                    IAType enforcedFieldType = recordNameTypesMap.get(splits.get(splits.size() - 1));
-                    if (enforcedFieldType != null && enforcedFieldType.getTypeTag() == ATypeTag.UNION
-                            && ((AUnionType) enforcedFieldType).isUnknownableType()) {
-                        enforcedFieldType = ((AUnionType) enforcedFieldType).getActualType();
-                    }
-                    if (enforcedFieldType != null && !ATypeHierarchy.canPromote(enforcedFieldType.getTypeTag(),
-                            index.getKeyFieldTypes().get(i).getTypeTag())) {
-                        throw new AsterixException(ErrorCode.COMPILATION_ERROR,
-                                "Cannot enforce field \"" + String.join(".", index.getKeyFieldNames().get(i))
-                                        + "\" to have type " + index.getKeyFieldTypes().get(i));
-                    }
-                    if (enforcedFieldType == null) {
-                        recordNameTypesMap.put(splits.get(splits.size() - 1),
-                                AUnionType.createUnknownableType(index.getKeyFieldTypes().get(i)));
-                    }
-                    enforcedRecordType = new ARecordType(lastNestedRecord.getTypeName(),
-                            recordNameTypesMap.keySet().toArray(new String[recordNameTypesMap.size()]),
-                            recordNameTypesMap.values().toArray(new IAType[recordNameTypesMap.size()]),
-                            lastNestedRecord.isOpen());
-                    // make nullable/missable if the original nestedFieldType was nullable/missable
-                    enforcedRecordType = keepUnknown(nestedFieldType, (ARecordType) enforcedRecordType);
-                }
-
-                // Create the enforced type for the nested fields in the schema, from the ground up
-                if (!nestedTypeStack.isEmpty()) {
-                    while (!nestedTypeStack.isEmpty()) {
-                        Pair<IAType, String> nestedType = nestedTypeStack.pop();
-                        ARecordType nestedRecType = (ARecordType) TypeComputeUtils.getActualType(nestedType.first);
-                        IAType[] nestedRecTypeFieldTypes = nestedRecType.getFieldTypes().clone();
-                        nestedRecTypeFieldTypes[nestedRecType.getFieldIndex(nestedType.second)] = enforcedRecordType;
-                        enforcedRecordType = new ARecordType(nestedRecType.getTypeName() + "_enforced",
-                                nestedRecType.getFieldNames(), nestedRecTypeFieldTypes, nestedRecType.isOpen());
-                        // make nullable/missable if the original nestedRecType was nullable/missable
-                        enforcedRecordType = keepUnknown(nestedType.first, (ARecordType) enforcedRecordType);
-                    }
-                }
+            switch (Index.IndexCategory.of(index.getIndexType())) {
+                case VALUE:
+                    enforcedRecordType = appendValueIndexType(enforcedRecordType,
+                            (Index.ValueIndexDetails) index.getIndexDetails(), enforcedTypeBuilder);
+                    break;
+                case TEXT:
+                    enforcedRecordType = appendTextIndexType(enforcedRecordType,
+                            (Index.TextIndexDetails) index.getIndexDetails(), enforcedTypeBuilder);
+                    break;
+                case ARRAY:
+                    enforcedRecordType = appendArrayIndexTypes(enforcedRecordType,
+                            (Index.ArrayIndexDetails) index.getIndexDetails(), enforcedTypeBuilder);
+                    break;
+                default:
+                    throw new CompilationException(ErrorCode.COMPILATION_UNKNOWN_INDEX_TYPE,
+                            String.valueOf(index.getIndexType()));
             }
         }
-        // the final enforcedRecordType built must be ARecordType since the original dataset rec. type can't be nullable
+
         validateRecord(enforcedRecordType);
-        return new Pair<>((ARecordType) enforcedRecordType, enforcedMetaType);
+        return new Pair<>(enforcedRecordType, metaType);
+    }
+
+    private static ARecordType appendValueIndexType(ARecordType enforcedRecordType,
+            Index.ValueIndexDetails valueIndexDetails, EnforcedTypeBuilder enforcedTypeBuilder)
+            throws AlgebricksException {
+        List<List<String>> keyFieldNames = valueIndexDetails.getKeyFieldNames();
+        List<IAType> keyFieldTypes = valueIndexDetails.getKeyFieldTypes();
+        List<Integer> keySources = valueIndexDetails.getKeyFieldSourceIndicators();
+        for (int i = 0; i < keyFieldNames.size(); i++) {
+            if (keySources.get(i) != Index.RECORD_INDICATOR) {
+                throw new CompilationException(ErrorCode.COMPILATION_ERROR,
+                        "Indexing an open field is only supported on the record part");
+            }
+            enforcedTypeBuilder.reset(enforcedRecordType, keyFieldNames.get(i),
+                    Collections.nCopies(keyFieldNames.get(i).size(), 0), keyFieldTypes.get(i));
+            validateRecord(enforcedRecordType);
+            enforcedRecordType = enforcedTypeBuilder.build();
+        }
+
+        return enforcedRecordType;
+    }
+
+    private static ARecordType appendTextIndexType(ARecordType enforcedRecordType,
+            Index.TextIndexDetails textIndexDetails, EnforcedTypeBuilder enforcedTypeBuilder)
+            throws AlgebricksException {
+        List<List<String>> keyFieldNames = textIndexDetails.getKeyFieldNames();
+        List<IAType> keyFieldTypes = textIndexDetails.getKeyFieldTypes();
+        List<Integer> keySources = textIndexDetails.getKeyFieldSourceIndicators();
+        for (int i = 0; i < keyFieldNames.size(); i++) {
+            if (keySources.get(i) != Index.RECORD_INDICATOR) {
+                throw new CompilationException(ErrorCode.COMPILATION_ERROR,
+                        "Indexing an open field is only supported on the record part");
+            }
+            enforcedTypeBuilder.reset(enforcedRecordType, keyFieldNames.get(i),
+                    Collections.nCopies(keyFieldNames.get(i).size(), 0), keyFieldTypes.get(i));
+            validateRecord(enforcedRecordType);
+            enforcedRecordType = enforcedTypeBuilder.build();
+        }
+
+        return enforcedRecordType;
+    }
+
+    private static ARecordType appendArrayIndexTypes(ARecordType enforcedRecordType,
+            Index.ArrayIndexDetails arrayIndexDetails, EnforcedTypeBuilder enforcedTypeBuilder)
+            throws AlgebricksException {
+        for (Index.ArrayIndexElement e : arrayIndexDetails.getElementList()) {
+            if (e.getSourceIndicator() != Index.RECORD_INDICATOR) {
+                throw new CompilationException(ErrorCode.COMPILATION_ERROR,
+                        "Indexing an open field is only supported on the record part");
+            }
+            List<List<String>> unnestList = e.getUnnestList();
+            List<List<String>> projectList = e.getProjectList();
+            List<IAType> typeList = e.getTypeList();
+            for (int i = 0; i < projectList.size(); i++) {
+                List<String> project = projectList.get(i);
+                enforcedTypeBuilder.reset(enforcedRecordType,
+                        ArrayIndexUtil.getFlattenedKeyFieldNames(unnestList, project),
+                        ArrayIndexUtil.getArrayDepthIndicator(unnestList, project), typeList.get(i));
+                validateRecord(enforcedRecordType);
+                enforcedRecordType = enforcedTypeBuilder.build();
+            }
+        }
+
+        return enforcedRecordType;
     }
 
     /**
diff --git a/asterixdb/asterix-metadata/src/test/java/org/apache/asterix/metadata/entitytupletranslators/IndexTupleTranslatorTest.java b/asterixdb/asterix-metadata/src/test/java/org/apache/asterix/metadata/entitytupletranslators/IndexTupleTranslatorTest.java
index 13524a2..f690018 100644
--- a/asterixdb/asterix-metadata/src/test/java/org/apache/asterix/metadata/entitytupletranslators/IndexTupleTranslatorTest.java
+++ b/asterixdb/asterix-metadata/src/test/java/org/apache/asterix/metadata/entitytupletranslators/IndexTupleTranslatorTest.java
@@ -73,7 +73,7 @@
             Index index = new Index(dvTest, "d1", "i1", IndexType.BTREE,
                     Collections.singletonList(Collections.singletonList("row_id")),
                     indicator == null ? null : Collections.singletonList(indicator),
-                    Collections.singletonList(BuiltinType.AINT64), -1, null, false, false, false, 0);
+                    Collections.singletonList(BuiltinType.AINT64), false, false, false, 0);
 
             MetadataNode mockMetadataNode = mock(MetadataNode.class);
             when(mockMetadataNode.getDatatype(any(), any(DataverseName.class), anyString())).thenReturn(new Datatype(
@@ -85,11 +85,11 @@
             ITupleReference tuple = idxTranslator.getTupleFromMetadataEntity(index);
             Index deserializedIndex = idxTranslator.getMetadataEntityFromTuple(tuple);
             if (indicator == null) {
-                Assert.assertEquals(Collections.singletonList(new Integer(0)),
-                        deserializedIndex.getKeyFieldSourceIndicators());
+                Assert.assertEquals(Collections.singletonList(0),
+                        ((Index.ValueIndexDetails) deserializedIndex.getIndexDetails()).getKeyFieldSourceIndicators());
             } else {
-                Assert.assertEquals(index.getKeyFieldSourceIndicators(),
-                        deserializedIndex.getKeyFieldSourceIndicators());
+                Assert.assertEquals(((Index.ValueIndexDetails) index.getIndexDetails()).getKeyFieldSourceIndicators(),
+                        ((Index.ValueIndexDetails) deserializedIndex.getIndexDetails()).getKeyFieldSourceIndicators());
             }
         }
     }
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/operators/LSMSecondaryInsertDeleteWithNestedPlanOperatorDescriptor.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/operators/LSMSecondaryInsertDeleteWithNestedPlanOperatorDescriptor.java
new file mode 100644
index 0000000..ae03be2
--- /dev/null
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/operators/LSMSecondaryInsertDeleteWithNestedPlanOperatorDescriptor.java
@@ -0,0 +1,54 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.asterix.runtime.operators;
+
+import java.util.List;
+
+import org.apache.hyracks.algebricks.runtime.base.AlgebricksPipeline;
+import org.apache.hyracks.api.context.IHyracksTaskContext;
+import org.apache.hyracks.api.dataflow.IOperatorNodePushable;
+import org.apache.hyracks.api.dataflow.value.IRecordDescriptorProvider;
+import org.apache.hyracks.api.dataflow.value.RecordDescriptor;
+import org.apache.hyracks.api.exceptions.HyracksDataException;
+import org.apache.hyracks.api.job.JobSpecification;
+import org.apache.hyracks.storage.am.common.api.IModificationOperationCallbackFactory;
+import org.apache.hyracks.storage.am.common.dataflow.IIndexDataflowHelperFactory;
+import org.apache.hyracks.storage.am.common.ophelpers.IndexOperation;
+import org.apache.hyracks.storage.am.lsm.common.dataflow.LSMTreeIndexInsertUpdateDeleteOperatorDescriptor;
+
+public class LSMSecondaryInsertDeleteWithNestedPlanOperatorDescriptor
+        extends LSMTreeIndexInsertUpdateDeleteOperatorDescriptor {
+    private static final long serialVersionUID = 1L;
+    private final List<AlgebricksPipeline> secondaryKeysPipeline;
+
+    public LSMSecondaryInsertDeleteWithNestedPlanOperatorDescriptor(JobSpecification spec, RecordDescriptor outRecDesc,
+            int[] fieldPermutation, IndexOperation op, IIndexDataflowHelperFactory indexHelperFactory,
+            IModificationOperationCallbackFactory modCallbackFactory, List<AlgebricksPipeline> secondaryKeysPipeline) {
+        super(spec, outRecDesc, indexHelperFactory, fieldPermutation, op, modCallbackFactory, null);
+        this.secondaryKeysPipeline = secondaryKeysPipeline;
+    }
+
+    @Override
+    public IOperatorNodePushable createPushRuntime(IHyracksTaskContext ctx,
+            IRecordDescriptorProvider recordDescProvider, int partition, int nPartitions) throws HyracksDataException {
+        RecordDescriptor inputRecDesc = recordDescProvider.getInputRecordDescriptor(getActivityId(), 0);
+        return new LSMSecondaryInsertDeleteWithNestedPlanOperatorNodePushable(ctx, partition, fieldPermutation,
+                inputRecDesc, op, indexHelperFactory, modCallbackFactory, secondaryKeysPipeline);
+    }
+}
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/operators/LSMSecondaryInsertDeleteWithNestedPlanOperatorNodePushable.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/operators/LSMSecondaryInsertDeleteWithNestedPlanOperatorNodePushable.java
new file mode 100644
index 0000000..0b36774
--- /dev/null
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/operators/LSMSecondaryInsertDeleteWithNestedPlanOperatorNodePushable.java
@@ -0,0 +1,181 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.asterix.runtime.operators;
+
+import java.nio.ByteBuffer;
+import java.util.List;
+
+import org.apache.asterix.common.exceptions.ErrorCode;
+import org.apache.asterix.common.exceptions.RuntimeDataException;
+import org.apache.hyracks.algebricks.runtime.base.AlgebricksPipeline;
+import org.apache.hyracks.algebricks.runtime.base.IPushRuntime;
+import org.apache.hyracks.algebricks.runtime.operators.meta.PipelineAssembler;
+import org.apache.hyracks.algebricks.runtime.operators.std.NestedTupleSourceRuntimeFactory.NestedTupleSourceRuntime;
+import org.apache.hyracks.api.comm.IFrameWriter;
+import org.apache.hyracks.api.context.IHyracksTaskContext;
+import org.apache.hyracks.api.dataflow.value.RecordDescriptor;
+import org.apache.hyracks.api.exceptions.HyracksDataException;
+import org.apache.hyracks.dataflow.common.comm.io.ArrayTupleBuilder;
+import org.apache.hyracks.dataflow.common.comm.io.ArrayTupleReference;
+import org.apache.hyracks.dataflow.common.comm.io.FrameTupleAccessor;
+import org.apache.hyracks.dataflow.common.comm.util.FrameUtils;
+import org.apache.hyracks.storage.am.common.api.IModificationOperationCallbackFactory;
+import org.apache.hyracks.storage.am.common.dataflow.IIndexDataflowHelperFactory;
+import org.apache.hyracks.storage.am.common.ophelpers.IndexOperation;
+import org.apache.hyracks.storage.am.lsm.common.api.ILSMIndexAccessor;
+import org.apache.hyracks.storage.am.lsm.common.dataflow.LSMIndexInsertUpdateDeleteOperatorNodePushable;
+
+public class LSMSecondaryInsertDeleteWithNestedPlanOperatorNodePushable
+        extends LSMIndexInsertUpdateDeleteOperatorNodePushable {
+    private final NestedTupleSourceRuntime[] startOfPipelines;
+    private final int numberOfPrimaryKeyAndFilterFields;
+
+    public LSMSecondaryInsertDeleteWithNestedPlanOperatorNodePushable(IHyracksTaskContext ctx, int partition,
+            int[] fieldPermutation, RecordDescriptor inputRecDesc, IndexOperation op,
+            IIndexDataflowHelperFactory indexHelperFactory, IModificationOperationCallbackFactory modCallbackFactory,
+            List<AlgebricksPipeline> secondaryKeysPipeline) throws HyracksDataException {
+        super(ctx, partition, indexHelperFactory, fieldPermutation, inputRecDesc, op, modCallbackFactory, null);
+        this.numberOfPrimaryKeyAndFilterFields = fieldPermutation.length;
+
+        // Build our pipeline.
+        startOfPipelines = new NestedTupleSourceRuntime[secondaryKeysPipeline.size()];
+        PipelineAssembler[] pipelineAssemblers = new PipelineAssembler[secondaryKeysPipeline.size()];
+        for (int p = 0; p < secondaryKeysPipeline.size(); p++) {
+            AlgebricksPipeline pipeline = secondaryKeysPipeline.get(p);
+            RecordDescriptor lastRecordDescriptorInPipeline =
+                    pipeline.getRecordDescriptors()[pipeline.getRecordDescriptors().length - 1];
+
+            IFrameWriter outputWriter;
+            if (p == 0) {
+                // Primary pipeline (the first). Here we append the PK, filters to the final variable.
+                outputWriter = new IndexTupleInsertDelete(lastRecordDescriptorInPipeline);
+
+            } else {
+                IPushRuntime outputPushRuntime = PipelineAssembler.linkPipeline(pipeline, pipelineAssemblers, p);
+                if (outputPushRuntime == null) {
+                    throw new RuntimeDataException(ErrorCode.ILLEGAL_STATE, "Invalid pipeline.");
+                }
+                outputPushRuntime.setInputRecordDescriptor(0, lastRecordDescriptorInPipeline);
+                outputWriter = outputPushRuntime;
+            }
+
+            PipelineAssembler pipelineAssembler =
+                    new PipelineAssembler(pipeline, 1, 1, inputRecDesc, lastRecordDescriptorInPipeline);
+            startOfPipelines[p] = (NestedTupleSourceRuntime) pipelineAssembler.assemblePipeline(outputWriter, ctx);
+            pipelineAssemblers[p] = pipelineAssembler;
+        }
+    }
+
+    @Override
+    public void nextFrame(ByteBuffer buffer) throws HyracksDataException {
+        accessor.reset(buffer);
+        int tupleCount = accessor.getTupleCount();
+        for (int i = 0; i < tupleCount; i++) {
+            // Pass the input to our pipeline. The last operator in this pipeline will pass all of the tuples
+            // to our LSM accessor.
+            tuple.reset(accessor, i);
+
+            for (NestedTupleSourceRuntime nts : startOfPipelines) {
+                nts.writeTuple(buffer, i);
+            }
+
+            int n = 0;
+            try {
+                for (; n < startOfPipelines.length; n++) {
+                    NestedTupleSourceRuntime nts = startOfPipelines[n];
+                    try {
+                        nts.open();
+                    } catch (Exception e) {
+                        nts.fail();
+                        throw e;
+                    }
+                }
+            } finally {
+                for (int j = n - 1; j >= 0; j--) {
+                    startOfPipelines[j].close();
+                }
+            }
+        }
+
+        // No partial flushing was necessary. Forward entire frame.
+        writeBuffer.ensureFrameSize(buffer.capacity());
+        FrameUtils.copyAndFlip(buffer, writeBuffer.getBuffer());
+        FrameUtils.flushFrame(writeBuffer.getBuffer(), writer);
+    }
+
+    private class IndexTupleInsertDelete implements IFrameWriter {
+        private final RecordDescriptor inputRecordDescriptor;
+        private FrameTupleAccessor endOfPipelineTupleAccessor;
+
+        // We are not writing the resulting tuple to a frame, we must store the result in an intermediate.
+        private ArrayTupleBuilder arrayTupleBuilder;
+        private ArrayTupleReference arrayTupleReference;
+
+        private IndexTupleInsertDelete(RecordDescriptor recordDescriptor) {
+            this.inputRecordDescriptor = recordDescriptor;
+        }
+
+        @Override
+        public void open() throws HyracksDataException {
+            int numSecondaryKeys = inputRecordDescriptor.getFieldCount();
+
+            endOfPipelineTupleAccessor = new FrameTupleAccessor(inputRecordDescriptor);
+            arrayTupleBuilder = new ArrayTupleBuilder(numberOfPrimaryKeyAndFilterFields + numSecondaryKeys);
+            arrayTupleReference = new ArrayTupleReference();
+        }
+
+        @Override
+        public void nextFrame(ByteBuffer buffer) throws HyracksDataException {
+            ILSMIndexAccessor workingLSMAccessor = (ILSMIndexAccessor) indexAccessor;
+
+            endOfPipelineTupleAccessor.reset(buffer);
+            int nTuple = endOfPipelineTupleAccessor.getTupleCount();
+            for (int t = 0; t < nTuple; t++) {
+
+                // First, add the secondary keys.
+                arrayTupleBuilder.reset();
+                int nFields = endOfPipelineTupleAccessor.getFieldCount();
+                for (int f = 0; f < nFields; f++) {
+                    arrayTupleBuilder.addField(endOfPipelineTupleAccessor, t, f);
+                }
+
+                // Next, add the primary keys and filter fields.
+                for (int f = 0; f < numberOfPrimaryKeyAndFilterFields; f++) {
+                    arrayTupleBuilder.addField(tuple.getFieldData(f), tuple.getFieldStart(f), tuple.getFieldLength(f));
+                }
+
+                // Finally, pass the tuple to our accessor. There are only two operations: insert or delete.
+                arrayTupleReference.reset(arrayTupleBuilder.getFieldEndOffsets(), arrayTupleBuilder.getByteArray());
+                if (op.equals(IndexOperation.INSERT)) {
+                    workingLSMAccessor.forceInsert(arrayTupleReference);
+                } else {
+                    workingLSMAccessor.forceDelete(arrayTupleReference);
+                }
+            }
+        }
+
+        @Override
+        public void fail() throws HyracksDataException {
+        }
+
+        @Override
+        public void close() throws HyracksDataException {
+        }
+    }
+}
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/operators/LSMSecondaryUpsertOperatorDescriptor.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/operators/LSMSecondaryUpsertOperatorDescriptor.java
index df658b6..a4b4012 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/operators/LSMSecondaryUpsertOperatorDescriptor.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/operators/LSMSecondaryUpsertOperatorDescriptor.java
@@ -35,8 +35,8 @@
 
     private static final long serialVersionUID = 1L;
     private final int[] prevValuePermutation;
-    private final int upsertIndiatorFieldIndex;
-    private final IBinaryBooleanInspectorFactory upsertIndicatorInspectorFactory;
+    protected final int upsertIndicatorFieldIndex;
+    protected final IBinaryBooleanInspectorFactory upsertIndicatorInspectorFactory;
 
     public LSMSecondaryUpsertOperatorDescriptor(IOperatorDescriptorRegistry spec, RecordDescriptor outRecDesc,
             int[] fieldPermutation, IIndexDataflowHelperFactory indexHelperFactory,
@@ -46,7 +46,7 @@
         super(spec, outRecDesc, fieldPermutation, IndexOperation.UPSERT, indexHelperFactory, tupleFilterFactory, false,
                 modificationOpCallbackFactory);
         this.prevValuePermutation = prevValuePermutation;
-        this.upsertIndiatorFieldIndex = upsertIndicatorFieldIndex;
+        this.upsertIndicatorFieldIndex = upsertIndicatorFieldIndex;
         this.upsertIndicatorInspectorFactory = upsertIndicatorInspectorFactory;
     }
 
@@ -55,7 +55,7 @@
             IRecordDescriptorProvider recordDescProvider, int partition, int nPartitions) throws HyracksDataException {
         RecordDescriptor intputRecDesc = recordDescProvider.getInputRecordDescriptor(getActivityId(), 0);
         return new LSMSecondaryUpsertOperatorNodePushable(ctx, partition, indexHelperFactory, modCallbackFactory,
-                tupleFilterFactory, fieldPermutation, intputRecDesc, upsertIndiatorFieldIndex,
+                tupleFilterFactory, fieldPermutation, intputRecDesc, upsertIndicatorFieldIndex,
                 upsertIndicatorInspectorFactory, prevValuePermutation);
     }
 }
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/operators/LSMSecondaryUpsertOperatorNodePushable.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/operators/LSMSecondaryUpsertOperatorNodePushable.java
index 35ae904..b588323 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/operators/LSMSecondaryUpsertOperatorNodePushable.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/operators/LSMSecondaryUpsertOperatorNodePushable.java
@@ -60,12 +60,13 @@
 public class LSMSecondaryUpsertOperatorNodePushable extends LSMIndexInsertUpdateDeleteOperatorNodePushable {
 
     private final PermutingFrameTupleReference prevValueTuple = new PermutingFrameTupleReference();
-    private final int upsertIndicatorFieldIndex;
-    private final IBinaryBooleanInspector upsertIndicatorInspector;
     private final int numberOfFields;
-    private AbstractIndexModificationOperationCallback abstractModCallback;
     private final boolean isPrimaryKeyIndex;
 
+    protected final int upsertIndicatorFieldIndex;
+    protected final IBinaryBooleanInspector upsertIndicatorInspector;
+    protected AbstractIndexModificationOperationCallback abstractModCallback;
+
     public LSMSecondaryUpsertOperatorNodePushable(IHyracksTaskContext ctx, int partition,
             IIndexDataflowHelperFactory indexHelperFactory, IModificationOperationCallbackFactory modCallbackFactory,
             ITupleFilterFactory tupleFilterFactory, int[] fieldPermutation, RecordDescriptor inputRecDesc,
@@ -76,7 +77,7 @@
         this.prevValueTuple.setFieldPermutation(prevValuePermutation);
         this.upsertIndicatorFieldIndex = upsertIndicatorFieldIndex;
         this.upsertIndicatorInspector = upsertIndicatorInspectorFactory.createBinaryBooleanInspector(ctx);
-        this.numberOfFields = prevValuePermutation.length;
+        this.numberOfFields = fieldPermutation.length;
         // a primary key index only has primary keys, and thus these two permutations are the same
         this.isPrimaryKeyIndex = Arrays.equals(fieldPermutation, prevValuePermutation);
     }
@@ -137,7 +138,12 @@
         FrameUtils.flushFrame(writeBuffer.getBuffer(), writer);
     }
 
-    private boolean hasNullOrMissing(PermutingFrameTupleReference tuple) {
+    private static boolean isNullOrMissing(FrameTupleReference tuple, int fieldIdx) {
+        return TypeTagUtil.isType(tuple, fieldIdx, ATypeTag.SERIALIZED_NULL_TYPE_TAG)
+                || TypeTagUtil.isType(tuple, fieldIdx, ATypeTag.SERIALIZED_MISSING_TYPE_TAG);
+    }
+
+    protected static boolean hasNullOrMissing(FrameTupleReference tuple) {
         int fieldCount = tuple.getFieldCount();
         for (int i = 0; i < fieldCount; i++) {
             if (isNullOrMissing(tuple, i)) {
@@ -146,9 +152,4 @@
         }
         return false;
     }
-
-    private static boolean isNullOrMissing(PermutingFrameTupleReference tuple, int fieldIdx) {
-        return TypeTagUtil.isType(tuple, fieldIdx, ATypeTag.SERIALIZED_NULL_TYPE_TAG)
-                || TypeTagUtil.isType(tuple, fieldIdx, ATypeTag.SERIALIZED_MISSING_TYPE_TAG);
-    }
 }
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/operators/LSMSecondaryUpsertWithNestedPlanOperatorDescriptor.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/operators/LSMSecondaryUpsertWithNestedPlanOperatorDescriptor.java
new file mode 100644
index 0000000..d077987
--- /dev/null
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/operators/LSMSecondaryUpsertWithNestedPlanOperatorDescriptor.java
@@ -0,0 +1,59 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.asterix.runtime.operators;
+
+import java.util.List;
+
+import org.apache.hyracks.algebricks.data.IBinaryBooleanInspectorFactory;
+import org.apache.hyracks.algebricks.runtime.base.AlgebricksPipeline;
+import org.apache.hyracks.api.context.IHyracksTaskContext;
+import org.apache.hyracks.api.dataflow.IOperatorNodePushable;
+import org.apache.hyracks.api.dataflow.value.IRecordDescriptorProvider;
+import org.apache.hyracks.api.dataflow.value.RecordDescriptor;
+import org.apache.hyracks.api.exceptions.HyracksDataException;
+import org.apache.hyracks.api.job.JobSpecification;
+import org.apache.hyracks.storage.am.common.api.IModificationOperationCallbackFactory;
+import org.apache.hyracks.storage.am.common.dataflow.IIndexDataflowHelperFactory;
+
+public class LSMSecondaryUpsertWithNestedPlanOperatorDescriptor extends LSMSecondaryUpsertOperatorDescriptor {
+    private static final long serialVersionUID = 1L;
+
+    private final List<AlgebricksPipeline> secondaryKeysPipeline;
+    private final List<AlgebricksPipeline> prevSecondaryKeysPipeline;
+
+    public LSMSecondaryUpsertWithNestedPlanOperatorDescriptor(JobSpecification spec, RecordDescriptor outRecDesc,
+            int[] fieldPermutation, IIndexDataflowHelperFactory indexHelperFactory,
+            IModificationOperationCallbackFactory modCallbackFactory, int upsertIndicatorFieldIndex,
+            IBinaryBooleanInspectorFactory upsertIndicatorInspectorFactory,
+            List<AlgebricksPipeline> secondaryKeysPipeline, List<AlgebricksPipeline> prevSecondaryKeysPipeline) {
+        super(spec, outRecDesc, fieldPermutation, indexHelperFactory, null, modCallbackFactory,
+                upsertIndicatorFieldIndex, upsertIndicatorInspectorFactory, null);
+        this.secondaryKeysPipeline = secondaryKeysPipeline;
+        this.prevSecondaryKeysPipeline = prevSecondaryKeysPipeline;
+    }
+
+    @Override
+    public IOperatorNodePushable createPushRuntime(IHyracksTaskContext ctx,
+            IRecordDescriptorProvider recordDescProvider, int partition, int nPartitions) throws HyracksDataException {
+        RecordDescriptor inputRecDesc = recordDescProvider.getInputRecordDescriptor(getActivityId(), 0);
+        return new LSMSecondaryUpsertWithNestedPlanOperatorNodePushable(ctx, partition, indexHelperFactory,
+                modCallbackFactory, fieldPermutation, inputRecDesc, upsertIndicatorFieldIndex,
+                upsertIndicatorInspectorFactory, secondaryKeysPipeline, prevSecondaryKeysPipeline);
+    }
+}
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/operators/LSMSecondaryUpsertWithNestedPlanOperatorNodePushable.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/operators/LSMSecondaryUpsertWithNestedPlanOperatorNodePushable.java
new file mode 100644
index 0000000..f1af496
--- /dev/null
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/operators/LSMSecondaryUpsertWithNestedPlanOperatorNodePushable.java
@@ -0,0 +1,222 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.asterix.runtime.operators;
+
+import java.nio.ByteBuffer;
+import java.util.List;
+
+import org.apache.asterix.common.exceptions.ErrorCode;
+import org.apache.asterix.common.exceptions.RuntimeDataException;
+import org.apache.asterix.transaction.management.opcallbacks.AbstractIndexModificationOperationCallback;
+import org.apache.hyracks.algebricks.data.IBinaryBooleanInspectorFactory;
+import org.apache.hyracks.algebricks.runtime.base.AlgebricksPipeline;
+import org.apache.hyracks.algebricks.runtime.base.IPushRuntime;
+import org.apache.hyracks.algebricks.runtime.operators.meta.PipelineAssembler;
+import org.apache.hyracks.algebricks.runtime.operators.std.NestedTupleSourceRuntimeFactory.NestedTupleSourceRuntime;
+import org.apache.hyracks.api.comm.IFrameWriter;
+import org.apache.hyracks.api.context.IHyracksTaskContext;
+import org.apache.hyracks.api.dataflow.value.RecordDescriptor;
+import org.apache.hyracks.api.exceptions.HyracksDataException;
+import org.apache.hyracks.dataflow.common.comm.io.ArrayTupleBuilder;
+import org.apache.hyracks.dataflow.common.comm.io.ArrayTupleReference;
+import org.apache.hyracks.dataflow.common.comm.io.FrameTupleAccessor;
+import org.apache.hyracks.dataflow.common.comm.util.FrameUtils;
+import org.apache.hyracks.dataflow.common.data.accessors.FrameTupleReference;
+import org.apache.hyracks.storage.am.common.api.IModificationOperationCallbackFactory;
+import org.apache.hyracks.storage.am.common.dataflow.IIndexDataflowHelperFactory;
+import org.apache.hyracks.storage.am.lsm.common.api.ILSMIndexAccessor;
+
+public class LSMSecondaryUpsertWithNestedPlanOperatorNodePushable extends LSMSecondaryUpsertOperatorNodePushable {
+    private final NestedTupleSourceRuntime[] startOfNewKeyPipelines;
+    private final NestedTupleSourceRuntime[] startOfPrevKeyPipelines;
+    private final int numberOfPrimaryKeyAndFilterFields;
+
+    public LSMSecondaryUpsertWithNestedPlanOperatorNodePushable(IHyracksTaskContext ctx, int partition,
+            IIndexDataflowHelperFactory indexHelperFactory, IModificationOperationCallbackFactory modCallbackFactory,
+            int[] fieldPermutation, RecordDescriptor inputRecDesc, int upsertIndicatorFieldIndex,
+            IBinaryBooleanInspectorFactory upsertIndicatorInspectorFactory,
+            List<AlgebricksPipeline> secondaryKeysPipeline, List<AlgebricksPipeline> prevSecondaryKeysPipeline)
+            throws HyracksDataException {
+        super(ctx, partition, indexHelperFactory, modCallbackFactory, null, fieldPermutation, inputRecDesc,
+                upsertIndicatorFieldIndex, upsertIndicatorInspectorFactory, null);
+        this.numberOfPrimaryKeyAndFilterFields = fieldPermutation.length;
+        this.startOfNewKeyPipelines = buildStartOfPipelines(secondaryKeysPipeline, inputRecDesc, false);
+        this.startOfPrevKeyPipelines = buildStartOfPipelines(prevSecondaryKeysPipeline, inputRecDesc, true);
+    }
+
+    private NestedTupleSourceRuntime[] buildStartOfPipelines(List<AlgebricksPipeline> pipelines,
+            RecordDescriptor inputRecordDescriptor, boolean isPrev) throws HyracksDataException {
+        NestedTupleSourceRuntime[] resultant = new NestedTupleSourceRuntime[pipelines.size()];
+        PipelineAssembler[] pipelineAssemblers = new PipelineAssembler[pipelines.size()];
+        for (int p = 0; p < pipelines.size(); p++) {
+            AlgebricksPipeline pipeline = pipelines.get(p);
+            RecordDescriptor lastRecordDescriptorInPipeline =
+                    pipeline.getRecordDescriptors()[pipeline.getRecordDescriptors().length - 1];
+
+            IFrameWriter outputWriter;
+            if (p == 0) {
+                // Primary pipeline (the first). Here we perform the insert / delete.
+                outputWriter = new IndexTupleUnconditionalOperation(lastRecordDescriptorInPipeline, !isPrev);
+
+            } else {
+                IPushRuntime outputPushRuntime = PipelineAssembler.linkPipeline(pipeline, pipelineAssemblers, p);
+                if (outputPushRuntime == null) {
+                    throw new RuntimeDataException(ErrorCode.ILLEGAL_STATE, "No output runtime factories found.");
+                }
+                outputPushRuntime.setInputRecordDescriptor(0, lastRecordDescriptorInPipeline);
+                outputWriter = outputPushRuntime;
+            }
+
+            PipelineAssembler pipelineAssembler =
+                    new PipelineAssembler(pipeline, 1, 1, inputRecordDescriptor, lastRecordDescriptorInPipeline);
+            resultant[p] = (NestedTupleSourceRuntime) pipelineAssembler.assemblePipeline(outputWriter, ctx);
+            pipelineAssemblers[p] = pipelineAssembler;
+        }
+
+        return resultant;
+    }
+
+    @Override
+    public void open() throws HyracksDataException {
+        super.open();
+        frameTuple = new FrameTupleReference();
+        abstractModCallback = (AbstractIndexModificationOperationCallback) modCallback;
+    }
+
+    @Override
+    public void nextFrame(ByteBuffer buffer) throws HyracksDataException {
+        accessor.reset(buffer);
+        int tupleCount = accessor.getTupleCount();
+        for (int i = 0; i < tupleCount; i++) {
+            tuple.reset(accessor, i);
+
+            // Delete all of our old keys.
+            writeTupleToPipelineStarts(buffer, i, startOfPrevKeyPipelines);
+
+            // Insert all of our new keys, if the PIDX operation was also an UPSERT (and not just a DELETE).
+            frameTuple.reset(accessor, i);
+            if (upsertIndicatorInspector.getBooleanValue(frameTuple.getFieldData(upsertIndicatorFieldIndex),
+                    frameTuple.getFieldStart(upsertIndicatorFieldIndex),
+                    frameTuple.getFieldLength(upsertIndicatorFieldIndex))) {
+                writeTupleToPipelineStarts(buffer, i, startOfNewKeyPipelines);
+            }
+        }
+
+        // No partial flushing was necessary. Forward entire frame.
+        writeBuffer.ensureFrameSize(buffer.capacity());
+        FrameUtils.copyAndFlip(buffer, writeBuffer.getBuffer());
+        FrameUtils.flushFrame(writeBuffer.getBuffer(), writer);
+    }
+
+    private void writeTupleToPipelineStarts(ByteBuffer buffer, int tupleIndex,
+            NestedTupleSourceRuntime[] pipelineStarts) throws HyracksDataException {
+        for (NestedTupleSourceRuntime nts : pipelineStarts) {
+            nts.writeTuple(buffer, tupleIndex);
+        }
+
+        int n = 0;
+        try {
+            for (; n < pipelineStarts.length; n++) {
+                NestedTupleSourceRuntime nts = pipelineStarts[n];
+                try {
+                    nts.open();
+                } catch (Exception e) {
+                    nts.fail();
+                    throw e;
+                }
+            }
+        } finally {
+            for (int j = n - 1; j >= 0; j--) {
+                pipelineStarts[j].close();
+            }
+        }
+    }
+
+    private class IndexTupleUnconditionalOperation implements IFrameWriter {
+        private final RecordDescriptor inputRecordDescriptor;
+        private final boolean isInsert; // If this is not an insert, then our operation is delete.
+
+        private FrameTupleAccessor endOfPipelineTupleAccessor;
+        private FrameTupleReference endOfPipelineTupleReference;
+
+        // We are not writing the resulting tuple to a frame, we must store the result in an intermediate.
+        private ArrayTupleBuilder arrayTupleBuilder;
+        private ArrayTupleReference arrayTupleReference;
+
+        private IndexTupleUnconditionalOperation(RecordDescriptor recordDescriptor, boolean isInsert) {
+            this.inputRecordDescriptor = recordDescriptor;
+            this.isInsert = isInsert;
+        }
+
+        @Override
+        public void open() throws HyracksDataException {
+            int numSecondaryKeys = inputRecordDescriptor.getFieldCount();
+            endOfPipelineTupleAccessor = new FrameTupleAccessor(inputRecordDescriptor);
+            endOfPipelineTupleReference = new FrameTupleReference();
+            arrayTupleBuilder = new ArrayTupleBuilder(numberOfPrimaryKeyAndFilterFields + numSecondaryKeys);
+            arrayTupleReference = new ArrayTupleReference();
+        }
+
+        @Override
+        public void nextFrame(ByteBuffer buffer) throws HyracksDataException {
+            ILSMIndexAccessor workingLSMAccessor = (ILSMIndexAccessor) indexAccessor;
+
+            endOfPipelineTupleAccessor.reset(buffer);
+            int nTuple = endOfPipelineTupleAccessor.getTupleCount();
+            for (int t = 0; t < nTuple; t++) {
+
+                endOfPipelineTupleReference.reset(endOfPipelineTupleAccessor, t);
+                if (hasNullOrMissing(endOfPipelineTupleReference)) {
+                    // Do not perform operations w/ null or missing values (same behavior as atomic upserts).
+                    continue;
+                }
+
+                // First, add the secondary keys.
+                arrayTupleBuilder.reset();
+                int nFields = endOfPipelineTupleAccessor.getFieldCount();
+                for (int f = 0; f < nFields; f++) {
+                    arrayTupleBuilder.addField(endOfPipelineTupleAccessor, t, f);
+                }
+
+                // Next, add the primary keys and filter fields.
+                for (int f = 0; f < numberOfPrimaryKeyAndFilterFields; f++) {
+                    arrayTupleBuilder.addField(tuple.getFieldData(f), tuple.getFieldStart(f), tuple.getFieldLength(f));
+                }
+
+                // Finally, pass the tuple to our accessor. There are only two operations: insert or delete.
+                arrayTupleReference.reset(arrayTupleBuilder.getFieldEndOffsets(), arrayTupleBuilder.getByteArray());
+                if (this.isInsert) {
+                    abstractModCallback.setOp(AbstractIndexModificationOperationCallback.Operation.INSERT);
+                    workingLSMAccessor.forceInsert(arrayTupleReference);
+                } else {
+                    abstractModCallback.setOp(AbstractIndexModificationOperationCallback.Operation.DELETE);
+                    workingLSMAccessor.forceDelete(arrayTupleReference);
+                }
+            }
+        }
+
+        @Override
+        public void fail() throws HyracksDataException {
+        }
+
+        @Override
+        public void close() throws HyracksDataException {
+        }
+    }
+}
diff --git a/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/metadata/IMetadataProvider.java b/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/metadata/IMetadataProvider.java
index 717eac7..e505960 100644
--- a/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/metadata/IMetadataProvider.java
+++ b/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/metadata/IMetadataProvider.java
@@ -32,6 +32,7 @@
 import org.apache.hyracks.algebricks.core.algebra.operators.logical.IOperatorSchema;
 import org.apache.hyracks.algebricks.core.jobgen.impl.JobGenContext;
 import org.apache.hyracks.algebricks.data.IPrinterFactory;
+import org.apache.hyracks.algebricks.runtime.base.AlgebricksPipeline;
 import org.apache.hyracks.algebricks.runtime.base.IPushRuntimeFactory;
 import org.apache.hyracks.api.dataflow.IOperatorDescriptor;
 import org.apache.hyracks.api.dataflow.value.RecordDescriptor;
@@ -108,6 +109,10 @@
      *            Job generation context.
      * @param spec
      *            Target job specification.
+     * @param secondaryKeysPipelines
+     *            Nested plans to extract secondary keys.
+     * @param pipelineTopSchema
+     *            Schema of the primary pipeline for secondary keys.
      * @return
      *         A Hyracks IOperatorDescriptor and its partition constraint.
      * @throws AlgebricksException
@@ -116,7 +121,9 @@
             IDataSourceIndex<I, S> dataSource, IOperatorSchema propagatedSchema, IOperatorSchema[] inputSchemas,
             IVariableTypeEnvironment typeEnv, List<LogicalVariable> primaryKeys, List<LogicalVariable> secondaryKeys,
             List<LogicalVariable> additionalNonKeyFields, ILogicalExpression filterExpr, RecordDescriptor recordDesc,
-            JobGenContext context, JobSpecification spec, boolean bulkload) throws AlgebricksException;
+            JobGenContext context, JobSpecification spec, boolean bulkload,
+            List<List<AlgebricksPipeline>> secondaryKeysPipelines, IOperatorSchema pipelineTopSchema)
+            throws AlgebricksException;
 
     /**
      * Creates the delete runtime of IndexInsertDeletePOperator, which models
@@ -147,6 +154,10 @@
      *            Job generation context.
      * @param spec
      *            Target job specification.
+     * @param secondaryKeysPipelines
+     *            Nested plan to extract secondary keys.
+     * @param pipelineTopSchema
+     *            Schema of the primary pipeline for secondary keys.
      * @return
      *         A Hyracks IOperatorDescriptor and its partition constraint.
      * @throws AlgebricksException
@@ -155,7 +166,8 @@
             IDataSourceIndex<I, S> dataSource, IOperatorSchema propagatedSchema, IOperatorSchema[] inputSchemas,
             IVariableTypeEnvironment typeEnv, List<LogicalVariable> primaryKeys, List<LogicalVariable> secondaryKeys,
             List<LogicalVariable> additionalNonKeyFields, ILogicalExpression filterExpr, RecordDescriptor recordDesc,
-            JobGenContext context, JobSpecification spec) throws AlgebricksException;
+            JobGenContext context, JobSpecification spec, List<List<AlgebricksPipeline>> secondaryKeysPipelines,
+            IOperatorSchema pipelineTopSchema) throws AlgebricksException;
 
     /**
      * Creates the TokenizeOperator for IndexInsertDeletePOperator, which tokenizes
@@ -209,7 +221,7 @@
             List<LogicalVariable> additionalFilteringKeys, ILogicalExpression filterExpr,
             LogicalVariable upsertIndicatorVar, List<LogicalVariable> prevSecondaryKeys,
             LogicalVariable prevAdditionalFilteringKeys, RecordDescriptor inputDesc, JobGenContext context,
-            JobSpecification spec) throws AlgebricksException;
+            JobSpecification spec, List<List<AlgebricksPipeline>> secondaryKeysPipelines) throws AlgebricksException;
 
     public ITupleFilterFactory createTupleFilterFactory(IOperatorSchema[] inputSchemas,
             IVariableTypeEnvironment typeEnv, ILogicalExpression filterExpr, JobGenContext context)
diff --git a/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/logical/IndexInsertDeleteUpsertOperator.java b/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/logical/IndexInsertDeleteUpsertOperator.java
index c84db0e..7b6ed26 100644
--- a/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/logical/IndexInsertDeleteUpsertOperator.java
+++ b/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/logical/IndexInsertDeleteUpsertOperator.java
@@ -19,6 +19,7 @@
 package org.apache.hyracks.algebricks.core.algebra.operators.logical;
 
 import java.util.ArrayList;
+import java.util.Collection;
 import java.util.List;
 
 import org.apache.commons.lang3.mutable.Mutable;
@@ -34,15 +35,49 @@
 import org.apache.hyracks.algebricks.core.algebra.visitors.ILogicalExpressionReferenceTransform;
 import org.apache.hyracks.algebricks.core.algebra.visitors.ILogicalOperatorVisitor;
 
-public class IndexInsertDeleteUpsertOperator extends AbstractLogicalOperator {
+/**
+ * Logical operator for handling secondary index maintenance / loading.
+ * <p>
+ *
+ * In both cases (whether the index is on an atomic field or an array field):
+ * <p>
+ * Primary keys will be given in {@link #primaryKeyExprs}. {@link #operation} specifies the type of index maintenance to
+ * perform. In the case of bulk-loading, {@link #operation} will be INSERT and the {@link #bulkload} flag will be
+ * raised. {@link #additionalFilteringExpressions} and {@link #numberOfAdditionalNonFilteringFields} refers to the
+ * additionalFilteringExpressions, numberOfAdditionalNonFilteringFields found in the corresponding primary index
+ * {@link InsertDeleteUpsertOperator} (i.e. to specify LSM filters). {@link #upsertIndicatorExpr} also originates from
+ * {@link InsertDeleteUpsertOperator}, and is only set when the operation is of kind UPSERT.
+ * <p>
+ *
+ * If the SIDX is on an atomic field <b>or</b> on an array field w/ a bulk-load operation:
+ * <p>
+ * We specify secondary key information in {@link #secondaryKeyExprs}. If we may encounter nullable keys, then we
+ * specify a {@link #filterExpr} to be evaluated inside the runtime. If the operation is of kind UPSERT, then we must
+ * also specify previous secondary key information in {@link #prevSecondaryKeyExprs}. If
+ * {@link #additionalFilteringExpressions} has been set, then {@link #prevAdditionalFilteringExpression} should also be
+ * set.
+ *
+ * <p>
+ * If the SIDX is on an array field <b>and</b> we are not performing a bulk-load operation:
+ * <p>
+ * We <b>do not</b> specify secondary key information in {@link #secondaryKeyExprs} (this is null). Instead, we specify
+ * how to extract secondary keys using {@link #nestedPlans}. If we may encounter nullable keys, then we <b>do not</b>
+ * specify a {@link #filterExpr} (this is null). Instead, this filter must be attached to the top of the nested plan
+ * itself. If the operation is not of type UPSERT, then we must only have one nested plan. Otherwise, the second nested
+ * plan must specify how to extract secondary keys from the previous record. {@link #prevSecondaryKeyExprs} and
+ * {@link #prevAdditionalFilteringExpression} will always be null here, even if the operation is UPSERT.
+ *
+ */
+public class IndexInsertDeleteUpsertOperator extends AbstractOperatorWithNestedPlans {
 
     private final IDataSourceIndex<?, ?> dataSourceIndex;
     private final List<Mutable<ILogicalExpression>> primaryKeyExprs;
     // In the bulk-load case on ngram or keyword index,
     // it contains [token, number of token] or [token].
+    // In the non bulk-load array-index case, it contains nothing.
     // Otherwise, it contains secondary key information.
-    private final List<Mutable<ILogicalExpression>> secondaryKeyExprs;
-    private final Mutable<ILogicalExpression> filterExpr;
+    private List<Mutable<ILogicalExpression>> secondaryKeyExprs;
+    private Mutable<ILogicalExpression> filterExpr;
     private final Kind operation;
     private final boolean bulkload;
     private List<Mutable<ILogicalExpression>> additionalFilteringExpressions;
@@ -66,12 +101,6 @@
     }
 
     @Override
-    public void recomputeSchema() throws AlgebricksException {
-        schema = new ArrayList<LogicalVariable>();
-        schema.addAll(inputs.get(0).getValue().getSchema());
-    }
-
-    @Override
     public boolean acceptExpressionTransform(ILogicalExpressionReferenceTransform visitor) throws AlgebricksException {
         boolean b = false;
         // Primary
@@ -125,8 +154,37 @@
     }
 
     @Override
-    public boolean isMap() {
-        return false;
+    public void getUsedVariablesExceptNestedPlans(Collection<LogicalVariable> vars) {
+        for (Mutable<ILogicalExpression> e : getPrimaryKeyExpressions()) {
+            e.getValue().getUsedVariables(vars);
+        }
+        for (Mutable<ILogicalExpression> e : getSecondaryKeyExpressions()) {
+            e.getValue().getUsedVariables(vars);
+        }
+        if (getFilterExpression() != null) {
+            getFilterExpression().getValue().getUsedVariables(vars);
+        }
+        if (getAdditionalFilteringExpressions() != null) {
+            for (Mutable<ILogicalExpression> e : getAdditionalFilteringExpressions()) {
+                e.getValue().getUsedVariables(vars);
+            }
+        }
+        if (getPrevAdditionalFilteringExpression() != null) {
+            getPrevAdditionalFilteringExpression().getValue().getUsedVariables(vars);
+        }
+        if (getPrevSecondaryKeyExprs() != null) {
+            for (Mutable<ILogicalExpression> e : getPrevSecondaryKeyExprs()) {
+                e.getValue().getUsedVariables(vars);
+            }
+        }
+        if (getUpsertIndicatorExpr() != null) {
+            getUpsertIndicatorExpr().getValue().getUsedVariables(vars);
+        }
+    }
+
+    @Override
+    public void getProducedVariablesExceptNestedPlans(Collection<LogicalVariable> vars) {
+        // Do nothing (no variables are produced here).
     }
 
     @Override
@@ -144,6 +202,12 @@
         return createPropagatingAllInputsTypeEnvironment(ctx);
     }
 
+    @Override
+    public void recomputeSchema() {
+        schema = new ArrayList<>();
+        schema.addAll(inputs.get(0).getValue().getSchema());
+    }
+
     public List<Mutable<ILogicalExpression>> getPrimaryKeyExpressions() {
         return primaryKeyExprs;
     }
@@ -160,10 +224,18 @@
         return secondaryKeyExprs;
     }
 
+    public void setSecondaryKeyExprs(List<Mutable<ILogicalExpression>> secondaryKeyExprs) {
+        this.secondaryKeyExprs = secondaryKeyExprs;
+    }
+
     public Mutable<ILogicalExpression> getFilterExpression() {
         return filterExpr;
     }
 
+    public void setFilterExpression(Mutable<ILogicalExpression> filterExpr) {
+        this.filterExpr = filterExpr;
+    }
+
     public Kind getOperation() {
         return operation;
     }
diff --git a/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/logical/visitors/FDsAndEquivClassesVisitor.java b/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/logical/visitors/FDsAndEquivClassesVisitor.java
index ccdbd1f..f3717c8 100644
--- a/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/logical/visitors/FDsAndEquivClassesVisitor.java
+++ b/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/logical/visitors/FDsAndEquivClassesVisitor.java
@@ -556,6 +556,17 @@
     public Void visitIndexInsertDeleteUpsertOperator(IndexInsertDeleteUpsertOperator op, IOptimizationContext ctx)
             throws AlgebricksException {
         propagateFDsAndEquivClasses(op, ctx);
+        if (!op.getNestedPlans().isEmpty()) {
+            Map<LogicalVariable, EquivalenceClass> equivalenceClasses = ctx.getEquivalenceClassMap(op);
+            List<FunctionalDependency> functionalDependencies = ctx.getFDList(op);
+            for (ILogicalPlan p : op.getNestedPlans()) {
+                for (Mutable<ILogicalOperator> r : p.getRoots()) {
+                    ILogicalOperator op2 = r.getValue();
+                    equivalenceClasses.putAll(getOrComputeEqClasses(op2, ctx));
+                    functionalDependencies.addAll(getOrComputeFDs(op2, ctx));
+                }
+            }
+        }
         return null;
     }
 
diff --git a/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/logical/visitors/IsomorphismOperatorVisitor.java b/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/logical/visitors/IsomorphismOperatorVisitor.java
index 09d0c14..ff60a90 100644
--- a/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/logical/visitors/IsomorphismOperatorVisitor.java
+++ b/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/logical/visitors/IsomorphismOperatorVisitor.java
@@ -602,10 +602,12 @@
         }
         IndexInsertDeleteUpsertOperator insertOpArg = (IndexInsertDeleteUpsertOperator) copyAndSubstituteVar(op, arg);
         boolean isomorphic = VariableUtilities.varListEqualUnordered(op.getSchema(), insertOpArg.getSchema());
-        if (!op.getDataSourceIndex().equals(insertOpArg.getDataSourceIndex())) {
-            isomorphic = false;
+        if (!isomorphic || !op.getDataSourceIndex().equals(insertOpArg.getDataSourceIndex())) {
+            return Boolean.FALSE;
         }
-        return isomorphic;
+
+        // Check our nested plans as well.
+        return (!compareSubplans(op.getNestedPlans(), insertOpArg.getNestedPlans())) ? Boolean.TRUE : Boolean.FALSE;
     }
 
     @Override
diff --git a/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/logical/visitors/IsomorphismVariableMappingVisitor.java b/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/logical/visitors/IsomorphismVariableMappingVisitor.java
index fe794c8..153ac04 100644
--- a/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/logical/visitors/IsomorphismVariableMappingVisitor.java
+++ b/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/logical/visitors/IsomorphismVariableMappingVisitor.java
@@ -297,7 +297,8 @@
     @Override
     public Void visitIndexInsertDeleteUpsertOperator(IndexInsertDeleteUpsertOperator op, ILogicalOperator arg)
             throws AlgebricksException {
-        mapVariablesStandard(op, arg);
+        mapChildren(op, arg);
+        mapVariablesInNestedPlans(op, arg);
         return null;
     }
 
diff --git a/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/logical/visitors/OperatorDeepCopyVisitor.java b/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/logical/visitors/OperatorDeepCopyVisitor.java
index 46999b0..c06ad5c 100644
--- a/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/logical/visitors/OperatorDeepCopyVisitor.java
+++ b/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/logical/visitors/OperatorDeepCopyVisitor.java
@@ -330,6 +330,9 @@
                 op.getDataSourceIndex(), newPrimaryKeyExpressions, newSecondaryKeyExpressions, newFilterExpression,
                 op.getOperation(), op.isBulkload(), op.getNumberOfAdditionalNonFilteringFields());
         indexInsertDeleteOp.setAdditionalFilteringExpressions(newLSMComponentFilterExpressions);
+        for (ILogicalPlan plan : op.getNestedPlans()) {
+            indexInsertDeleteOp.getNestedPlans().add(OperatorManipulationUtil.deepCopy(plan, indexInsertDeleteOp));
+        }
         return indexInsertDeleteOp;
     }
 
diff --git a/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/logical/visitors/SchemaVariableVisitor.java b/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/logical/visitors/SchemaVariableVisitor.java
index bd90729..44bb7e2 100644
--- a/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/logical/visitors/SchemaVariableVisitor.java
+++ b/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/logical/visitors/SchemaVariableVisitor.java
@@ -309,6 +309,7 @@
     @Override
     public Void visitIndexInsertDeleteUpsertOperator(IndexInsertDeleteUpsertOperator op, Void arg)
             throws AlgebricksException {
+        // Variables in our nested plan are not propagated.
         standardLayout(op);
         return null;
     }
diff --git a/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/logical/visitors/SubstituteVariableVisitor.java b/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/logical/visitors/SubstituteVariableVisitor.java
index a2107e5..6e8b425 100644
--- a/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/logical/visitors/SubstituteVariableVisitor.java
+++ b/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/logical/visitors/SubstituteVariableVisitor.java
@@ -456,6 +456,9 @@
         substUsedVariablesInExpr(op.getUpsertIndicatorExpr(), pair.first, pair.second);
         substUsedVariablesInExpr(op.getPrevSecondaryKeyExprs(), pair.first, pair.second);
         substUsedVariablesInExpr(op.getPrevAdditionalFilteringExpression(), pair.first, pair.second);
+        if (!op.getNestedPlans().isEmpty()) {
+            substInNestedPlans(op, pair.first, pair.second);
+        }
         return null;
     }
 
diff --git a/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/logical/visitors/UsedVariableVisitor.java b/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/logical/visitors/UsedVariableVisitor.java
index 65e9023..23fe3b2 100644
--- a/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/logical/visitors/UsedVariableVisitor.java
+++ b/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/logical/visitors/UsedVariableVisitor.java
@@ -405,7 +405,8 @@
     }
 
     @Override
-    public Void visitIndexInsertDeleteUpsertOperator(IndexInsertDeleteUpsertOperator op, Void arg) {
+    public Void visitIndexInsertDeleteUpsertOperator(IndexInsertDeleteUpsertOperator op, Void arg)
+            throws AlgebricksException {
         for (Mutable<ILogicalExpression> e : op.getPrimaryKeyExpressions()) {
             e.getValue().getUsedVariables(usedVariables);
         }
@@ -431,6 +432,7 @@
         if (op.getUpsertIndicatorExpr() != null) {
             op.getUpsertIndicatorExpr().getValue().getUsedVariables(usedVariables);
         }
+        visitNestedPlans(op);
         return null;
     }
 
diff --git a/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/physical/AbstractPhysicalOperator.java b/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/physical/AbstractPhysicalOperator.java
index 438d9ee..4bc7502 100644
--- a/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/physical/AbstractPhysicalOperator.java
+++ b/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/physical/AbstractPhysicalOperator.java
@@ -137,7 +137,7 @@
     protected AlgebricksPipeline[] compileSubplans(IOperatorSchema outerPlanSchema,
             AbstractOperatorWithNestedPlans npOp, IOperatorSchema opSchema, JobGenContext context)
             throws AlgebricksException {
-        List<List<AlgebricksPipeline>> subplans = compileSubplansImpl(outerPlanSchema, npOp, opSchema, context);
+        List<List<AlgebricksPipeline>> subplans = compileSubplansImpl(outerPlanSchema, npOp, opSchema, context, true);
         int n = subplans.size();
         AlgebricksPipeline[] result = new AlgebricksPipeline[n];
         for (int i = 0; i < n; i++) {
@@ -153,26 +153,28 @@
     }
 
     protected List<List<AlgebricksPipeline>> compileSubplansImpl(IOperatorSchema outerPlanSchema,
-            AbstractOperatorWithNestedPlans npOp, IOperatorSchema opSchema, JobGenContext context)
-            throws AlgebricksException {
+            AbstractOperatorWithNestedPlans npOp, IOperatorSchema opSchema, JobGenContext context,
+            boolean withProjection) throws AlgebricksException {
         List<List<AlgebricksPipeline>> subplans = new ArrayList<>(npOp.getNestedPlans().size());
         PlanCompiler pc = new PlanCompiler(context);
         for (ILogicalPlan p : npOp.getNestedPlans()) {
-            subplans.add(buildPipelineWithProjection(p, outerPlanSchema, opSchema, pc));
+            subplans.add(buildPipeline(p, outerPlanSchema, opSchema, pc, withProjection));
         }
         return subplans;
     }
 
-    private List<AlgebricksPipeline> buildPipelineWithProjection(ILogicalPlan p, IOperatorSchema outerPlanSchema,
-            IOperatorSchema opSchema, PlanCompiler pc) throws AlgebricksException {
+    private List<AlgebricksPipeline> buildPipeline(ILogicalPlan p, IOperatorSchema outerPlanSchema,
+            IOperatorSchema opSchema, PlanCompiler pc, boolean withProjection) throws AlgebricksException {
         if (p.getRoots().size() > 1) {
             throw new NotImplementedException("Nested plans with several roots are not supported.");
         }
         JobSpecification nestedJob = pc.compileNestedPlan(p, outerPlanSchema);
-        ILogicalOperator topOpInSubplan = p.getRoots().get(0).getValue();
-        JobGenContext context = pc.getContext();
-        IOperatorSchema topOpInSubplanScm = context.getSchema(topOpInSubplan);
-        opSchema.addAllVariables(topOpInSubplanScm);
+        if (withProjection) {
+            ILogicalOperator topOpInSubplan = p.getRoots().get(0).getValue();
+            JobGenContext context = pc.getContext();
+            IOperatorSchema topOpInSubplanScm = context.getSchema(topOpInSubplan);
+            opSchema.addAllVariables(topOpInSubplanScm);
+        }
 
         Map<OperatorDescriptorId, IOperatorDescriptor> opMap = nestedJob.getOperatorMap();
         List<? extends IOperatorDescriptor> metaOps = nestedJob.getMetaOps();
diff --git a/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/physical/IndexBulkloadPOperator.java b/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/physical/IndexBulkloadPOperator.java
index 6512700..7d70117 100644
--- a/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/physical/IndexBulkloadPOperator.java
+++ b/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/physical/IndexBulkloadPOperator.java
@@ -134,7 +134,7 @@
                 context.getTypeEnvironment(op.getInputs().get(0).getValue()), inputSchemas[0], context);
         Pair<IOperatorDescriptor, AlgebricksPartitionConstraint> runtimeAndConstraints =
                 mp.getIndexInsertRuntime(dataSourceIndex, propagatedSchema, inputSchemas, typeEnv, primaryKeys,
-                        secondaryKeys, additionalFilteringKeys, filterExpr, inputDesc, context, spec, true);
+                        secondaryKeys, additionalFilteringKeys, filterExpr, inputDesc, context, spec, true, null, null);
         IOperatorDescriptor opDesc = runtimeAndConstraints.first;
         opDesc.setSourceLocation(indexInsertDeleteOp.getSourceLocation());
         builder.contributeHyracksOperator(indexInsertDeleteOp, opDesc);
diff --git a/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/physical/IndexInsertDeleteUpsertPOperator.java b/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/physical/IndexInsertDeleteUpsertPOperator.java
index 228ca52..2d9dabe 100644
--- a/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/physical/IndexInsertDeleteUpsertPOperator.java
+++ b/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/physical/IndexInsertDeleteUpsertPOperator.java
@@ -43,8 +43,11 @@
 import org.apache.hyracks.algebricks.core.algebra.properties.PhysicalRequirements;
 import org.apache.hyracks.algebricks.core.jobgen.impl.JobGenContext;
 import org.apache.hyracks.algebricks.core.jobgen.impl.JobGenHelper;
+import org.apache.hyracks.algebricks.core.jobgen.impl.PlanCompiler;
+import org.apache.hyracks.algebricks.runtime.base.AlgebricksPipeline;
 import org.apache.hyracks.api.dataflow.IOperatorDescriptor;
 import org.apache.hyracks.api.dataflow.value.RecordDescriptor;
+import org.apache.hyracks.api.exceptions.ErrorCode;
 import org.apache.hyracks.api.job.JobSpecification;
 
 public class IndexInsertDeleteUpsertPOperator extends AbstractPhysicalOperator {
@@ -113,8 +116,27 @@
             IOperatorSchema propagatedSchema, IOperatorSchema[] inputSchemas, IOperatorSchema outerPlanSchema)
             throws AlgebricksException {
         IndexInsertDeleteUpsertOperator insertDeleteUpsertOp = (IndexInsertDeleteUpsertOperator) op;
-        IMetadataProvider mp = context.getMetadataProvider();
 
+        // Compile our nested plans if any exist.
+        int numberOfNestedPlans = insertDeleteUpsertOp.getNestedPlans().size();
+        List<List<AlgebricksPipeline>> secondaryKeyPipelines = null;
+        IOperatorSchema pipelineTopSchema = null;
+        if ((numberOfNestedPlans > 1 && !insertDeleteUpsertOp.getOperation().equals(Kind.UPSERT))
+                || numberOfNestedPlans > 2) {
+            throw AlgebricksException.create(ErrorCode.ILLEGAL_STATE, op.getSourceLocation(),
+                    "Using multiple nested plans to extract secondary keys is not supported.");
+        }
+        if (numberOfNestedPlans > 0) {
+            secondaryKeyPipelines =
+                    compileSubplansImpl(inputSchemas[0], insertDeleteUpsertOp, propagatedSchema, context, false);
+
+            // Get the schema of our nested plan root.
+            ILogicalOperator topOpInSubplan = insertDeleteUpsertOp.getNestedPlans().get(0).getRoots().get(0).getValue();
+            PlanCompiler pc = new PlanCompiler(context);
+            pipelineTopSchema = pc.getContext().getSchema(topOpInSubplan);
+        }
+
+        IMetadataProvider mp = context.getMetadataProvider();
         JobSpecification spec = builder.getJobSpec();
         RecordDescriptor inputDesc = JobGenHelper.mkRecordDescriptor(
                 context.getTypeEnvironment(op.getInputs().get(0).getValue()), inputSchemas[0], context);
@@ -124,19 +146,19 @@
         Kind operation = insertDeleteUpsertOp.getOperation();
         switch (operation) {
             case INSERT:
-                runtimeAndConstraints =
-                        mp.getIndexInsertRuntime(dataSourceIndex, propagatedSchema, inputSchemas, typeEnv, primaryKeys,
-                                secondaryKeys, additionalFilteringKeys, filterExpr, inputDesc, context, spec, false);
+                runtimeAndConstraints = mp.getIndexInsertRuntime(dataSourceIndex, propagatedSchema, inputSchemas,
+                        typeEnv, primaryKeys, secondaryKeys, additionalFilteringKeys, filterExpr, inputDesc, context,
+                        spec, false, secondaryKeyPipelines, pipelineTopSchema);
                 break;
             case DELETE:
-                runtimeAndConstraints =
-                        mp.getIndexDeleteRuntime(dataSourceIndex, propagatedSchema, inputSchemas, typeEnv, primaryKeys,
-                                secondaryKeys, additionalFilteringKeys, filterExpr, inputDesc, context, spec);
+                runtimeAndConstraints = mp.getIndexDeleteRuntime(dataSourceIndex, propagatedSchema, inputSchemas,
+                        typeEnv, primaryKeys, secondaryKeys, additionalFilteringKeys, filterExpr, inputDesc, context,
+                        spec, secondaryKeyPipelines, pipelineTopSchema);
                 break;
             case UPSERT:
                 runtimeAndConstraints = mp.getIndexUpsertRuntime(dataSourceIndex, propagatedSchema, inputSchemas,
                         typeEnv, primaryKeys, secondaryKeys, additionalFilteringKeys, filterExpr, upsertIndicatorVar,
-                        prevSecondaryKeys, prevAdditionalFilteringKey, inputDesc, context, spec);
+                        prevSecondaryKeys, prevAdditionalFilteringKey, inputDesc, context, spec, secondaryKeyPipelines);
                 break;
             default:
                 throw new AlgebricksException("Unsupported Operation " + operation);
diff --git a/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/physical/SubplanPOperator.java b/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/physical/SubplanPOperator.java
index 5ec6d0a..898fbe8 100644
--- a/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/physical/SubplanPOperator.java
+++ b/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/physical/SubplanPOperator.java
@@ -89,7 +89,8 @@
         if (subplan.getNestedPlans().size() != 1) {
             throw new NotImplementedException("Subplan currently works only for one nested plan with one root.");
         }
-        List<List<AlgebricksPipeline>> subplans = compileSubplansImpl(inputSchemas[0], subplan, opSchema, context);
+        List<List<AlgebricksPipeline>> subplans =
+                compileSubplansImpl(inputSchemas[0], subplan, opSchema, context, true);
         assert subplans.size() == 1;
         List<AlgebricksPipeline> np = subplans.get(0);
         RecordDescriptor inputRecordDesc = JobGenHelper.mkRecordDescriptor(
diff --git a/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/prettyprint/LogicalOperatorPrettyPrintVisitor.java b/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/prettyprint/LogicalOperatorPrettyPrintVisitor.java
index f6e4742..f17fa70 100644
--- a/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/prettyprint/LogicalOperatorPrettyPrintVisitor.java
+++ b/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/prettyprint/LogicalOperatorPrettyPrintVisitor.java
@@ -461,7 +461,10 @@
             buffer.append(" replace:");
             pprintExprList(op.getPrevSecondaryKeyExprs(), indent);
             buffer.append(" with:");
-            pprintExprList(op.getSecondaryKeyExpressions(), indent);
+        }
+        if (!op.getNestedPlans().isEmpty()) {
+            buffer.append("{");
+            printNestedPlans(op, indent);
         } else {
             pprintExprList(op.getSecondaryKeyExpressions(), indent);
         }
diff --git a/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/prettyprint/LogicalOperatorPrettyPrintVisitorJson.java b/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/prettyprint/LogicalOperatorPrettyPrintVisitorJson.java
index f404058..a243996 100644
--- a/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/prettyprint/LogicalOperatorPrettyPrintVisitorJson.java
+++ b/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/prettyprint/LogicalOperatorPrettyPrintVisitorJson.java
@@ -620,9 +620,15 @@
             jsonGenerator.writeObjectFieldStart("from");
             if (op.getOperation() == Kind.UPSERT) {
                 writeArrayFieldOfExpressions("replace", op.getPrevSecondaryKeyExprs(), indent);
-                writeArrayFieldOfExpressions("with", op.getSecondaryKeyExpressions(), indent);
-            } else {
+                if (op.getNestedPlans().isEmpty()) {
+                    writeArrayFieldOfExpressions("with", op.getSecondaryKeyExpressions(), indent);
+                } else {
+                    writeNestedPlans(op, indent);
+                }
+            } else if (op.getNestedPlans().isEmpty()) {
                 writeArrayFieldOfExpressions(EXPRESSIONS_FIELD, op.getSecondaryKeyExpressions(), indent);
+            } else {
+                writeNestedPlans(op, indent);
             }
             jsonGenerator.writeEndObject();
             if (op.isBulkload()) {
diff --git a/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/config/AlgebricksConfig.java b/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/config/AlgebricksConfig.java
index c99117b..944ddf7 100644
--- a/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/config/AlgebricksConfig.java
+++ b/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/config/AlgebricksConfig.java
@@ -33,4 +33,5 @@
     public static final boolean SUBPLAN_MERGE_DEFAULT = true;
     public static final boolean SUBPLAN_NESTEDPUSHDOWN_DEFAULT = true;
     public static final boolean MIN_MEMORY_ALLOCATION_DEFAULT = true;
+    public static final boolean ARRAY_INDEX_DEFAULT = false;
 }
diff --git a/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/rewriter/base/PhysicalOptimizationConfig.java b/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/rewriter/base/PhysicalOptimizationConfig.java
index 92f579c..f18cad2 100644
--- a/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/rewriter/base/PhysicalOptimizationConfig.java
+++ b/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/rewriter/base/PhysicalOptimizationConfig.java
@@ -45,6 +45,7 @@
     private static final String SUBPLAN_MERGE = "SUBPLAN_MERGE";
     private static final String SUBPLAN_NESTEDPUSHDOWN = "SUBPLAN_NESTEDPUSHDOWN";
     private static final String MIN_MEMORY_ALLOCATION = "MIN_MEMORY_ALLOCATION";
+    private static final String ARRAY_INDEX = "ARRAY_INDEX";
 
     private final Properties properties = new Properties();
 
@@ -226,6 +227,14 @@
         setBoolean(MIN_MEMORY_ALLOCATION, value);
     }
 
+    public boolean isArrayIndexEnabled() {
+        return getBoolean(ARRAY_INDEX, AlgebricksConfig.ARRAY_INDEX_DEFAULT);
+    }
+
+    public void setArrayIndexEnabled(boolean arrayIndex) {
+        setBoolean(ARRAY_INDEX, arrayIndex);
+    }
+
     private void setInt(String property, int value) {
         properties.setProperty(property, Integer.toString(value));
     }
diff --git a/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/utils/LogicalOperatorDotVisitor.java b/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/utils/LogicalOperatorDotVisitor.java
index 67963ce..01f7403 100644
--- a/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/utils/LogicalOperatorDotVisitor.java
+++ b/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/utils/LogicalOperatorDotVisitor.java
@@ -529,6 +529,10 @@
             printExprList(op.getPrevSecondaryKeyExprs());
             stringBuilder.append(" with:");
             printExprList(op.getSecondaryKeyExpressions());
+        }
+        if (!op.getNestedPlans().isEmpty()) {
+            // TODO: Finish this for nested plans.
+            stringBuilder.append("{ a nested plan }");
         } else {
             printExprList(op.getSecondaryKeyExpressions());
         }
diff --git a/hyracks-fullstack/algebricks/algebricks-runtime/src/main/java/org/apache/hyracks/algebricks/runtime/operators/meta/PipelineAssembler.java b/hyracks-fullstack/algebricks/algebricks-runtime/src/main/java/org/apache/hyracks/algebricks/runtime/operators/meta/PipelineAssembler.java
index 81f5d08..858fcfa 100644
--- a/hyracks-fullstack/algebricks/algebricks-runtime/src/main/java/org/apache/hyracks/algebricks/runtime/operators/meta/PipelineAssembler.java
+++ b/hyracks-fullstack/algebricks/algebricks-runtime/src/main/java/org/apache/hyracks/algebricks/runtime/operators/meta/PipelineAssembler.java
@@ -29,6 +29,7 @@
 import org.apache.hyracks.api.context.IHyracksTaskContext;
 import org.apache.hyracks.api.dataflow.EnforceFrameWriter;
 import org.apache.hyracks.api.dataflow.value.RecordDescriptor;
+import org.apache.hyracks.api.exceptions.ErrorCode;
 import org.apache.hyracks.api.exceptions.HyracksDataException;
 import org.apache.hyracks.api.job.JobFlag;
 
@@ -123,4 +124,21 @@
         }
         return start;
     }
+
+    public static IPushRuntime linkPipeline(AlgebricksPipeline pipeline, PipelineAssembler[] pipelineAssemblers,
+            int pipelineAssemblersCount) throws HyracksDataException {
+        IPushRuntimeFactory[] outputRuntimeFactories = pipeline.getOutputRuntimeFactories();
+        if (outputRuntimeFactories == null || outputRuntimeFactories.length != 1) {
+            throw HyracksDataException.create(ErrorCode.ILLEGAL_STATE, "No output runtime factories found.");
+        }
+        IPushRuntimeFactory outRuntimeFactory = outputRuntimeFactories[0];
+        int outputPosition = pipeline.getOutputPositions()[0];
+        for (int i = 0; i < pipelineAssemblersCount; i++) {
+            IPushRuntime[] p = pipelineAssemblers[i].getPushRuntime(outRuntimeFactory);
+            if (p != null) {
+                return p[outputPosition];
+            }
+        }
+        return null;
+    }
 }
diff --git a/hyracks-fullstack/algebricks/algebricks-runtime/src/main/java/org/apache/hyracks/algebricks/runtime/operators/meta/SubplanRuntimeFactory.java b/hyracks-fullstack/algebricks/algebricks-runtime/src/main/java/org/apache/hyracks/algebricks/runtime/operators/meta/SubplanRuntimeFactory.java
index 6e5c2c9..48d7fd2 100644
--- a/hyracks-fullstack/algebricks/algebricks-runtime/src/main/java/org/apache/hyracks/algebricks/runtime/operators/meta/SubplanRuntimeFactory.java
+++ b/hyracks-fullstack/algebricks/algebricks-runtime/src/main/java/org/apache/hyracks/algebricks/runtime/operators/meta/SubplanRuntimeFactory.java
@@ -117,7 +117,7 @@
                     outputRecordDescriptor = SubplanRuntimeFactory.this.outputRecordDesc;
                 } else {
                     // secondary pipeline
-                    IPushRuntime outputPushRuntime = linkSecondaryPipeline(pipeline, pipelineAssemblers, i);
+                    IPushRuntime outputPushRuntime = PipelineAssembler.linkPipeline(pipeline, pipelineAssemblers, i);
                     if (outputPushRuntime == null) {
                         throw new IllegalStateException("Invalid pipeline");
                     }
@@ -132,23 +132,6 @@
             }
         }
 
-        IPushRuntime linkSecondaryPipeline(AlgebricksPipeline pipeline, PipelineAssembler[] pipelineAssemblers,
-                int pipelineAssemblersCount) {
-            IPushRuntimeFactory[] outputRuntimeFactories = pipeline.getOutputRuntimeFactories();
-            if (outputRuntimeFactories == null || outputRuntimeFactories.length != 1) {
-                throw new IllegalStateException();
-            }
-            IPushRuntimeFactory outRuntimeFactory = outputRuntimeFactories[0];
-            int outputPosition = pipeline.getOutputPositions()[0];
-            for (int i = 0; i < pipelineAssemblersCount; i++) {
-                IPushRuntime[] p = pipelineAssemblers[i].getPushRuntime(outRuntimeFactory);
-                if (p != null) {
-                    return p[outputPosition];
-                }
-            }
-            return null;
-        }
-
         @Override
         public void open() throws HyracksDataException {
             // writer opened many times?