Merge branch 'gerrit/ionic' into 'master'

Change-Id: I61f6c538fe19f93413847555bce3884c1c04f159
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 df8d4cd..33c5baa 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
@@ -860,7 +860,7 @@
 
                 AbstractFunctionCallExpression theFieldAccessFunc;
                 LogicalVariable fieldVar = context.newVar();
-                if (fieldType == null && !ATypeTag.ANY.equals(skType.getTypeTag())) {
+                if (fieldType == null) {
                     // Open field. must prevent inlining to maintain the cast before the primaryOp and
                     // make handling of records with incorrect value type for this field easier and cleaner
                     context.addNotToBeInlinedVar(fieldVar);
@@ -871,6 +871,13 @@
                     theFieldAccessFunc = createCastExpression(index, skType, fieldAccessFunc, sourceLoc,
                             indexFieldId.funId, indexFieldId.extraArg);
                 } else {
+                    // sanity check. (heterogeneous) index keys with type ANY should not be on typed fields
+                    ATypeTag skTag = skType.getTypeTag();
+                    if (ATypeTag.ANY.equals(skTag)) {
+                        throw new CompilationException(ErrorCode.COMPILATION_ILLEGAL_STATE, sourceLoc,
+                                "type mismatch for field " + indexFieldId.fieldName + ". secondary key type " + skTag
+                                        + ", field type " + fieldType.getTypeTag());
+                    }
                     // Get the desired field position
                     int pos = indexFieldId.fieldName.size() > 1 ? -1
                             : sourceType.getFieldIndex(indexFieldId.fieldName.get(0));
@@ -903,12 +910,13 @@
     private static IndexFieldId createIndexFieldId(Index index, List<String> skName, IAType skType, Integer skSrc,
             ARecordType sourceType, SourceLocation srcLoc) throws AlgebricksException {
         IAType fieldType = sourceType.getSubFieldType(skName);
-        FunctionIdentifier skFun = null;
-        IAObject fmtArg = null;
         Pair<FunctionIdentifier, IAObject> castExpr;
         if (ATypeTag.ANY.equals(skType.getTypeTag())) {
-            return new IndexFieldId(skSrc, skName, skType.getTypeTag(), skFun, fmtArg);
+            // heterogeneous index needs to make sure the keys are in the open format (for records & lists values)
+            return new IndexFieldId(skSrc, skName, skType.getTypeTag(), BuiltinFunctions.CAST_TYPE, null);
         }
+        FunctionIdentifier skFun = null;
+        IAObject fmtArg = null;
         if (fieldType == null) {
             // open field
             castExpr = getCastExpression(index, skType, srcLoc);
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 b4c9484..d82a8be 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
@@ -48,8 +48,10 @@
 import org.apache.asterix.om.base.ADouble;
 import org.apache.asterix.om.base.IAObject;
 import org.apache.asterix.om.functions.BuiltinFunctions;
+import org.apache.asterix.om.typecomputer.base.TypeCastUtils;
 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.BuiltinType;
 import org.apache.asterix.om.types.IAType;
 import org.apache.asterix.om.types.hierachy.ATypeHierarchy;
@@ -86,6 +88,7 @@
 import org.apache.hyracks.algebricks.core.algebra.operators.logical.SelectOperator;
 import org.apache.hyracks.algebricks.core.algebra.operators.logical.UnnestMapOperator;
 import org.apache.hyracks.algebricks.core.algebra.util.OperatorManipulationUtil;
+import org.apache.hyracks.api.exceptions.SourceLocation;
 import org.apache.hyracks.util.LogRedactionUtil;
 
 /**
@@ -348,6 +351,7 @@
         // we made sure indexSubTree has datasource scan
         AbstractDataSourceOperator dataSourceOp =
                 (AbstractDataSourceOperator) indexSubTree.getDataSourceRef().getValue();
+        SourceLocation dataSrcLoc = dataSourceOp.getSourceLocation();
         List<Pair<Integer, Integer>> exprAndVarList = analysisCtx.getIndexExprsFromIndexExprsAndVars(chosenIndex);
         int numSecondaryKeys = analysisCtx.getNumberOfMatchedKeys(chosenIndex);
 
@@ -643,7 +647,7 @@
         if (!assignKeyVarList.isEmpty()) {
             // Assign operator that sets the constant secondary-index search-key fields if necessary.
             AssignOperator assignSearchKeys = new AssignOperator(assignKeyVarList, assignKeyExprList);
-            assignSearchKeys.setSourceLocation(dataSourceOp.getSourceLocation());
+            assignSearchKeys.setSourceLocation(dataSrcLoc);
             if (probeSubTree == null) {
                 // We are optimizing a selection query.
                 // Input to this assign is the EmptyTupleSource (which the dataSourceScan also must have had as input).
@@ -674,6 +678,11 @@
             inputOp = probeSubTree.getRoot();
         }
 
+        // if a key is of type ANY, we need to cast the search value to ANY if it is record/list for proper comparison
+        if (chosenIndexKeyFieldTypes.stream().anyMatch(t -> t.getTypeTag() == ATypeTag.ANY)) {
+            inputOp = addCastAssignOp(context, chosenIndexKeyFieldTypes, jobGenParams, inputOp, dataSrcLoc);
+        }
+
         // Creates an unnest-map for the secondary index search.
         // The result: SK, PK, [Optional - the result of an instantTrylock on PK]
         ILogicalOperator secondaryIndexUnnestOp = AccessMethodUtils.createSecondaryIndexUnnestMap(dataset, recordType,
@@ -763,7 +772,7 @@
                 IFunctionInfo primaryIndexSearch = FunctionUtil.getFunctionInfo(BuiltinFunctions.INDEX_SEARCH);
                 UnnestingFunctionCallExpression primaryIndexSearchFunc =
                         new UnnestingFunctionCallExpression(primaryIndexSearch, primaryIndexFuncArgs);
-                primaryIndexSearchFunc.setSourceLocation(dataSourceOp.getSourceLocation());
+                primaryIndexSearchFunc.setSourceLocation(dataSrcLoc);
                 primaryIndexSearchFunc.setReturnsUniqueValues(true);
                 if (!leftOuterUnnestMapRequired) {
                     unnestMapOp = new UnnestMapOperator(scanVariables, new MutableObject<>(primaryIndexSearchFunc),
@@ -785,7 +794,7 @@
                 }
             }
             unnestMapOp.setExecutionMode(ExecutionMode.PARTITIONED);
-            unnestMapOp.setSourceLocation(dataSourceOp.getSourceLocation());
+            unnestMapOp.setSourceLocation(dataSrcLoc);
             unnestMapOp.getInputs().add(new MutableObject<>(inputOp));
             context.computeAndSetTypeEnvironmentForOperator(unnestMapOp);
             indexSearchOp = unnestMapOp;
@@ -800,6 +809,51 @@
         return indexSearchOp;
     }
 
+    private static ILogicalOperator addCastAssignOp(IOptimizationContext ctx, List<IAType> indexKeysTypes,
+            BTreeJobGenParams jobGenParams, ILogicalOperator inputOp, SourceLocation srcLoc)
+            throws AlgebricksException {
+        // cast the input values (low/high vars) if needed and update the jobGenParams
+        List<LogicalVariable> lowKeyVars = jobGenParams.getLowKeyVarList();
+        List<LogicalVariable> highKeyVars = jobGenParams.getHighKeyVarList();
+        List<LogicalVariable> castAssignVars = new ArrayList<>();
+        List<Mutable<ILogicalExpression>> castAssignExprs = new ArrayList<>();
+        castInputValues(ctx, indexKeysTypes, lowKeyVars, inputOp, castAssignVars, castAssignExprs, srcLoc);
+        castInputValues(ctx, indexKeysTypes, highKeyVars, inputOp, castAssignVars, castAssignExprs, srcLoc);
+        if (castAssignVars.isEmpty()) {
+            return inputOp;
+        }
+        AssignOperator castAssignOp = new AssignOperator(castAssignVars, castAssignExprs);
+        castAssignOp.setSourceLocation(srcLoc);
+        castAssignOp.getInputs().add(new MutableObject<>(inputOp));
+        castAssignOp.setExecutionMode(inputOp.getExecutionMode());
+        return castAssignOp;
+    }
+
+    private static void castInputValues(IOptimizationContext ctx, List<IAType> indexKeyTypes,
+            List<LogicalVariable> inputVars, ILogicalOperator inputOp, List<LogicalVariable> castAssignVars,
+            List<Mutable<ILogicalExpression>> castAssignExprs, SourceLocation srcLoc) throws AlgebricksException {
+        for (int i = 0; i < inputVars.size(); i++) {
+            LogicalVariable inputVar = inputVars.get(i);
+            IVariableTypeEnvironment typeEnv = ctx.getOutputTypeEnvironment(inputOp);
+            IAType varType = (IAType) typeEnv.getVarType(inputVar);
+            IAType varActualType = TypeComputeUtils.getActualType(varType);
+            IAType indexKeyType = indexKeyTypes.get(i);
+            if (varActualType.getTypeTag().isDerivedType() && indexKeyType.getTypeTag() == ATypeTag.ANY) {
+                LogicalVariable newInputVar = ctx.newVar();
+                castAssignVars.add(newInputVar);
+                VariableReferenceExpression newInputVarRef = new VariableReferenceExpression(inputVar);
+                newInputVarRef.setSourceLocation(srcLoc);
+                ScalarFunctionCallExpression castFunc = new ScalarFunctionCallExpression(
+                        BuiltinFunctions.getBuiltinFunctionInfo(BuiltinFunctions.CAST_TYPE),
+                        new ArrayList<>(Collections.singletonList(new MutableObject<>(newInputVarRef))));
+                castFunc.setSourceLocation(srcLoc);
+                TypeCastUtils.setRequiredAndInputTypes(castFunc, BuiltinType.ANY, varType);
+                castAssignExprs.add(new MutableObject<>(castFunc));
+                inputVars.set(i, newInputVar);
+            }
+        }
+    }
+
     private int createKeyVarsAndExprs(int numKeys, LimitType[] keyLimits, ILogicalExpression[] searchKeyExprs,
             ArrayList<LogicalVariable> assignKeyVarList, ArrayList<Mutable<ILogicalExpression>> assignKeyExprList,
             ArrayList<LogicalVariable> keyVarList, IOptimizationContext context, ILogicalExpression[] constExpressions,
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 579ad51..333b82f 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
@@ -276,8 +276,12 @@
                     case UUID:
                     case YEARMONTHDURATION:
                     case DAYTIMEDURATION:
-                    case ANY:
                         break;
+                    case ANY:
+                        if (indexType == IndexType.BTREE) {
+                            // ANY is only allowed for normal secondary indexes
+                            break;
+                        }
                     default:
                         throw new CompilationException(ErrorCode.COMPILATION_ERROR, sourceLoc,
                                 "The field '"
diff --git a/asterixdb/asterix-app/pom.xml b/asterixdb/asterix-app/pom.xml
index 3df87ce..a431f68 100644
--- a/asterixdb/asterix-app/pom.xml
+++ b/asterixdb/asterix-app/pom.xml
@@ -1183,10 +1183,12 @@
     <dependency>
       <groupId>org.apache.iceberg</groupId>
       <artifactId>iceberg-data</artifactId>
+      <scope>test</scope>
     </dependency>
     <dependency>
       <groupId>org.apache.iceberg</groupId>
       <artifactId>iceberg-parquet</artifactId>
+      <scope>test</scope>
     </dependency>
     <dependency>
       <groupId>tech.allegro.schema.json2avro</groupId>
diff --git a/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/function/TPCDSAllTablesDataGeneratorDatasource.java b/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/function/TPCDSAllTablesDataGeneratorDatasource.java
index ac19b00..89d4120 100644
--- a/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/function/TPCDSAllTablesDataGeneratorDatasource.java
+++ b/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/function/TPCDSAllTablesDataGeneratorDatasource.java
@@ -70,7 +70,7 @@
 
     @Override
     protected AlgebricksAbsolutePartitionConstraint getLocations(IClusterStateManager csm, MetadataProvider md) {
-        return md.getDataPartitioningProvider().getClusterLocations();
+        return md.getClusterLocations();
     }
 
     @Override
diff --git a/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/function/TPCDSSingleTableDataGeneratorDatasource.java b/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/function/TPCDSSingleTableDataGeneratorDatasource.java
index 6551346..46a9303 100644
--- a/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/function/TPCDSSingleTableDataGeneratorDatasource.java
+++ b/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/function/TPCDSSingleTableDataGeneratorDatasource.java
@@ -78,7 +78,7 @@
 
     @Override
     protected AlgebricksAbsolutePartitionConstraint getLocations(IClusterStateManager csm, MetadataProvider md) {
-        return md.getDataPartitioningProvider().getClusterLocations();
+        return md.getClusterLocations();
     }
 
     @Override
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/index-selection/heterogeneous-index-complex-types/heterogeneous-index-complex-types.01.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/index-selection/heterogeneous-index-complex-types/heterogeneous-index-complex-types.01.ddl.sqlpp
new file mode 100644
index 0000000..d27c56e
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/index-selection/heterogeneous-index-complex-types/heterogeneous-index-complex-types.01.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.
+ */
+
+DROP DATAVERSE test IF EXISTS;
+CREATE DATAVERSE test;
+USE test;
+
+CREATE TYPE t1 AS {
+    id: string,
+    c_intarr: [int],
+    c_strarr: [string],
+    c_obj: {a: int},
+    c_int: int
+};
+
+CREATE COLLECTION ds1(t1) PRIMARY KEY id;
+CREATE COLLECTION ds2 PRIMARY KEY(id: string);
+
+CREATE INDEX idx_intarr ON ds1(intarr);
+CREATE INDEX idx_strarr ON ds1(strarr);
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/index-selection/heterogeneous-index-complex-types/heterogeneous-index-complex-types.02.update.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/index-selection/heterogeneous-index-complex-types/heterogeneous-index-complex-types.02.update.sqlpp
new file mode 100644
index 0000000..bce03c1
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/index-selection/heterogeneous-index-complex-types/heterogeneous-index-complex-types.02.update.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.
+ */
+
+USE test;
+
+// testing index maintenance path on ds1
+UPSERT INTO ds1({
+"id": "11",
+"strarr": [ "a", "b" ],
+"intarr": [ 1, 2],
+"c_strarr": [ "a", "b" ],
+"c_intarr": [ 1, 2 ],
+"c_obj": {"a": 9},
+"c_int": 10
+});
+
+// testing index bulk loading path
+UPSERT INTO ds2({
+"id": "22",
+"strarr": [ "a", "b" ],
+"intarr": [ 1, 2]
+});
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/index-selection/heterogeneous-index-complex-types/heterogeneous-index-complex-types.03.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/index-selection/heterogeneous-index-complex-types/heterogeneous-index-complex-types.03.ddl.sqlpp
new file mode 100644
index 0000000..33a36dc
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/index-selection/heterogeneous-index-complex-types/heterogeneous-index-complex-types.03.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.
+ */
+// testing index bulk loading path
+USE test;
+
+CREATE INDEX idx_intarr ON ds2(intarr);
+CREATE INDEX idx_strarr ON ds2(strarr);
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/index-selection/heterogeneous-index-complex-types/heterogeneous-index-complex-types.04.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/index-selection/heterogeneous-index-complex-types/heterogeneous-index-complex-types.04.query.sqlpp
new file mode 100644
index 0000000..0ac93a5
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/index-selection/heterogeneous-index-complex-types/heterogeneous-index-complex-types.04.query.sqlpp
@@ -0,0 +1,22 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+USE test;
+
+SELECT VALUE ds1 FROM ds1 WHERE intarr = [1, 2];
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/index-selection/heterogeneous-index-complex-types/heterogeneous-index-complex-types.05.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/index-selection/heterogeneous-index-complex-types/heterogeneous-index-complex-types.05.query.sqlpp
new file mode 100644
index 0000000..cb0d7ba
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/index-selection/heterogeneous-index-complex-types/heterogeneous-index-complex-types.05.query.sqlpp
@@ -0,0 +1,22 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+USE test;
+
+SELECT VALUE ds1 FROM ds1 WHERE strarr = [ "a", "b" ];
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/index-selection/heterogeneous-index-complex-types/heterogeneous-index-complex-types.06.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/index-selection/heterogeneous-index-complex-types/heterogeneous-index-complex-types.06.query.sqlpp
new file mode 100644
index 0000000..cf2b1e5
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/index-selection/heterogeneous-index-complex-types/heterogeneous-index-complex-types.06.query.sqlpp
@@ -0,0 +1,22 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+USE test;
+
+SELECT VALUE ds2 FROM ds2 WHERE intarr = [1, 2];
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/index-selection/heterogeneous-index-complex-types/heterogeneous-index-complex-types.07.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/index-selection/heterogeneous-index-complex-types/heterogeneous-index-complex-types.07.query.sqlpp
new file mode 100644
index 0000000..8121dd2
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/index-selection/heterogeneous-index-complex-types/heterogeneous-index-complex-types.07.query.sqlpp
@@ -0,0 +1,22 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+USE test;
+
+SELECT VALUE ds2 FROM ds2 WHERE strarr = [ "a", "b" ];
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/index-selection/heterogeneous-index-complex-types/heterogeneous-index-complex-types.08.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/index-selection/heterogeneous-index-complex-types/heterogeneous-index-complex-types.08.query.sqlpp
new file mode 100644
index 0000000..143691a
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/index-selection/heterogeneous-index-complex-types/heterogeneous-index-complex-types.08.query.sqlpp
@@ -0,0 +1,22 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+USE test;
+
+SELECT d1,d2 FROM ds1 d1 JOIN ds2 d2 ON d1.intarr /*+ indexnl */ = d2.intarr;
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/index-selection/heterogeneous-index-complex-types/heterogeneous-index-complex-types.09.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/index-selection/heterogeneous-index-complex-types/heterogeneous-index-complex-types.09.query.sqlpp
new file mode 100644
index 0000000..643896b
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/index-selection/heterogeneous-index-complex-types/heterogeneous-index-complex-types.09.query.sqlpp
@@ -0,0 +1,22 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+USE test;
+
+SELECT d1,d2 FROM ds1 d1 JOIN ds2 d2 ON d1.c_intarr /*+ indexnl */ = d2.intarr;
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/index-selection/heterogeneous-index-complex-types/heterogeneous-index-complex-types.10.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/index-selection/heterogeneous-index-complex-types/heterogeneous-index-complex-types.10.query.sqlpp
new file mode 100644
index 0000000..028fadf
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/index-selection/heterogeneous-index-complex-types/heterogeneous-index-complex-types.10.query.sqlpp
@@ -0,0 +1,22 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+USE test;
+
+SELECT d1,d2 FROM ds1 d1 JOIN ds2 d2 ON d1.strarr /*+ indexnl */ = d2.strarr;
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/index-selection/heterogeneous-index-complex-types/heterogeneous-index-complex-types.11.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/index-selection/heterogeneous-index-complex-types/heterogeneous-index-complex-types.11.query.sqlpp
new file mode 100644
index 0000000..dc8c2da
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/index-selection/heterogeneous-index-complex-types/heterogeneous-index-complex-types.11.query.sqlpp
@@ -0,0 +1,22 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+USE test;
+
+SELECT d1,d2 FROM ds1 d1 JOIN ds2 d2 ON d1.c_strarr /*+ indexnl */ = d2.strarr;
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/index-selection/heterogeneous-index-complex-types/heterogeneous-index-complex-types.12.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/index-selection/heterogeneous-index-complex-types/heterogeneous-index-complex-types.12.query.sqlpp
new file mode 100644
index 0000000..32f6b8f
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/index-selection/heterogeneous-index-complex-types/heterogeneous-index-complex-types.12.query.sqlpp
@@ -0,0 +1,22 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+USE test;
+
+SELECT d1,d2 FROM ds1 d1 JOIN ds2 d2 ON d1.c_obj /*+ indexnl */ = d2.strarr;
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/index-selection/heterogeneous-index-complex-types/heterogeneous-index-complex-types.13.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/index-selection/heterogeneous-index-complex-types/heterogeneous-index-complex-types.13.query.sqlpp
new file mode 100644
index 0000000..c14cd29
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/index-selection/heterogeneous-index-complex-types/heterogeneous-index-complex-types.13.query.sqlpp
@@ -0,0 +1,22 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+USE test;
+
+SELECT d1,d2 FROM ds1 d1 JOIN ds2 d2 ON d1.c_int /*+ indexnl */ = d2.strarr;
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/index-selection/heterogeneous-index-complex-types/heterogeneous-index-complex-types.14.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/index-selection/heterogeneous-index-complex-types/heterogeneous-index-complex-types.14.query.sqlpp
new file mode 100644
index 0000000..374c400
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/index-selection/heterogeneous-index-complex-types/heterogeneous-index-complex-types.14.query.sqlpp
@@ -0,0 +1,22 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+USE test;
+SET `compiler.cbo` "false";
+EXPLAIN SELECT d1,d2 FROM ds1 d1 JOIN ds2 d2 ON d1.intarr /*+ indexnl */ = d2.intarr;
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/index-selection/heterogeneous-index-complex-types/heterogeneous-index-complex-types.15.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/index-selection/heterogeneous-index-complex-types/heterogeneous-index-complex-types.15.query.sqlpp
new file mode 100644
index 0000000..5f5a1eb
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/index-selection/heterogeneous-index-complex-types/heterogeneous-index-complex-types.15.query.sqlpp
@@ -0,0 +1,22 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+USE test;
+SET `compiler.cbo` "false";
+EXPLAIN SELECT d1,d2 FROM ds1 d1 JOIN ds2 d2 ON d1.c_intarr /*+ indexnl */ = d2.intarr;
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/index-selection/heterogeneous-index-complex-types/heterogeneous-index-complex-types.16.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/index-selection/heterogeneous-index-complex-types/heterogeneous-index-complex-types.16.query.sqlpp
new file mode 100644
index 0000000..38d7553
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/index-selection/heterogeneous-index-complex-types/heterogeneous-index-complex-types.16.query.sqlpp
@@ -0,0 +1,22 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+USE test;
+SET `compiler.cbo` "false";
+EXPLAIN SELECT d1,d2 FROM ds1 d1 JOIN ds2 d2 ON d1.strarr /*+ indexnl */ = d2.strarr;
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/index-selection/heterogeneous-index-complex-types/heterogeneous-index-complex-types.17.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/index-selection/heterogeneous-index-complex-types/heterogeneous-index-complex-types.17.query.sqlpp
new file mode 100644
index 0000000..c6d5981
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/index-selection/heterogeneous-index-complex-types/heterogeneous-index-complex-types.17.query.sqlpp
@@ -0,0 +1,22 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+USE test;
+SET `compiler.cbo` "false";
+EXPLAIN SELECT d1,d2 FROM ds1 d1 JOIN ds2 d2 ON d1.c_strarr /*+ indexnl */ = d2.strarr;
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/index-selection/heterogeneous-index-complex-types/heterogeneous-index-complex-types.18.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/index-selection/heterogeneous-index-complex-types/heterogeneous-index-complex-types.18.query.sqlpp
new file mode 100644
index 0000000..98cefb4
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/index-selection/heterogeneous-index-complex-types/heterogeneous-index-complex-types.18.query.sqlpp
@@ -0,0 +1,22 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+USE test;
+SET `compiler.cbo` "false";
+EXPLAIN SELECT d1,d2 FROM ds1 d1 JOIN ds2 d2 ON d1.c_obj /*+ indexnl */ = d2.strarr;
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/index-selection/heterogeneous-index-complex-types/heterogeneous-index-complex-types.19.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/index-selection/heterogeneous-index-complex-types/heterogeneous-index-complex-types.19.query.sqlpp
new file mode 100644
index 0000000..68672e4
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/index-selection/heterogeneous-index-complex-types/heterogeneous-index-complex-types.19.query.sqlpp
@@ -0,0 +1,22 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+USE test;
+SET `compiler.cbo` "false";
+EXPLAIN SELECT d1,d2 FROM ds1 d1 JOIN ds2 d2 ON d1.c_int /*+ indexnl */ = d2.strarr;
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/index-selection/heterogeneous-index-complex-types/heterogeneous-index-complex-types.99.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/index-selection/heterogeneous-index-complex-types/heterogeneous-index-complex-types.99.ddl.sqlpp
new file mode 100644
index 0000000..36b2bab
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/index-selection/heterogeneous-index-complex-types/heterogeneous-index-complex-types.99.ddl.sqlpp
@@ -0,0 +1,20 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+DROP DATAVERSE test IF EXISTS;
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/index-selection/heterogeneous-index-complex-types/heterogeneous-index-complex-types.04.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/index-selection/heterogeneous-index-complex-types/heterogeneous-index-complex-types.04.adm
new file mode 100644
index 0000000..3bd1a41
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/index-selection/heterogeneous-index-complex-types/heterogeneous-index-complex-types.04.adm
@@ -0,0 +1 @@
+{ "id": "11", "c_intarr": [ 1, 2 ], "c_strarr": [ "a", "b" ], "c_obj": { "a": 9 }, "c_int": 10, "strarr": [ "a", "b" ], "intarr": [ 1, 2 ] }
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/index-selection/heterogeneous-index-complex-types/heterogeneous-index-complex-types.05.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/index-selection/heterogeneous-index-complex-types/heterogeneous-index-complex-types.05.adm
new file mode 100644
index 0000000..3bd1a41
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/index-selection/heterogeneous-index-complex-types/heterogeneous-index-complex-types.05.adm
@@ -0,0 +1 @@
+{ "id": "11", "c_intarr": [ 1, 2 ], "c_strarr": [ "a", "b" ], "c_obj": { "a": 9 }, "c_int": 10, "strarr": [ "a", "b" ], "intarr": [ 1, 2 ] }
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/index-selection/heterogeneous-index-complex-types/heterogeneous-index-complex-types.06.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/index-selection/heterogeneous-index-complex-types/heterogeneous-index-complex-types.06.adm
new file mode 100644
index 0000000..1e2a25b
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/index-selection/heterogeneous-index-complex-types/heterogeneous-index-complex-types.06.adm
@@ -0,0 +1 @@
+{ "id": "22", "strarr": [ "a", "b" ], "intarr": [ 1, 2 ] }
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/index-selection/heterogeneous-index-complex-types/heterogeneous-index-complex-types.07.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/index-selection/heterogeneous-index-complex-types/heterogeneous-index-complex-types.07.adm
new file mode 100644
index 0000000..1e2a25b
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/index-selection/heterogeneous-index-complex-types/heterogeneous-index-complex-types.07.adm
@@ -0,0 +1 @@
+{ "id": "22", "strarr": [ "a", "b" ], "intarr": [ 1, 2 ] }
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/index-selection/heterogeneous-index-complex-types/heterogeneous-index-complex-types.08.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/index-selection/heterogeneous-index-complex-types/heterogeneous-index-complex-types.08.adm
new file mode 100644
index 0000000..8fa90c6
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/index-selection/heterogeneous-index-complex-types/heterogeneous-index-complex-types.08.adm
@@ -0,0 +1 @@
+{ "d1": { "id": "11", "c_intarr": [ 1, 2 ], "c_strarr": [ "a", "b" ], "c_obj": { "a": 9 }, "c_int": 10, "strarr": [ "a", "b" ], "intarr": [ 1, 2 ] }, "d2": { "id": "22", "strarr": [ "a", "b" ], "intarr": [ 1, 2 ] } }
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/index-selection/heterogeneous-index-complex-types/heterogeneous-index-complex-types.09.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/index-selection/heterogeneous-index-complex-types/heterogeneous-index-complex-types.09.adm
new file mode 100644
index 0000000..8fa90c6
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/index-selection/heterogeneous-index-complex-types/heterogeneous-index-complex-types.09.adm
@@ -0,0 +1 @@
+{ "d1": { "id": "11", "c_intarr": [ 1, 2 ], "c_strarr": [ "a", "b" ], "c_obj": { "a": 9 }, "c_int": 10, "strarr": [ "a", "b" ], "intarr": [ 1, 2 ] }, "d2": { "id": "22", "strarr": [ "a", "b" ], "intarr": [ 1, 2 ] } }
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/index-selection/heterogeneous-index-complex-types/heterogeneous-index-complex-types.10.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/index-selection/heterogeneous-index-complex-types/heterogeneous-index-complex-types.10.adm
new file mode 100644
index 0000000..8fa90c6
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/index-selection/heterogeneous-index-complex-types/heterogeneous-index-complex-types.10.adm
@@ -0,0 +1 @@
+{ "d1": { "id": "11", "c_intarr": [ 1, 2 ], "c_strarr": [ "a", "b" ], "c_obj": { "a": 9 }, "c_int": 10, "strarr": [ "a", "b" ], "intarr": [ 1, 2 ] }, "d2": { "id": "22", "strarr": [ "a", "b" ], "intarr": [ 1, 2 ] } }
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/index-selection/heterogeneous-index-complex-types/heterogeneous-index-complex-types.11.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/index-selection/heterogeneous-index-complex-types/heterogeneous-index-complex-types.11.adm
new file mode 100644
index 0000000..8fa90c6
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/index-selection/heterogeneous-index-complex-types/heterogeneous-index-complex-types.11.adm
@@ -0,0 +1 @@
+{ "d1": { "id": "11", "c_intarr": [ 1, 2 ], "c_strarr": [ "a", "b" ], "c_obj": { "a": 9 }, "c_int": 10, "strarr": [ "a", "b" ], "intarr": [ 1, 2 ] }, "d2": { "id": "22", "strarr": [ "a", "b" ], "intarr": [ 1, 2 ] } }
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/index-selection/heterogeneous-index-complex-types/heterogeneous-index-complex-types.12.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/index-selection/heterogeneous-index-complex-types/heterogeneous-index-complex-types.12.adm
new file mode 100644
index 0000000..e69de29
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/index-selection/heterogeneous-index-complex-types/heterogeneous-index-complex-types.12.adm
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/index-selection/heterogeneous-index-complex-types/heterogeneous-index-complex-types.13.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/index-selection/heterogeneous-index-complex-types/heterogeneous-index-complex-types.13.adm
new file mode 100644
index 0000000..e69de29
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/index-selection/heterogeneous-index-complex-types/heterogeneous-index-complex-types.13.adm
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/index-selection/heterogeneous-index-complex-types/heterogeneous-index-complex-types.14.plan b/asterixdb/asterix-app/src/test/resources/runtimets/results/index-selection/heterogeneous-index-complex-types/heterogeneous-index-complex-types.14.plan
new file mode 100644
index 0000000..abe5395
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/index-selection/heterogeneous-index-complex-types/heterogeneous-index-complex-types.14.plan
@@ -0,0 +1,40 @@
+distribute result [$$29]
+-- DISTRIBUTE_RESULT  |PARTITIONED|
+  exchange
+  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+    assign [$$29] <- [{"d1": $$d1, "d2": $$d2}] project: [$$29]
+    -- ASSIGN  |PARTITIONED|
+      select (eq($$32, $$d2.getField("intarr"))) project: [$$d1, $$d2]
+      -- STREAM_SELECT  |PARTITIONED|
+        project ([$$d1, $$32, $$d2])
+        -- STREAM_PROJECT  |PARTITIONED|
+          exchange
+          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+            unnest-map [$$31, $$d2] <- index-search("ds2", 0, "Default", "test", "ds2", true, false, 1, $$35, 1, $$35, true, true, true)
+            -- BTREE_SEARCH  |PARTITIONED|
+              exchange
+              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                order (ASC, $$35)
+                -- STABLE_SORT [$$35(ASC)]  |PARTITIONED|
+                  exchange
+                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                    select (eq($$32, $$34)) project: [$$d1, $$32, $$35]
+                    -- STREAM_SELECT  |PARTITIONED|
+                      exchange
+                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                        unnest-map [$$34, $$35] <- index-search("idx_intarr", 0, "Default", "test", "ds2", true, true, 1, $$32, 1, $$32, true, true, true)
+                        -- BTREE_SEARCH  |PARTITIONED|
+                          exchange
+                          -- BROADCAST_EXCHANGE  |PARTITIONED|
+                            assign [$$32] <- [$$d1.getField("intarr")]
+                            -- ASSIGN  |PARTITIONED|
+                              project ([$$d1])
+                              -- STREAM_PROJECT  |PARTITIONED|
+                                exchange
+                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                  data-scan []<-[$$30, $$d1] <- test.ds1
+                                  -- DATASOURCE_SCAN  |PARTITIONED|
+                                    exchange
+                                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                      empty-tuple-source
+                                      -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/index-selection/heterogeneous-index-complex-types/heterogeneous-index-complex-types.15.plan b/asterixdb/asterix-app/src/test/resources/runtimets/results/index-selection/heterogeneous-index-complex-types/heterogeneous-index-complex-types.15.plan
new file mode 100644
index 0000000..d0e0a38
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/index-selection/heterogeneous-index-complex-types/heterogeneous-index-complex-types.15.plan
@@ -0,0 +1,42 @@
+distribute result [$$29]
+-- DISTRIBUTE_RESULT  |PARTITIONED|
+  exchange
+  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+    assign [$$29] <- [{"d1": $$d1, "d2": $$d2}] project: [$$29]
+    -- ASSIGN  |PARTITIONED|
+      select (eq($$32, $$d2.getField("intarr"))) project: [$$d1, $$d2]
+      -- STREAM_SELECT  |PARTITIONED|
+        project ([$$d1, $$32, $$d2])
+        -- STREAM_PROJECT  |PARTITIONED|
+          exchange
+          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+            unnest-map [$$31, $$d2] <- index-search("ds2", 0, "Default", "test", "ds2", true, false, 1, $$37, 1, $$37, true, true, true)
+            -- BTREE_SEARCH  |PARTITIONED|
+              exchange
+              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                order (ASC, $$37)
+                -- STABLE_SORT [$$37(ASC)]  |PARTITIONED|
+                  exchange
+                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                    project ([$$d1, $$32, $$37])
+                    -- STREAM_PROJECT  |PARTITIONED|
+                      exchange
+                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                        unnest-map [$$36, $$37] <- index-search("idx_intarr", 0, "Default", "test", "ds2", true, true, 1, $$38, 1, $$38, true, true, true)
+                        -- BTREE_SEARCH  |PARTITIONED|
+                          exchange
+                          -- BROADCAST_EXCHANGE  |PARTITIONED|
+                            assign [$$38] <- [cast($$32)]
+                            -- ASSIGN  |PARTITIONED|
+                              assign [$$32] <- [$$d1.getField(1)]
+                              -- ASSIGN  |PARTITIONED|
+                                project ([$$d1])
+                                -- STREAM_PROJECT  |PARTITIONED|
+                                  exchange
+                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                    data-scan []<-[$$30, $$d1] <- test.ds1
+                                    -- DATASOURCE_SCAN  |PARTITIONED|
+                                      exchange
+                                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                        empty-tuple-source
+                                        -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/index-selection/heterogeneous-index-complex-types/heterogeneous-index-complex-types.16.plan b/asterixdb/asterix-app/src/test/resources/runtimets/results/index-selection/heterogeneous-index-complex-types/heterogeneous-index-complex-types.16.plan
new file mode 100644
index 0000000..e4b088e
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/index-selection/heterogeneous-index-complex-types/heterogeneous-index-complex-types.16.plan
@@ -0,0 +1,40 @@
+distribute result [$$29]
+-- DISTRIBUTE_RESULT  |PARTITIONED|
+  exchange
+  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+    assign [$$29] <- [{"d1": $$d1, "d2": $$d2}] project: [$$29]
+    -- ASSIGN  |PARTITIONED|
+      select (eq($$32, $$d2.getField("strarr"))) project: [$$d1, $$d2]
+      -- STREAM_SELECT  |PARTITIONED|
+        project ([$$d1, $$32, $$d2])
+        -- STREAM_PROJECT  |PARTITIONED|
+          exchange
+          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+            unnest-map [$$31, $$d2] <- index-search("ds2", 0, "Default", "test", "ds2", true, false, 1, $$35, 1, $$35, true, true, true)
+            -- BTREE_SEARCH  |PARTITIONED|
+              exchange
+              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                order (ASC, $$35)
+                -- STABLE_SORT [$$35(ASC)]  |PARTITIONED|
+                  exchange
+                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                    select (eq($$32, $$34)) project: [$$d1, $$32, $$35]
+                    -- STREAM_SELECT  |PARTITIONED|
+                      exchange
+                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                        unnest-map [$$34, $$35] <- index-search("idx_strarr", 0, "Default", "test", "ds2", true, true, 1, $$32, 1, $$32, true, true, true)
+                        -- BTREE_SEARCH  |PARTITIONED|
+                          exchange
+                          -- BROADCAST_EXCHANGE  |PARTITIONED|
+                            assign [$$32] <- [$$d1.getField("strarr")]
+                            -- ASSIGN  |PARTITIONED|
+                              project ([$$d1])
+                              -- STREAM_PROJECT  |PARTITIONED|
+                                exchange
+                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                  data-scan []<-[$$30, $$d1] <- test.ds1
+                                  -- DATASOURCE_SCAN  |PARTITIONED|
+                                    exchange
+                                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                      empty-tuple-source
+                                      -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/index-selection/heterogeneous-index-complex-types/heterogeneous-index-complex-types.17.plan b/asterixdb/asterix-app/src/test/resources/runtimets/results/index-selection/heterogeneous-index-complex-types/heterogeneous-index-complex-types.17.plan
new file mode 100644
index 0000000..bc277e2
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/index-selection/heterogeneous-index-complex-types/heterogeneous-index-complex-types.17.plan
@@ -0,0 +1,42 @@
+distribute result [$$29]
+-- DISTRIBUTE_RESULT  |PARTITIONED|
+  exchange
+  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+    assign [$$29] <- [{"d1": $$d1, "d2": $$d2}] project: [$$29]
+    -- ASSIGN  |PARTITIONED|
+      select (eq($$32, $$d2.getField("strarr"))) project: [$$d1, $$d2]
+      -- STREAM_SELECT  |PARTITIONED|
+        project ([$$d1, $$32, $$d2])
+        -- STREAM_PROJECT  |PARTITIONED|
+          exchange
+          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+            unnest-map [$$31, $$d2] <- index-search("ds2", 0, "Default", "test", "ds2", true, false, 1, $$37, 1, $$37, true, true, true)
+            -- BTREE_SEARCH  |PARTITIONED|
+              exchange
+              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                order (ASC, $$37)
+                -- STABLE_SORT [$$37(ASC)]  |PARTITIONED|
+                  exchange
+                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                    project ([$$d1, $$32, $$37])
+                    -- STREAM_PROJECT  |PARTITIONED|
+                      exchange
+                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                        unnest-map [$$36, $$37] <- index-search("idx_strarr", 0, "Default", "test", "ds2", true, true, 1, $$38, 1, $$38, true, true, true)
+                        -- BTREE_SEARCH  |PARTITIONED|
+                          exchange
+                          -- BROADCAST_EXCHANGE  |PARTITIONED|
+                            assign [$$38] <- [cast($$32)]
+                            -- ASSIGN  |PARTITIONED|
+                              assign [$$32] <- [$$d1.getField(2)]
+                              -- ASSIGN  |PARTITIONED|
+                                project ([$$d1])
+                                -- STREAM_PROJECT  |PARTITIONED|
+                                  exchange
+                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                    data-scan []<-[$$30, $$d1] <- test.ds1
+                                    -- DATASOURCE_SCAN  |PARTITIONED|
+                                      exchange
+                                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                        empty-tuple-source
+                                        -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/index-selection/heterogeneous-index-complex-types/heterogeneous-index-complex-types.18.plan b/asterixdb/asterix-app/src/test/resources/runtimets/results/index-selection/heterogeneous-index-complex-types/heterogeneous-index-complex-types.18.plan
new file mode 100644
index 0000000..094f4ac
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/index-selection/heterogeneous-index-complex-types/heterogeneous-index-complex-types.18.plan
@@ -0,0 +1,42 @@
+distribute result [$$29]
+-- DISTRIBUTE_RESULT  |PARTITIONED|
+  exchange
+  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+    assign [$$29] <- [{"d1": $$d1, "d2": $$d2}] project: [$$29]
+    -- ASSIGN  |PARTITIONED|
+      select (eq($$32, $$d2.getField("strarr"))) project: [$$d1, $$d2]
+      -- STREAM_SELECT  |PARTITIONED|
+        project ([$$d1, $$32, $$d2])
+        -- STREAM_PROJECT  |PARTITIONED|
+          exchange
+          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+            unnest-map [$$31, $$d2] <- index-search("ds2", 0, "Default", "test", "ds2", true, false, 1, $$37, 1, $$37, true, true, true)
+            -- BTREE_SEARCH  |PARTITIONED|
+              exchange
+              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                order (ASC, $$37)
+                -- STABLE_SORT [$$37(ASC)]  |PARTITIONED|
+                  exchange
+                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                    project ([$$d1, $$32, $$37])
+                    -- STREAM_PROJECT  |PARTITIONED|
+                      exchange
+                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                        unnest-map [$$36, $$37] <- index-search("idx_strarr", 0, "Default", "test", "ds2", true, true, 1, $$38, 1, $$38, true, true, true)
+                        -- BTREE_SEARCH  |PARTITIONED|
+                          exchange
+                          -- BROADCAST_EXCHANGE  |PARTITIONED|
+                            assign [$$38] <- [cast($$32)]
+                            -- ASSIGN  |PARTITIONED|
+                              assign [$$32] <- [$$d1.getField(3)]
+                              -- ASSIGN  |PARTITIONED|
+                                project ([$$d1])
+                                -- STREAM_PROJECT  |PARTITIONED|
+                                  exchange
+                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                    data-scan []<-[$$30, $$d1] <- test.ds1
+                                    -- DATASOURCE_SCAN  |PARTITIONED|
+                                      exchange
+                                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                        empty-tuple-source
+                                        -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/index-selection/heterogeneous-index-complex-types/heterogeneous-index-complex-types.19.plan b/asterixdb/asterix-app/src/test/resources/runtimets/results/index-selection/heterogeneous-index-complex-types/heterogeneous-index-complex-types.19.plan
new file mode 100644
index 0000000..e2aa114
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/index-selection/heterogeneous-index-complex-types/heterogeneous-index-complex-types.19.plan
@@ -0,0 +1,40 @@
+distribute result [$$29]
+-- DISTRIBUTE_RESULT  |PARTITIONED|
+  exchange
+  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+    assign [$$29] <- [{"d1": $$d1, "d2": $$d2}] project: [$$29]
+    -- ASSIGN  |PARTITIONED|
+      select (eq($$32, $$d2.getField("strarr"))) project: [$$d1, $$d2]
+      -- STREAM_SELECT  |PARTITIONED|
+        project ([$$d1, $$32, $$d2])
+        -- STREAM_PROJECT  |PARTITIONED|
+          exchange
+          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+            unnest-map [$$31, $$d2] <- index-search("ds2", 0, "Default", "test", "ds2", true, false, 1, $$35, 1, $$35, true, true, true)
+            -- BTREE_SEARCH  |PARTITIONED|
+              exchange
+              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                order (ASC, $$35)
+                -- STABLE_SORT [$$35(ASC)]  |PARTITIONED|
+                  exchange
+                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                    select (eq($$32, $$34)) project: [$$d1, $$32, $$35]
+                    -- STREAM_SELECT  |PARTITIONED|
+                      exchange
+                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                        unnest-map [$$34, $$35] <- index-search("idx_strarr", 0, "Default", "test", "ds2", true, true, 1, $$32, 1, $$32, true, true, true)
+                        -- BTREE_SEARCH  |PARTITIONED|
+                          exchange
+                          -- BROADCAST_EXCHANGE  |PARTITIONED|
+                            assign [$$32] <- [$$d1.getField(4)]
+                            -- ASSIGN  |PARTITIONED|
+                              project ([$$d1])
+                              -- STREAM_PROJECT  |PARTITIONED|
+                                exchange
+                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                  data-scan []<-[$$30, $$d1] <- test.ds1
+                                  -- DATASOURCE_SCAN  |PARTITIONED|
+                                    exchange
+                                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                      empty-tuple-source
+                                      -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/sqlpp_queries.xml b/asterixdb/asterix-app/src/test/resources/runtimets/sqlpp_queries.xml
index fadf9df..4ae1ea6 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/sqlpp_queries.xml
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/sqlpp_queries.xml
@@ -6736,6 +6736,11 @@
         <output-dir compare="Text">heterogeneous-index-Leftouterjoin</output-dir>
       </compilation-unit>
     </test-case>
+    <test-case FilePath="index-selection">
+      <compilation-unit name="heterogeneous-index-complex-types">
+        <output-dir compare="Text">heterogeneous-index-complex-types</output-dir>
+      </compilation-unit>
+    </test-case>
   </test-group>
   <test-group name="inverted-index-join">
     <test-case FilePath="inverted-index-join">
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 260617a..f9c4f21 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
@@ -329,6 +329,7 @@
     NO_VALID_AUTHENTICATION_PARAMS_PROVIDED(1221),
     NO_VALID_AUTHENTICATION_PARAMS_PROVIDED_TO_IMPERSONATE_SERVICE_ACCOUNT(1222),
     INVALID_TRANSFORM_FUNCTION(1223),
+    CANNOT_REPLACE_OBJECT_DEPENDENT_EXISTS(1224),
 
     // 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 68c23c7..c760e55 100644
--- a/asterixdb/asterix-common/src/main/resources/asx_errormsg/en.properties
+++ b/asterixdb/asterix-common/src/main/resources/asx_errormsg/en.properties
@@ -331,6 +331,7 @@
 1221 = No valid authentication parameters were provided
 1222 = No valid authentication parameters were provided to impersonate service account
 1223 = Failed to create transform function. Encountered error: '%1$s'
+1224 = Cannot replace %1$s %2$s being used by %3$s %4$s
 
 # Feed Errors
 3001 = Illegal state.
diff --git a/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/declared/FunctionDataSource.java b/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/declared/FunctionDataSource.java
index 2322e53..02ffc25 100644
--- a/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/declared/FunctionDataSource.java
+++ b/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/declared/FunctionDataSource.java
@@ -127,9 +127,8 @@
             AlgebricksAbsolutePartitionConstraint locations);
 
     protected AlgebricksAbsolutePartitionConstraint getLocations(IClusterStateManager csm, MetadataProvider md) {
-        String[] sortedLocations = md.getDataPartitioningProvider().getClusterLocations().getLocations();
-        return new AlgebricksAbsolutePartitionConstraint(
-                Arrays.stream(sortedLocations).distinct().toArray(String[]::new));
+        String[] locations = md.getClusterLocations().getLocations();
+        return new AlgebricksAbsolutePartitionConstraint(Arrays.stream(locations).distinct().toArray(String[]::new));
     }
 
     protected IDataParserFactory createDataParserFactory() {
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 71f479f..941c8f7 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
@@ -25,7 +25,6 @@
 import java.io.IOException;
 import java.nio.charset.StandardCharsets;
 import java.util.ArrayList;
-import java.util.Arrays;
 import java.util.Collections;
 import java.util.HashMap;
 import java.util.HashSet;
@@ -1059,10 +1058,6 @@
         return dataPartitioningProvider.getClusterLocations();
     }
 
-    public DataPartitioningProvider getDataPartitioningProvider() {
-        return dataPartitioningProvider;
-    }
-
     public Pair<IOperatorDescriptor, AlgebricksPartitionConstraint> buildExternalDataLookupRuntime(
             JobSpecification jobSpec, Dataset dataset, int[] ridIndexes, boolean retainInput,
             IVariableTypeEnvironment typeEnv, IOperatorSchema opSchema, JobGenContext context,
@@ -1880,10 +1875,8 @@
         if (!(nodeDomain instanceof DefaultNodeGroupDomain inputDomain)) {
             return null;
         }
-        String[] inputLocations = inputDomain.getSortedNodes();
         AlgebricksAbsolutePartitionConstraint locations = dataPartitioningProvider.getClusterLocations();
-        String[] clusterLocations = locations.getLocations();
-        if (!Arrays.equals(inputLocations, clusterLocations)) {
+        if (!inputDomain.sameAs(new DefaultNodeGroupDomain(locations))) {
             return null;
         }
         return dataPartitioningProvider.getPartitionsMap();
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 cf4f8e0..7e1fd48 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
@@ -329,9 +329,10 @@
             throws AlgebricksException {
         IFunctionManager funManger = metadataProvider.getFunctionManager();
         IDataFormat dataFormat = metadataProvider.getDataFormat();
-        //if the target type is "BuiltinType.ANY" there is no need to cast. If not we have to cast.
         if (ATypeTag.ANY.equals(targetType.getTypeTag())) {
-            return fieldEvalFactory;
+            // this is to ensure records and lists values are in the open format
+            IScalarEvaluatorFactory[] castArg = new IScalarEvaluatorFactory[] { fieldEvalFactory };
+            return createCastFunction(targetType, BuiltinType.ANY, true, sourceLoc).createEvaluatorFactory(castArg);
         }
 
         // check IndexUtil.castDefaultNull(index), too, because we always want to cast even if the overriding type is
diff --git a/asterixdb/asterix-server/pom.xml b/asterixdb/asterix-server/pom.xml
index 8af400b..a20659c 100644
--- a/asterixdb/asterix-server/pom.xml
+++ b/asterixdb/asterix-server/pom.xml
@@ -1398,6 +1398,16 @@
       <groupId>org.apache.hyracks</groupId>
       <artifactId>hyracks-api</artifactId>
     </dependency>
+    <dependency>
+      <groupId>org.apache.iceberg</groupId>
+      <artifactId>iceberg-data</artifactId>
+      <scope>test</scope>
+    </dependency>
+    <dependency>
+      <groupId>org.apache.iceberg</groupId>
+      <artifactId>iceberg-parquet</artifactId>
+      <scope>test</scope>
+    </dependency>
   </dependencies>
   <!-- apply patch for HADOOP-17225 to workaround CVE-2019-10172 -->
   <repositories>
diff --git a/asterixdb/pom.xml b/asterixdb/pom.xml
index ae80656..5d80365 100644
--- a/asterixdb/pom.xml
+++ b/asterixdb/pom.xml
@@ -102,7 +102,7 @@
     <awsjavasdk.version>2.29.27</awsjavasdk.version>
     <awsjavasdk.crt.version>0.33.3</awsjavasdk.crt.version>
 
-    <parquet.version>1.14.3</parquet.version>
+    <parquet.version>1.15.2</parquet.version>
     <hadoop-awsjavasdk.version>1.12.779</hadoop-awsjavasdk.version>
     <azureblobjavasdk.version>12.25.1</azureblobjavasdk.version>
     <azurecommonjavasdk.version>12.24.1</azurecommonjavasdk.version>
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 20fc9c5..a74f100 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
@@ -201,6 +201,11 @@
 
     boolean isBlockingOperatorDisabled();
 
+    /**
+     * Returns the cluster partitions map when the input node domain is the same as the cluster. Otherwise, null.
+     * @param nodeDomain input node domain
+     * @return the cluster partitions map.
+     */
     int[][] getPartitionsMap(INodeDomain nodeDomain);
 
 }
diff --git a/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/properties/DefaultNodeGroupDomain.java b/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/properties/DefaultNodeGroupDomain.java
index 5921eea..f785fec 100644
--- a/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/properties/DefaultNodeGroupDomain.java
+++ b/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/properties/DefaultNodeGroupDomain.java
@@ -18,7 +18,6 @@
  */
 package org.apache.hyracks.algebricks.core.algebra.properties;
 
-import java.util.Arrays;
 import java.util.List;
 
 import org.apache.commons.collections4.MultiSet;
@@ -73,10 +72,4 @@
     public String[] getNodes() {
         return nodes.toArray(new String[0]);
     }
-
-    public String[] getSortedNodes() {
-        String[] sortedNodes = nodes.toArray(new String[0]);
-        Arrays.sort(sortedNodes);
-        return sortedNodes;
-    }
 }