[ASTERIXDB-1975][COMP] Joins using non-enforced secondary indexes

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

Details:
- Added support for non-enforced indexes in IntroduceJoinAccessMethodRule
- Added testcases

Change-Id: Ibc90ebe439a9b45b72ca549ba7ee9e1881221524
Reviewed-on: https://asterix-gerrit.ics.uci.edu/1869
Integration-Tests: Jenkins <jenkins@fulliautomatix.ics.uci.edu>
Tested-by: Jenkins <jenkins@fulliautomatix.ics.uci.edu>
BAD: Jenkins <jenkins@fulliautomatix.ics.uci.edu>
Reviewed-by: Yingyi Bu <buyingyi@gmail.com>
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 8df1756..bed5c7f 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
@@ -473,16 +473,15 @@
             OptimizableOperatorSubTree matchedSubTree, AccessMethodAnalysisContext analysisCtx)
             throws AlgebricksException {
         List<Index> indexCandidates = new ArrayList<>();
-        // Add an index to the candidates if one of the indexed fields is
-        // fieldName
+        // Add an index to the candidates if one of the indexed fields is fieldName
         for (Index index : datasetIndexes) {
             // Need to also verify the index is pending no op
             if (index.getKeyFieldNames().contains(fieldName) && index.getPendingOp() == MetadataUtil.PENDING_NO_OP) {
                 indexCandidates.add(index);
-                if (optFuncExpr.getFieldType(varIdx) == BuiltinType.AMISSING
-                        || optFuncExpr.getFieldType(varIdx) == BuiltinType.ANY) {
-                    optFuncExpr.setFieldType(varIdx,
-                            index.getKeyFieldTypes().get(index.getKeyFieldNames().indexOf(fieldName)));
+                boolean isFieldTypeUnknown = fieldType == BuiltinType.AMISSING || fieldType == BuiltinType.ANY;
+                if (isFieldTypeUnknown && (!index.isOverridingKeyFieldTypes() || index.isEnforced())) {
+                    IAType indexedType = index.getKeyFieldTypes().get(index.getKeyFieldNames().indexOf(fieldName));
+                    optFuncExpr.setFieldType(varIdx, indexedType);
                 }
                 analysisCtx.addIndexExpr(matchedSubTree.getDataset(), index, matchedFuncExprIndex, varIdx);
             }
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 b4b83e5..24bad15 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
@@ -44,6 +44,8 @@
 import org.apache.asterix.om.base.IAObject;
 import org.apache.asterix.om.constants.AsterixConstantValue;
 import org.apache.asterix.om.functions.BuiltinFunctions;
+import org.apache.asterix.om.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;
@@ -379,19 +381,21 @@
     }
 
     /**
-     * Returns the search key expression which feeds a secondary-index search. If we are optimizing a selection query then this method returns
-     * the a ConstantExpression from the first constant value in the optimizable function expression.
-     * If we are optimizing a join, then this method returns the VariableReferenceExpression that should feed the secondary index probe.
+     * Returns the search key expression which feeds a secondary-index search. If we are optimizing a selection query
+     * then this method returns the a ConstantExpression from the first constant value in the optimizable function
+     * expression.
+     * If we are optimizing a join, then this method returns the VariableReferenceExpression that should feed the
+     * secondary index probe.
      *
      * @throws AlgebricksException
      */
     public static Pair<ILogicalExpression, Boolean> createSearchKeyExpr(Index index, IOptimizableFuncExpr optFuncExpr,
-            OptimizableOperatorSubTree indexSubTree, OptimizableOperatorSubTree probeSubTree)
+            IAType indexedFieldType, OptimizableOperatorSubTree indexSubTree, OptimizableOperatorSubTree probeSubTree)
             throws AlgebricksException {
+
         if (probeSubTree == null) {
             // We are optimizing a selection query. Search key is a constant.
             // Type Checking and type promotion is done here
-            IAType fieldType = optFuncExpr.getFieldType(0);
 
             if (optFuncExpr.getNumConstantExpr() == 0) {
                 //We are looking at a selection case, but using two variables
@@ -400,69 +404,80 @@
                 return new Pair<>(new VariableReferenceExpression(optFuncExpr.getLogicalVar(1)), false);
             }
 
-            ILogicalExpression constantAtRuntimeExpression = null;
+            ILogicalExpression constantAtRuntimeExpression = optFuncExpr.getConstantExpr(0);
             AsterixConstantValue constantValue = null;
-            ATypeTag constantValueTag = null;
-
-            constantAtRuntimeExpression = optFuncExpr.getConstantExpr(0);
-
             if (constantAtRuntimeExpression.getExpressionTag() == LogicalExpressionTag.CONSTANT) {
                 constantValue = (AsterixConstantValue) ((ConstantExpression) constantAtRuntimeExpression).getValue();
             }
 
-            constantValueTag = optFuncExpr.getConstantType(0).getTypeTag();
-
-            // type casting applied?
-            boolean typeCastingApplied = false;
-            // type casting happened from real (FLOAT, DOUBLE) value -> INT value?
-            boolean realTypeConvertedToIntegerType = false;
-            AsterixConstantValue replacedConstantValue = null;
+            ATypeTag constantValueTag = optFuncExpr.getConstantType(0).getTypeTag();
+            ATypeTag indexedFieldTypeTag = TypeComputeUtils.getActualType(indexedFieldType).getTypeTag();
 
             // if the constant type and target type does not match, we do a type conversion
-            if (constantValueTag != fieldType.getTypeTag() && constantValue != null) {
+            AsterixConstantValue replacedConstantValue = null;
+            // type casting happened from real (FLOAT, DOUBLE) value -> INT value?
+            boolean realTypeConvertedToIntegerType = false;
+
+            if (constantValueTag != indexedFieldTypeTag && constantValue != null) {
                 try {
                     replacedConstantValue = ATypeHierarchy.getAsterixConstantValueFromNumericTypeObject(
-                            constantValue.getObject(), fieldType.getTypeTag(), index.isEnforced());
+                            constantValue.getObject(), indexedFieldTypeTag, index.isEnforced());
+                    realTypeConvertedToIntegerType =
+                            isRealTypeConvertedToIntegerType(constantValueTag, indexedFieldTypeTag);
                 } catch (HyracksDataException e) {
                     throw new AlgebricksException(e);
                 }
-                if (replacedConstantValue != null) {
-                    typeCastingApplied = true;
-                }
+            }
 
-                // To check whether the constant is REAL values, and target field is an INT type field.
-                // In this case, we need to change the search parameter. Refer to the caller section for the detail.
-                switch (constantValueTag) {
-                    case DOUBLE:
-                    case FLOAT:
-                        switch (fieldType.getTypeTag()) {
-                            case TINYINT:
-                            case SMALLINT:
-                            case INTEGER:
-                            case BIGINT:
-                                realTypeConvertedToIntegerType = true;
-                                break;
-                            default:
-                                break;
-                        }
+            return replacedConstantValue != null
+                    ? new Pair<>(new ConstantExpression(replacedConstantValue), realTypeConvertedToIntegerType)
+                    : new Pair<>(constantAtRuntimeExpression, false);
+        } else {
+            // We are optimizing a join query. Determine which variable feeds the secondary index.
+            OptimizableOperatorSubTree opSubTree0 = optFuncExpr.getOperatorSubTree(0);
+            int probeVarIndex = opSubTree0 == null || opSubTree0 == probeSubTree ? 0 : 1;
+            LogicalVariable probeVar = optFuncExpr.getLogicalVar(probeVarIndex);
+            ILogicalExpression probeExpr = new VariableReferenceExpression(probeVar);
+
+            ATypeTag indexedFieldTypeTag = TypeComputeUtils.getActualType(indexedFieldType).getTypeTag();
+            if (ATypeHierarchy.getTypeDomain(indexedFieldTypeTag) == ATypeHierarchy.Domain.NUMERIC) {
+                IAType probeType = TypeComputeUtils.getActualType(optFuncExpr.getFieldType(probeVarIndex));
+                ATypeTag probeTypeTypeTag = probeType.getTypeTag();
+                if (probeTypeTypeTag != indexedFieldTypeTag) {
+                    ScalarFunctionCallExpression castFunc =
+                            new ScalarFunctionCallExpression(FunctionUtil.getFunctionInfo(BuiltinFunctions.CAST_TYPE_LAX));
+                    castFunc.getArguments().add(new MutableObject<>(probeExpr));
+                    TypeCastUtils.setRequiredAndInputTypes(castFunc, indexedFieldType, probeType);
+                    boolean realTypeConvertedToIntegerType =
+                            isRealTypeConvertedToIntegerType(probeTypeTypeTag, indexedFieldTypeTag);
+                    return new Pair<>(castFunc, realTypeConvertedToIntegerType);
+                }
+            }
+
+            return new Pair<>(probeExpr, false);
+        }
+    }
+
+    private static boolean isRealTypeConvertedToIntegerType(ATypeTag probeTypeTag, ATypeTag indexedFieldTypeTag) {
+        // To check whether the constant is REAL values, and target field is an INT type field.
+        // In this case, we need to change the search parameter. Refer to the caller section for the detail.
+        switch (probeTypeTag) {
+            case DOUBLE:
+            case FLOAT:
+                switch (indexedFieldTypeTag) {
+                    case TINYINT:
+                    case SMALLINT:
+                    case INTEGER:
+                    case BIGINT:
+                        return true;
                     default:
                         break;
                 }
-            }
-
-            if (typeCastingApplied) {
-                return new Pair<>(new ConstantExpression(replacedConstantValue), realTypeConvertedToIntegerType);
-            } else {
-                return new Pair<>(optFuncExpr.getConstantExpr(0), false);
-            }
-        } else {
-            // We are optimizing a join query. Determine which variable feeds the secondary index.
-            if (optFuncExpr.getOperatorSubTree(0) == null || optFuncExpr.getOperatorSubTree(0) == probeSubTree) {
-                return new Pair<>(new VariableReferenceExpression(optFuncExpr.getLogicalVar(0)), false);
-            } else {
-                return new Pair<>(new VariableReferenceExpression(optFuncExpr.getLogicalVar(1)), false);
-            }
+                break;
+            default:
+                break;
         }
+        return false;
     }
 
     /**
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 189f238..ddc93f8 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
@@ -273,8 +273,9 @@
             if (keyPos < 0) {
                 throw CompilationException.create(ErrorCode.NO_INDEX_FIELD_NAME_FOR_GIVEN_FUNC_EXPR);
             }
-            Pair<ILogicalExpression, Boolean> returnedSearchKeyExpr =
-                    AccessMethodUtils.createSearchKeyExpr(chosenIndex, optFuncExpr, indexSubTree, probeSubTree);
+            IAType indexedFieldType = chosenIndex.getKeyFieldTypes().get(keyPos);
+            Pair<ILogicalExpression, Boolean> returnedSearchKeyExpr = AccessMethodUtils.createSearchKeyExpr(chosenIndex,
+                    optFuncExpr, indexedFieldType, indexSubTree, probeSubTree);
             ILogicalExpression searchKeyExpr = returnedSearchKeyExpr.first;
             if (searchKeyExpr.getExpressionTag() == LogicalExpressionTag.FUNCTION_CALL) {
                 constantAtRuntimeExpressions[keyPos] = searchKeyExpr;
@@ -469,15 +470,21 @@
         jobGenParams.setLowKeyVarList(keyVarList, 0, numLowKeys);
         jobGenParams.setHighKeyVarList(keyVarList, numLowKeys, numHighKeys);
 
-        ILogicalOperator inputOp = null;
+        ILogicalOperator inputOp;
         if (!assignKeyVarList.isEmpty()) {
             // Assign operator that sets the constant secondary-index search-key fields if necessary.
-            AssignOperator assignConstantSearchKeys = new AssignOperator(assignKeyVarList, assignKeyExprList);
-            // Input to this assign is the EmptyTupleSource (which the dataSourceScan also must have had as input).
-            assignConstantSearchKeys.getInputs().add(new MutableObject<>(
-                    OperatorManipulationUtil.deepCopy(dataSourceOp.getInputs().get(0).getValue())));
-            assignConstantSearchKeys.setExecutionMode(dataSourceOp.getExecutionMode());
-            inputOp = assignConstantSearchKeys;
+            AssignOperator assignSearchKeys = new AssignOperator(assignKeyVarList, assignKeyExprList);
+            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).
+                assignSearchKeys.getInputs().add(new MutableObject<>(
+                        OperatorManipulationUtil.deepCopy(dataSourceOp.getInputs().get(0).getValue())));
+                assignSearchKeys.setExecutionMode(dataSourceOp.getExecutionMode());
+            } else {
+                // We are optimizing a join, place the assign op top of the probe subtree.
+                assignSearchKeys.getInputs().add(probeSubTree.getRootRef());
+            }
+            inputOp = assignSearchKeys;
         } else if (probeSubTree == null) {
             //nonpure case
             //Make sure that the nonpure function is unpartitioned
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 d16eb18..7fc7902 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
@@ -192,25 +192,6 @@
     }
 
     /**
-     * Removes non-enforced indexes on open types
-     */
-    protected void pruneIndexCandidatesOpenNonEnforced(Map<IAccessMethod, AccessMethodAnalysisContext> analyzedAMs) {
-        for (Map.Entry<IAccessMethod, AccessMethodAnalysisContext> entry : analyzedAMs.entrySet()) {
-            AccessMethodAnalysisContext amCtx = entry.getValue();
-
-            // Fetch index
-            Iterator<Map.Entry<Index, List<Pair<Integer, Integer>>>> indexIt = amCtx.getIteratorForIndexExprsAndVars();
-
-            while (indexIt.hasNext()) {
-                Index index = indexIt.next().getKey();
-                if (index.isOverridingKeyFieldTypes() && !index.isEnforced()) {
-                    indexIt.remove();
-                }
-            }
-        }
-    }
-
-    /**
      * Checks whether the given operator is LEFTOUTERJOIN.
      * If so, also checks that GROUPBY is placed after LEFTOUTERJOIN.
      */
@@ -366,9 +347,6 @@
                 // Remove the indexes from the outer branch in the optimizer's consideration list for this rule.
                 pruneIndexCandidatesFromOuterBranch(analyzedAMs);
 
-                // Remove non-enforced indexes on open types as they currently can't be used with joins
-                pruneIndexCandidatesOpenNonEnforced(analyzedAMs);
-
                 // We are going to use indexes from the inner branch.
                 // If no index is available, then we stop here.
                 Pair<IAccessMethod, Index> chosenIndex = chooseBestIndex(analyzedAMs);
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 4d219a7..b66a663 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
@@ -184,8 +184,9 @@
         ARecordType metaRecordType = indexSubTree.getMetaRecordType();
 
         int optFieldIdx = AccessMethodUtils.chooseFirstOptFuncVar(chosenIndex, analysisCtx);
-        Pair<IAType, Boolean> keyPairType = Index.getNonNullableOpenFieldType(optFuncExpr.getFieldType(optFieldIdx),
-                optFuncExpr.getFieldName(optFieldIdx), recordType);
+        IAType optFieldType = optFuncExpr.getFieldType(optFieldIdx);
+        List<String> optFieldName = optFuncExpr.getFieldName(optFieldIdx);
+        Pair<IAType, Boolean> keyPairType = Index.getNonNullableOpenFieldType(optFieldType, optFieldName, recordType);
         if (keyPairType == null) {
             return null;
         }
@@ -207,8 +208,8 @@
         ArrayList<LogicalVariable> keyVarList = new ArrayList<>();
         // List of expressions for the assign.
         ArrayList<Mutable<ILogicalExpression>> keyExprList = new ArrayList<>();
-        Pair<ILogicalExpression, Boolean> returnedSearchKeyExpr =
-                AccessMethodUtils.createSearchKeyExpr(chosenIndex, optFuncExpr, indexSubTree, probeSubTree);
+        Pair<ILogicalExpression, Boolean> returnedSearchKeyExpr = AccessMethodUtils.createSearchKeyExpr(chosenIndex,
+                optFuncExpr, optFieldType, indexSubTree, probeSubTree);
         ILogicalExpression searchKeyExpr = returnedSearchKeyExpr.first;
 
         for (int i = 0; i < numSecondaryKeys; i++) {
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/open-index-non-enforced/btree-equi-join-non-enforced/btree-equi-join-non-enforced-05.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/open-index-non-enforced/btree-equi-join-non-enforced/btree-equi-join-non-enforced-05.sqlpp
new file mode 100644
index 0000000..175fa08
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/open-index-non-enforced/btree-equi-join-non-enforced/btree-equi-join-non-enforced-05.sqlpp
@@ -0,0 +1,52 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT 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  : No index join because there's no hint and
+ *                the probe type is unknown
+ * Expected Res : Success
+ * Date         : 20 Jun 2017
+ */
+drop dataverse test if exists;
+create dataverse test;
+use test;
+
+write output to asterix_nc1:"rttest/btree-equi-join-non-enforced_btree-equi-join-non-enforced-05.adm"
+
+create type TestOpenType as open {
+  c_id: int64
+}
+
+create dataset TestOpen1(TestOpenType)
+primary key c_id;
+
+create dataset TestOpen2(TestOpenType)
+primary key c_id;
+
+create index idx_t2_s on TestOpen2(c_s:string);
+
+create index idx_t2_i64 on TestOpen2(c_i64:int64);
+
+create index idx_t2_i8 on TestOpen2(c_i8:int8);
+
+create index idx_t2_d on TestOpen2(c_d:double);
+
+select t1.c_x as c1, t2.c_x as c2
+from  TestOpen1 as t1, TestOpen2 as t2
+where t1.c_s = t2.c_s
+order by t1.c_x, t2.c_x;
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/open-index-non-enforced/btree-equi-join-non-enforced/btree-equi-join-non-enforced-06.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/open-index-non-enforced/btree-equi-join-non-enforced/btree-equi-join-non-enforced-06.sqlpp
new file mode 100644
index 0000000..edff5ce
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/open-index-non-enforced/btree-equi-join-non-enforced/btree-equi-join-non-enforced-06.sqlpp
@@ -0,0 +1,51 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+/*
+ * Description  : No index join because the probe type is unknown
+ * Expected Res : Success
+ * Date         : 20 Jun 2017
+ */
+drop dataverse test if exists;
+create dataverse test;
+use test;
+
+write output to asterix_nc1:"rttest/btree-equi-join-non-enforced_btree-equi-join-non-enforced-06.adm"
+
+create type TestOpenType as open {
+  c_id: int64
+}
+
+create dataset TestOpen1(TestOpenType)
+primary key c_id;
+
+create dataset TestOpen2(TestOpenType)
+primary key c_id;
+
+create index idx_t2_s on TestOpen2(c_s:string);
+
+create index idx_t2_i64 on TestOpen2(c_i64:int64);
+
+create index idx_t2_i8 on TestOpen2(c_i8:int8);
+
+create index idx_t2_d on TestOpen2(c_d:double);
+
+select t1.c_x as c1, t2.c_x as c2
+from  TestOpen1 as t1, TestOpen2 as t2
+where t1.c_s /*+ indexnl */ = t2.c_s
+order by t1.c_x, t2.c_x;
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/open-index-non-enforced/btree-equi-join-non-enforced/btree-equi-join-non-enforced-07.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/open-index-non-enforced/btree-equi-join-non-enforced/btree-equi-join-non-enforced-07.sqlpp
new file mode 100644
index 0000000..32f4c5a
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/open-index-non-enforced/btree-equi-join-non-enforced/btree-equi-join-non-enforced-07.sqlpp
@@ -0,0 +1,51 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+/*
+ * Description  : No index join because there's no hint
+ * Expected Res : Success
+ * Date         : 20 Jun 2017
+ */
+drop dataverse test if exists;
+create dataverse test;
+use test;
+
+write output to asterix_nc1:"rttest/btree-equi-join-non-enforced_btree-equi-join-non-enforced-07.adm"
+
+create type TestOpenType as open {
+  c_id: int64
+}
+
+create dataset TestOpen1(TestOpenType)
+primary key c_id;
+
+create dataset TestOpen2(TestOpenType)
+primary key c_id;
+
+create index idx_t2_s on TestOpen2(c_s:string);
+
+create index idx_t2_i64 on TestOpen2(c_i64:int64);
+
+create index idx_t2_i8 on TestOpen2(c_i8:int8);
+
+create index idx_t2_d on TestOpen2(c_d:double);
+
+select t1.c_x as c1, t2.c_x as c2
+from TestOpen1 as t1, TestOpen2 as t2
+where to_string(t1.c_s) = t2.c_s
+order by t1.c_x, t2.c_x;
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/open-index-non-enforced/btree-equi-join-non-enforced/btree-equi-join-non-enforced-08.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/open-index-non-enforced/btree-equi-join-non-enforced/btree-equi-join-non-enforced-08.sqlpp
new file mode 100644
index 0000000..aef434a
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/open-index-non-enforced/btree-equi-join-non-enforced/btree-equi-join-non-enforced-08.sqlpp
@@ -0,0 +1,52 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT 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  : Index join because there's a hint and
+ *                the probe type is known (string)
+ * Expected Res : Success
+ * Date         : 20 Jun 2017
+ */
+drop dataverse test if exists;
+create dataverse test;
+use test;
+
+write output to asterix_nc1:"rttest/btree-equi-join-non-enforced_btree-equi-join-non-enforced-08.adm"
+
+create type TestOpenType as open {
+  c_id: int64
+}
+
+create dataset TestOpen1(TestOpenType)
+primary key c_id;
+
+create dataset TestOpen2(TestOpenType)
+primary key c_id;
+
+create index idx_t2_s on TestOpen2(c_s:string);
+
+create index idx_t2_i64 on TestOpen2(c_i64:int64);
+
+create index idx_t2_i8 on TestOpen2(c_i8:int8);
+
+create index idx_t2_d on TestOpen2(c_d:double);
+
+select t1.c_x as c1, t2.c_x as c2
+from TestOpen1 as t1, TestOpen2 as t2
+where to_string(t1.c_s) /*+ indexnl */ = t2.c_s
+order by t1.c_x, t2.c_x;
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/open-index-non-enforced/btree-equi-join-non-enforced/btree-equi-join-non-enforced-09.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/open-index-non-enforced/btree-equi-join-non-enforced/btree-equi-join-non-enforced-09.sqlpp
new file mode 100644
index 0000000..a8b24b1
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/open-index-non-enforced/btree-equi-join-non-enforced/btree-equi-join-non-enforced-09.sqlpp
@@ -0,0 +1,52 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT 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  : Index join because there's a hint and
+ *                the probe type is known (bigint)
+ * Expected Res : Success
+ * Date         : 20 Jun 2017
+ */
+drop dataverse test if exists;
+create dataverse test;
+use test;
+
+write output to asterix_nc1:"rttest/btree-equi-join-non-enforced_btree-equi-join-non-enforced-08.adm"
+
+create type TestOpenType as open {
+  c_id: int64
+}
+
+create dataset TestOpen1(TestOpenType)
+primary key c_id;
+
+create dataset TestOpen2(TestOpenType)
+primary key c_id;
+
+create index idx_t2_s on TestOpen2(c_s:string);
+
+create index idx_t2_i64 on TestOpen2(c_i64:int64);
+
+create index idx_t2_i8 on TestOpen2(c_i8:int8);
+
+create index idx_t2_d on TestOpen2(c_d:double);
+
+select t1.c_x as c1, t2.c_x as c2
+from TestOpen1 as t1, TestOpen2 as t2
+where to_bigint(t1.c_i64) /*+ indexnl */ = t2.c_i64
+order by t1.c_x, t2.c_x;
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/open-index-non-enforced/btree-index-non-enforced/btree-index-non-enforced-105.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/open-index-non-enforced/btree-index-non-enforced/btree-index-non-enforced-105.sqlpp
new file mode 100644
index 0000000..0ef985e
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/open-index-non-enforced/btree-index-non-enforced/btree-index-non-enforced-105.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.
+ */
+ /*
+  *  Description     : BTree non-enforced index verification test
+  *                  : Test that the secondary non-enforced BTree index is used
+  *                  : when multiple indexes match
+  *  Expected Result : Success
+  *  Date            : 29 Jun 2017
+  */
+
+drop dataverse test if exists;
+create dataverse test;
+use test;
+
+write output to asterix_nc1:"rttest/btree-index-non-enforced_btree-index-non-enforced-104.adm"
+
+create type TestOpenType as open {
+  c_id: int64
+}
+
+create dataset TestOpen(TestOpenType)
+primary key c_id;
+
+create index idx_3 on TestOpen(c_i64:int64);
+
+create index idx_4 on TestOpen(c_i8:int8);
+
+select t.c_x as res
+from TestOpen t
+where t.c_i8 > 2 and t.c_i64 < 3
+order by t.c_x;
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/open-index-non-enforced/btree-equi-join-non-enforced/btree-equi-join-non-enforced-05.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/open-index-non-enforced/btree-equi-join-non-enforced/btree-equi-join-non-enforced-05.plan
new file mode 100644
index 0000000..15f9171
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/open-index-non-enforced/btree-equi-join-non-enforced/btree-equi-join-non-enforced-05.plan
@@ -0,0 +1,26 @@
+-- DISTRIBUTE_RESULT  |PARTITIONED|
+  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+    -- STREAM_PROJECT  |PARTITIONED|
+      -- ASSIGN  |PARTITIONED|
+        -- SORT_MERGE_EXCHANGE [$$18(ASC), $$19(ASC) ]  |PARTITIONED|
+          -- STABLE_SORT [$$18(ASC), $$19(ASC)]  |PARTITIONED|
+            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+              -- STREAM_PROJECT  |PARTITIONED|
+                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                  -- HYBRID_HASH_JOIN [$$16][$$17]  |PARTITIONED|
+                    -- HASH_PARTITION_EXCHANGE [$$16]  |PARTITIONED|
+                      -- STREAM_PROJECT  |PARTITIONED|
+                        -- ASSIGN  |PARTITIONED|
+                          -- STREAM_PROJECT  |PARTITIONED|
+                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                              -- DATASOURCE_SCAN  |PARTITIONED|
+                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                  -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                    -- HASH_PARTITION_EXCHANGE [$$17]  |PARTITIONED|
+                      -- STREAM_PROJECT  |PARTITIONED|
+                        -- ASSIGN  |PARTITIONED|
+                          -- STREAM_PROJECT  |PARTITIONED|
+                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                              -- DATASOURCE_SCAN  |PARTITIONED|
+                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                  -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/open-index-non-enforced/btree-equi-join-non-enforced/btree-equi-join-non-enforced-06.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/open-index-non-enforced/btree-equi-join-non-enforced/btree-equi-join-non-enforced-06.plan
new file mode 100644
index 0000000..15f9171
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/open-index-non-enforced/btree-equi-join-non-enforced/btree-equi-join-non-enforced-06.plan
@@ -0,0 +1,26 @@
+-- DISTRIBUTE_RESULT  |PARTITIONED|
+  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+    -- STREAM_PROJECT  |PARTITIONED|
+      -- ASSIGN  |PARTITIONED|
+        -- SORT_MERGE_EXCHANGE [$$18(ASC), $$19(ASC) ]  |PARTITIONED|
+          -- STABLE_SORT [$$18(ASC), $$19(ASC)]  |PARTITIONED|
+            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+              -- STREAM_PROJECT  |PARTITIONED|
+                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                  -- HYBRID_HASH_JOIN [$$16][$$17]  |PARTITIONED|
+                    -- HASH_PARTITION_EXCHANGE [$$16]  |PARTITIONED|
+                      -- STREAM_PROJECT  |PARTITIONED|
+                        -- ASSIGN  |PARTITIONED|
+                          -- STREAM_PROJECT  |PARTITIONED|
+                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                              -- DATASOURCE_SCAN  |PARTITIONED|
+                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                  -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                    -- HASH_PARTITION_EXCHANGE [$$17]  |PARTITIONED|
+                      -- STREAM_PROJECT  |PARTITIONED|
+                        -- ASSIGN  |PARTITIONED|
+                          -- STREAM_PROJECT  |PARTITIONED|
+                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                              -- DATASOURCE_SCAN  |PARTITIONED|
+                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                  -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/open-index-non-enforced/btree-equi-join-non-enforced/btree-equi-join-non-enforced-07.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/open-index-non-enforced/btree-equi-join-non-enforced/btree-equi-join-non-enforced-07.plan
new file mode 100644
index 0000000..7f5a841
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/open-index-non-enforced/btree-equi-join-non-enforced/btree-equi-join-non-enforced-07.plan
@@ -0,0 +1,26 @@
+-- DISTRIBUTE_RESULT  |PARTITIONED|
+  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+    -- STREAM_PROJECT  |PARTITIONED|
+      -- ASSIGN  |PARTITIONED|
+        -- SORT_MERGE_EXCHANGE [$$20(ASC), $$21(ASC) ]  |PARTITIONED|
+          -- STABLE_SORT [$$20(ASC), $$21(ASC)]  |PARTITIONED|
+            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+              -- STREAM_PROJECT  |PARTITIONED|
+                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                  -- HYBRID_HASH_JOIN [$$17][$$18]  |PARTITIONED|
+                    -- HASH_PARTITION_EXCHANGE [$$17]  |PARTITIONED|
+                      -- STREAM_PROJECT  |PARTITIONED|
+                        -- ASSIGN  |PARTITIONED|
+                          -- STREAM_PROJECT  |PARTITIONED|
+                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                              -- DATASOURCE_SCAN  |PARTITIONED|
+                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                  -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                    -- HASH_PARTITION_EXCHANGE [$$18]  |PARTITIONED|
+                      -- STREAM_PROJECT  |PARTITIONED|
+                        -- ASSIGN  |PARTITIONED|
+                          -- STREAM_PROJECT  |PARTITIONED|
+                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                              -- DATASOURCE_SCAN  |PARTITIONED|
+                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                  -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/open-index-non-enforced/btree-equi-join-non-enforced/btree-equi-join-non-enforced-08.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/open-index-non-enforced/btree-equi-join-non-enforced/btree-equi-join-non-enforced-08.plan
new file mode 100644
index 0000000..06483e4
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/open-index-non-enforced/btree-equi-join-non-enforced/btree-equi-join-non-enforced-08.plan
@@ -0,0 +1,27 @@
+-- DISTRIBUTE_RESULT  |PARTITIONED|
+  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+    -- STREAM_PROJECT  |PARTITIONED|
+      -- ASSIGN  |PARTITIONED|
+        -- SORT_MERGE_EXCHANGE [$$20(ASC), $$21(ASC) ]  |PARTITIONED|
+          -- STABLE_SORT [$$20(ASC), $$21(ASC)]  |PARTITIONED|
+            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+              -- STREAM_PROJECT  |PARTITIONED|
+                -- STREAM_SELECT  |PARTITIONED|
+                  -- ASSIGN  |PARTITIONED|
+                    -- STREAM_PROJECT  |PARTITIONED|
+                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                        -- BTREE_SEARCH  |PARTITIONED|
+                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                            -- STABLE_SORT [$$25(ASC)]  |PARTITIONED|
+                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                -- STREAM_PROJECT  |PARTITIONED|
+                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                    -- BTREE_SEARCH  |PARTITIONED|
+                                      -- BROADCAST_EXCHANGE  |PARTITIONED|
+                                        -- STREAM_PROJECT  |PARTITIONED|
+                                          -- ASSIGN  |PARTITIONED|
+                                            -- STREAM_PROJECT  |PARTITIONED|
+                                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                -- DATASOURCE_SCAN  |PARTITIONED|
+                                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                    -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/open-index-non-enforced/btree-equi-join-non-enforced/btree-equi-join-non-enforced-09.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/open-index-non-enforced/btree-equi-join-non-enforced/btree-equi-join-non-enforced-09.plan
new file mode 100644
index 0000000..06483e4
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/open-index-non-enforced/btree-equi-join-non-enforced/btree-equi-join-non-enforced-09.plan
@@ -0,0 +1,27 @@
+-- DISTRIBUTE_RESULT  |PARTITIONED|
+  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+    -- STREAM_PROJECT  |PARTITIONED|
+      -- ASSIGN  |PARTITIONED|
+        -- SORT_MERGE_EXCHANGE [$$20(ASC), $$21(ASC) ]  |PARTITIONED|
+          -- STABLE_SORT [$$20(ASC), $$21(ASC)]  |PARTITIONED|
+            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+              -- STREAM_PROJECT  |PARTITIONED|
+                -- STREAM_SELECT  |PARTITIONED|
+                  -- ASSIGN  |PARTITIONED|
+                    -- STREAM_PROJECT  |PARTITIONED|
+                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                        -- BTREE_SEARCH  |PARTITIONED|
+                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                            -- STABLE_SORT [$$25(ASC)]  |PARTITIONED|
+                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                -- STREAM_PROJECT  |PARTITIONED|
+                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                    -- BTREE_SEARCH  |PARTITIONED|
+                                      -- BROADCAST_EXCHANGE  |PARTITIONED|
+                                        -- STREAM_PROJECT  |PARTITIONED|
+                                          -- ASSIGN  |PARTITIONED|
+                                            -- STREAM_PROJECT  |PARTITIONED|
+                                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                -- DATASOURCE_SCAN  |PARTITIONED|
+                                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                    -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/open-index-non-enforced/btree-index-non-enforced/btree-index-non-enforced-105.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/open-index-non-enforced/btree-index-non-enforced/btree-index-non-enforced-105.plan
new file mode 100644
index 0000000..aa6a642
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/open-index-non-enforced/btree-index-non-enforced/btree-index-non-enforced-105.plan
@@ -0,0 +1,33 @@
+-- DISTRIBUTE_RESULT  |PARTITIONED|
+  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+    -- STREAM_PROJECT  |PARTITIONED|
+      -- ASSIGN  |PARTITIONED|
+        -- SORT_MERGE_EXCHANGE [$$15(ASC) ]  |PARTITIONED|
+          -- STABLE_SORT [$$15(ASC)]  |PARTITIONED|
+            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+              -- STREAM_PROJECT  |PARTITIONED|
+                -- STREAM_SELECT  |PARTITIONED|
+                  -- ASSIGN  |PARTITIONED|
+                    -- STREAM_PROJECT  |PARTITIONED|
+                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                        -- BTREE_SEARCH  |PARTITIONED|
+                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                            -- INTERSECT  |PARTITIONED|
+                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                -- STABLE_SORT [$$19(ASC)]  |PARTITIONED|
+                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                    -- STREAM_PROJECT  |PARTITIONED|
+                                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                        -- BTREE_SEARCH  |PARTITIONED|
+                                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                            -- ASSIGN  |PARTITIONED|
+                                              -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                -- STABLE_SORT [$$22(ASC)]  |PARTITIONED|
+                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                    -- STREAM_PROJECT  |PARTITIONED|
+                                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                        -- BTREE_SEARCH  |PARTITIONED|
+                                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                            -- ASSIGN  |PARTITIONED|
+                                              -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/open-index-non-enforced/index-join/btree-equi-join-01/btree-equi-join-01.1.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/open-index-non-enforced/index-join/btree-equi-join-01/btree-equi-join-01.1.ddl.sqlpp
index a957c0b..3edf47b 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/open-index-non-enforced/index-join/btree-equi-join-01/btree-equi-join-01.1.ddl.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/open-index-non-enforced/index-join/btree-equi-join-01/btree-equi-join-01.1.ddl.sqlpp
@@ -17,9 +17,9 @@
  * under the License.
  */
 /*
-* Description  : Index join is currently disabled for non-enforced indexes
+* Description  : Index join for non-enforced indexes
 * Expected Res : Success
-* Date         : 7 Jun 2017
+* Date         : 20 Jun 2017
 */
 drop dataverse test if exists;
 create dataverse test;
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/open-index-non-enforced/index-join/btree-equi-join-01/btree-equi-join-01.10.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/open-index-non-enforced/index-join/btree-equi-join-01/btree-equi-join-01.10.query.sqlpp
new file mode 100644
index 0000000..0df3732
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/open-index-non-enforced/index-join/btree-equi-join-01/btree-equi-join-01.10.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.
+ *
+ * Description  : Index join because there's a hint and
+ *                the probe type is known.
+ *                Promotable numeric type.
+ */
+
+use test;
+
+select t1.c_x as c1, t2.c_x as c2
+from  TestOpen1 as t1, TestOpen2 as t2
+where to_double(t1.c_i8) /*+ indexnl */ = t2.c_i8
+order by t1.c_x, t2.c_x;
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/open-index-non-enforced/index-join/btree-equi-join-01/btree-equi-join-01.5.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/open-index-non-enforced/index-join/btree-equi-join-01/btree-equi-join-01.5.query.sqlpp
index 5f0ae38..d0ca206 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/open-index-non-enforced/index-join/btree-equi-join-01/btree-equi-join-01.5.query.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/open-index-non-enforced/index-join/btree-equi-join-01/btree-equi-join-01.5.query.sqlpp
@@ -16,11 +16,12 @@
  * specific language governing permissions and limitations
  * under the License.
  *
+ * Description  : No index join because there's no hint
  */
 
 use test;
 
 select t1.c_x as c1, t2.c_x as c2
 from  TestOpen1 as t1, TestOpen2 as t2
-where (t1.c_s /*+ indexnl */ = t2.c_s)
+where t1.c_s = t2.c_s
 order by t1.c_x, t2.c_x;
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/open-index-non-enforced/index-join/btree-equi-join-01/btree-equi-join-01.6.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/open-index-non-enforced/index-join/btree-equi-join-01/btree-equi-join-01.6.query.sqlpp
new file mode 100644
index 0000000..7b8a44b
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/open-index-non-enforced/index-join/btree-equi-join-01/btree-equi-join-01.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.
+ *
+ * Description  : No index join because the probe type is unknown
+ */
+
+use test;
+
+select t1.c_x as c1, t2.c_x as c2
+from  TestOpen1 as t1, TestOpen2 as t2
+where t1.c_s /*+ indexnl */ = t2.c_s
+order by t1.c_x, t2.c_x;
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/open-index-non-enforced/index-join/btree-equi-join-01/btree-equi-join-01.7.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/open-index-non-enforced/index-join/btree-equi-join-01/btree-equi-join-01.7.query.sqlpp
new file mode 100644
index 0000000..5b6de4b
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/open-index-non-enforced/index-join/btree-equi-join-01/btree-equi-join-01.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.
+ *
+ * Description  : No index join because there's no hint
+ */
+
+use test;
+
+select t1.c_x as c1, t2.c_x as c2
+from  TestOpen1 as t1, TestOpen2 as t2
+where to_string(t1.c_s) = t2.c_s
+order by t1.c_x, t2.c_x;
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/open-index-non-enforced/index-join/btree-equi-join-01/btree-equi-join-01.8.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/open-index-non-enforced/index-join/btree-equi-join-01/btree-equi-join-01.8.query.sqlpp
new file mode 100644
index 0000000..ed9ddff
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/open-index-non-enforced/index-join/btree-equi-join-01/btree-equi-join-01.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.
+ *
+ * Description  : Index join because there's a hint and
+ *                the probe type is known (string)
+ */
+
+use test;
+
+select t1.c_x as c1, t2.c_x as c2
+from  TestOpen1 as t1, TestOpen2 as t2
+where to_string(t1.c_s) /*+ indexnl */ = t2.c_s
+order by t1.c_x, t2.c_x;
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/open-index-non-enforced/index-join/btree-equi-join-01/btree-equi-join-01.9.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/open-index-non-enforced/index-join/btree-equi-join-01/btree-equi-join-01.9.query.sqlpp
new file mode 100644
index 0000000..8c501e6
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/open-index-non-enforced/index-join/btree-equi-join-01/btree-equi-join-01.9.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.
+ *
+ * Description  : Index join because there's a hint and
+ *                the probe type is known (bigint)
+ */
+
+use test;
+
+select t1.c_x as c1, t2.c_x as c2
+from  TestOpen1 as t1, TestOpen2 as t2
+where to_bigint(t1.c_i64) /*+ indexnl */ = t2.c_i64
+order by t1.c_x, t2.c_x;
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/open-index-non-enforced/index-selection/btree-index-02/btree-index-02.1.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/open-index-non-enforced/index-selection/btree-index-02/btree-index-02.1.ddl.sqlpp
new file mode 100644
index 0000000..68e0eb4
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/open-index-non-enforced/index-selection/btree-index-02/btree-index-02.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  : Index selection for non-enforced indexes
+*                Multiple indexes match.
+* Expected Res : Success
+* Date         : 29 Jun 2017
+*/
+drop dataverse test if exists;
+create dataverse test;
+use test;
+
+create type TestOpenType as open {
+  c_id: int64
+}
+
+create dataset TestOpen(TestOpenType)
+primary key c_id;
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/open-index-non-enforced/index-selection/btree-index-02/btree-index-02.2.update.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/open-index-non-enforced/index-selection/btree-index-02/btree-index-02.2.update.sqlpp
new file mode 100644
index 0000000..eba8642
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/open-index-non-enforced/index-selection/btree-index-02/btree-index-02.2.update.sqlpp
@@ -0,0 +1,79 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+use test;
+
+insert into TestOpen ({
+    "c_id": 1,
+    "c_x": 1,
+    "c_s": "hello",
+    "c_i64": 2,
+    "c_i8": 2,
+    "c_d": 2
+});
+insert into TestOpen ({
+    "c_id": 2,
+    "c_x": 2,
+    "c_s": 2,
+    "c_i64": "2",
+    "c_i8": 2.5,
+    "c_d": 3
+});
+insert into TestOpen ({
+    "c_id": 3,
+    "c_x": 3,
+    "c_s": "world",
+    "c_i64": 2,
+    "c_i8": 4,
+    "c_d": 3.125
+});
+insert into TestOpen ({
+    "c_id": 4,
+    "c_x": 4,
+    "c_s": null,
+    "c_i64": null,
+    "c_i8": 500,
+    "c_d": 3.25
+});
+insert into TestOpen ({
+    "c_id": 5,
+    "c_x": 5,
+    "c_s": "hello",
+    "c_i64": 2.25,
+    "c_i8": 10000.25,
+    "c_d": 3.5
+});
+insert into TestOpen ({
+    "c_id": 6,
+    "c_x": 6,
+    "c_s": false,
+    "c_i64": false,
+    "c_i8": 2e100,
+    "c_d": 2e100
+});
+insert into TestOpen ({
+    "c_id": 7,
+    "c_x": 7,
+    "c_s": "world",
+    "c_i64": 3
+});
+insert into TestOpen ({
+    "c_id": 8,
+    "c_x": 8
+});
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/open-index-non-enforced/index-selection/btree-index-02/btree-index-02.3.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/open-index-non-enforced/index-selection/btree-index-02/btree-index-02.3.ddl.sqlpp
new file mode 100644
index 0000000..ecf3da2
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/open-index-non-enforced/index-selection/btree-index-02/btree-index-02.3.ddl.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.
+ */
+
+use test;
+
+create index idx_2 on TestOpen(c_s:string, c_i64:int64);
+
+create index idx_1 on TestOpen(c_s:string, c_d:double);
+
+create index idx_3 on TestOpen(c_i64:int64);
+
+create index idx_4 on TestOpen(c_i8:int8);
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/open-index-non-enforced/index-selection/btree-index-02/btree-index-02.4.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/open-index-non-enforced/index-selection/btree-index-02/btree-index-02.4.query.sqlpp
new file mode 100644
index 0000000..8b33f18
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/open-index-non-enforced/index-selection/btree-index-02/btree-index-02.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.
+ */
+
+use test;
+
+select t.c_x as res
+from TestOpen t
+where t.c_s = 'world'
+order by t.c_x;
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/open-index-non-enforced/index-selection/btree-index-02/btree-index-02.5.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/open-index-non-enforced/index-selection/btree-index-02/btree-index-02.5.query.sqlpp
new file mode 100644
index 0000000..d8fb0a8
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/open-index-non-enforced/index-selection/btree-index-02/btree-index-02.5.query.sqlpp
@@ -0,0 +1,25 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+use test;
+
+select t.c_x as res
+from TestOpen t
+where t.c_i8 > 2 and t.c_i64 < 3
+order by t.c_x;
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/open-index-non-enforced/index-join/btree-equi-join-01/btree-equi-join-01.10.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/open-index-non-enforced/index-join/btree-equi-join-01/btree-equi-join-01.10.adm
new file mode 100644
index 0000000..c45a808
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/open-index-non-enforced/index-join/btree-equi-join-01/btree-equi-join-01.10.adm
@@ -0,0 +1,6 @@
+{ "c1": 1, "c2": 101 }
+{ "c1": 2, "c2": 102 }
+{ "c1": 3, "c2": 103 }
+{ "c1": 4, "c2": 104 }
+{ "c1": 5, "c2": 105 }
+{ "c1": 6, "c2": 106 }
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/open-index-non-enforced/index-join/btree-equi-join-01/btree-equi-join-01.6.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/open-index-non-enforced/index-join/btree-equi-join-01/btree-equi-join-01.6.adm
new file mode 100644
index 0000000..8643d5f
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/open-index-non-enforced/index-join/btree-equi-join-01/btree-equi-join-01.6.adm
@@ -0,0 +1,10 @@
+{ "c1": 1, "c2": 101 }
+{ "c1": 1, "c2": 105 }
+{ "c1": 2, "c2": 102 }
+{ "c1": 3, "c2": 103 }
+{ "c1": 3, "c2": 107 }
+{ "c1": 5, "c2": 101 }
+{ "c1": 5, "c2": 105 }
+{ "c1": 6, "c2": 106 }
+{ "c1": 7, "c2": 103 }
+{ "c1": 7, "c2": 107 }
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/open-index-non-enforced/index-join/btree-equi-join-01/btree-equi-join-01.7.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/open-index-non-enforced/index-join/btree-equi-join-01/btree-equi-join-01.7.adm
new file mode 100644
index 0000000..78a404d
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/open-index-non-enforced/index-join/btree-equi-join-01/btree-equi-join-01.7.adm
@@ -0,0 +1,8 @@
+{ "c1": 1, "c2": 101 }
+{ "c1": 1, "c2": 105 }
+{ "c1": 3, "c2": 103 }
+{ "c1": 3, "c2": 107 }
+{ "c1": 5, "c2": 101 }
+{ "c1": 5, "c2": 105 }
+{ "c1": 7, "c2": 103 }
+{ "c1": 7, "c2": 107 }
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/open-index-non-enforced/index-join/btree-equi-join-01/btree-equi-join-01.8.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/open-index-non-enforced/index-join/btree-equi-join-01/btree-equi-join-01.8.adm
new file mode 100644
index 0000000..78a404d
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/open-index-non-enforced/index-join/btree-equi-join-01/btree-equi-join-01.8.adm
@@ -0,0 +1,8 @@
+{ "c1": 1, "c2": 101 }
+{ "c1": 1, "c2": 105 }
+{ "c1": 3, "c2": 103 }
+{ "c1": 3, "c2": 107 }
+{ "c1": 5, "c2": 101 }
+{ "c1": 5, "c2": 105 }
+{ "c1": 7, "c2": 103 }
+{ "c1": 7, "c2": 107 }
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/open-index-non-enforced/index-join/btree-equi-join-01/btree-equi-join-01.9.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/open-index-non-enforced/index-join/btree-equi-join-01/btree-equi-join-01.9.adm
new file mode 100644
index 0000000..348aeda
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/open-index-non-enforced/index-join/btree-equi-join-01/btree-equi-join-01.9.adm
@@ -0,0 +1,9 @@
+{ "c1": 1, "c2": 101 }
+{ "c1": 1, "c2": 103 }
+{ "c1": 2, "c2": 101 }
+{ "c1": 2, "c2": 103 }
+{ "c1": 3, "c2": 101 }
+{ "c1": 3, "c2": 103 }
+{ "c1": 5, "c2": 101 }
+{ "c1": 5, "c2": 103 }
+{ "c1": 7, "c2": 107 }
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/open-index-non-enforced/index-selection/btree-index-02/btree-index-02.4.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/open-index-non-enforced/index-selection/btree-index-02/btree-index-02.4.adm
new file mode 100644
index 0000000..66fa150
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/open-index-non-enforced/index-selection/btree-index-02/btree-index-02.4.adm
@@ -0,0 +1 @@
+{ "res": 3 }
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/open-index-non-enforced/index-selection/btree-index-02/btree-index-02.5.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/open-index-non-enforced/index-selection/btree-index-02/btree-index-02.5.adm
new file mode 100644
index 0000000..18cc982
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/open-index-non-enforced/index-selection/btree-index-02/btree-index-02.5.adm
@@ -0,0 +1,2 @@
+{ "res": 3 }
+{ "res": 5 }
\ No newline at end of file
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 0ad091f..a2f6d7d 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/testsuite_sqlpp.xml
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/testsuite_sqlpp.xml
@@ -3584,6 +3584,11 @@
           <output-dir compare="Text">btree-index-01</output-dir>
         </compilation-unit>
       </test-case>
+      <test-case FilePath="open-index-non-enforced/index-selection">
+        <compilation-unit name="btree-index-02">
+          <output-dir compare="Text">btree-index-02</output-dir>
+        </compilation-unit>
+      </test-case>
     <test-case FilePath="open-index-non-enforced/correlated-index-selection">
       <compilation-unit name="btree-index-01">
         <output-dir compare="Text">btree-index-01</output-dir>