You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@asterixdb.apache.org by wa...@apache.org on 2015/10/06 02:36:48 UTC

incubator-asterixdb git commit: ASTERIXDB-920: Fixed consecutive index-nested-loop joins failure

Repository: incubator-asterixdb
Updated Branches:
  refs/heads/master 3db1397c3 -> f758b488c


ASTERIXDB-920: Fixed consecutive index-nested-loop joins failure

Index-nested-loop join now tries to use an index from the inner relation first.

Change-Id: Ie7bcc70c19d2c1e70ce274eaacfe2c3ce89c8de8
Reviewed-on: https://asterix-gerrit.ics.uci.edu/434
Tested-by: Jenkins <je...@fulliautomatix.ics.uci.edu>
Reviewed-by: Yingyi Bu <bu...@gmail.com>


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

Branch: refs/heads/master
Commit: f758b488c9963e0a328b08b7b935dc26e96f5cfa
Parents: 3db1397
Author: Taewoo Kim <wa...@yahoo.com>
Authored: Mon Oct 5 14:39:17 2015 -0700
Committer: Taewoo Kim <wa...@gmail.com>
Committed: Mon Oct 5 17:33:39 2015 -0700

----------------------------------------------------------------------
 .../am/AbstractIntroduceAccessMethodRule.java   |  79 ++++--
 .../rules/am/IntroduceJoinAccessMethodRule.java |  54 +++-
 .../rules/am/OptimizableOperatorSubTree.java    | 258 +++++++++++++++----
 asterix-app/data/index-join/results.adm         |   8 +
 asterix-app/data/index-join/tw_messages.adm     |  13 +
 asterix-app/data/index-join/tw_users.adm        |  12 +
 .../btree-index-nested-loop-join.1.ddl.aql      |  60 +++++
 .../btree-index-nested-loop-join.2.update.aql   |  35 +++
 .../btree-index-nested-loop-join.3.query.aql    |  33 +++
 .../btree-secondary-equi-join.1.ddl.aql         |  49 ----
 .../btree-secondary-equi-join.2.update.aql      |  35 ---
 .../btree-secondary-equi-join.3.ddl.aql         |  29 ---
 .../btree-secondary-equi-join.4.query.aql       |  32 ---
 .../btree-secondary-equi-join_01.1.ddl.aql      |  49 ++++
 .../btree-secondary-equi-join_01.2.update.aql   |  35 +++
 .../btree-secondary-equi-join_01.3.ddl.aql      |  29 +++
 .../btree-secondary-equi-join_01.4.query.aql    |  32 +++
 .../btree-secondary-equi-join_02.1.ddl.aql      |  49 ++++
 .../btree-secondary-equi-join_02.2.update.aql   |  35 +++
 .../btree-secondary-equi-join_02.3.ddl.aql      |  29 +++
 .../btree-secondary-equi-join_02.4.query.aql    |  32 +++
 .../btree-secondary-equi-join_03.1.ddl.aql      |  50 ++++
 .../btree-secondary-equi-join_03.2.update.aql   |  36 +++
 .../btree-secondary-equi-join_03.3.ddl.aql      |  31 +++
 .../btree-secondary-equi-join_03.4.query.aql    |  33 +++
 .../btree-index-nested-loop-join.1.adm          |   4 +
 .../btree-secondary-equi-join.1.adm             |   6 -
 .../btree-secondary-equi-join_01.1.adm          |   6 +
 .../btree-secondary-equi-join_02.1.adm          |   6 +
 .../btree-secondary-equi-join_03.1.adm          |   6 +
 .../src/test/resources/runtimets/testsuite.xml  |  19 +-
 31 files changed, 942 insertions(+), 242 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-asterixdb/blob/f758b488/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/am/AbstractIntroduceAccessMethodRule.java
----------------------------------------------------------------------
diff --git a/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/am/AbstractIntroduceAccessMethodRule.java b/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/am/AbstractIntroduceAccessMethodRule.java
index e2db199..b736361 100644
--- a/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/am/AbstractIntroduceAccessMethodRule.java
+++ b/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/am/AbstractIntroduceAccessMethodRule.java
@@ -513,33 +513,66 @@ public abstract class AbstractIntroduceAccessMethodRule implements IAlgebraicRew
             // Try to match variables from optFuncExpr to datasourcescan if not
             // already matched in assigns.
             List<LogicalVariable> dsVarList = subTree.getDataSourceVariables();
-            for (int varIndex = 0; varIndex < dsVarList.size(); varIndex++) {
-                LogicalVariable var = dsVarList.get(varIndex);
-                int funcVarIndex = optFuncExpr.findLogicalVar(var);
-                // No matching var in optFuncExpr.
-                if (funcVarIndex == -1) {
-                    continue;
-                }
-                // The variable value is one of the partitioning fields.
-                List<String> fieldName = DatasetUtils.getPartitioningKeys(subTree.dataset).get(varIndex);
-                IAType fieldType = (IAType) context.getOutputTypeEnvironment(subTree.dataSourceRef.getValue())
-                        .getVarType(var);
-                // Set the fieldName in the corresponding matched function
-                // expression, and remember matching subtree.
-                optFuncExpr.setFieldName(funcVarIndex, fieldName);
-                optFuncExpr.setOptimizableSubTree(funcVarIndex, subTree);
-                optFuncExpr.setSourceVar(funcVarIndex, var);
-                optFuncExpr.setLogicalExpr(funcVarIndex, new VariableReferenceExpression(var));
-                setTypeTag(context, subTree, optFuncExpr, funcVarIndex);
-                if (subTree.hasDataSourceScan()) {
-                    fillIndexExprs(datasetIndexes, fieldName, fieldType, optFuncExpr, optFuncExprIndex, funcVarIndex,
-                            subTree, analysisCtx);
+
+            matchVarsFromOptFuncExprToDataSourceScan(optFuncExpr, optFuncExprIndex, datasetIndexes, dsVarList, subTree,
+                    analysisCtx, context, false);
+
+            // If there is one more datasource in the subtree, we need to scan that datasource, too.
+            List<LogicalVariable> additionalDsVarList = null;
+
+            if (subTree.hasIxJoinOuterAdditionalDataSource()) {
+                additionalDsVarList = new ArrayList<LogicalVariable>();
+                for (int i = 0; i < subTree.ixJoinOuterAdditionalDataSourceRefs.size(); i++) {
+                    additionalDsVarList.addAll(subTree.getIxJoinOuterAdditionalDataSourceVariables(i));
                 }
+
+                matchVarsFromOptFuncExprToDataSourceScan(optFuncExpr, optFuncExprIndex, datasetIndexes,
+                        additionalDsVarList, subTree, analysisCtx, context, true);
+
             }
+
             optFuncExprIndex++;
         }
     }
 
+    private void matchVarsFromOptFuncExprToDataSourceScan(IOptimizableFuncExpr optFuncExpr, int optFuncExprIndex,
+            List<Index> datasetIndexes, List<LogicalVariable> dsVarList, OptimizableOperatorSubTree subTree,
+            AccessMethodAnalysisContext analysisCtx, IOptimizationContext context, boolean fromAdditionalDataSource)
+            throws AlgebricksException {
+        for (int varIndex = 0; varIndex < dsVarList.size(); varIndex++) {
+            LogicalVariable var = dsVarList.get(varIndex);
+            int funcVarIndex = optFuncExpr.findLogicalVar(var);
+            // No matching var in optFuncExpr.
+            if (funcVarIndex == -1) {
+                continue;
+            }
+            // The variable value is one of the partitioning fields.
+            List<String> fieldName = null;
+            IAType fieldType = null;
+
+            if (!fromAdditionalDataSource) {
+                fieldName = DatasetUtils.getPartitioningKeys(subTree.dataset).get(varIndex);
+                fieldType = (IAType) context.getOutputTypeEnvironment(subTree.dataSourceRef.getValue()).getVarType(var);
+            } else {
+                fieldName = DatasetUtils.getPartitioningKeys(subTree.ixJoinOuterAdditionalDatasets.get(varIndex)).get(
+                        varIndex);
+                fieldType = (IAType) context.getOutputTypeEnvironment(
+                        subTree.ixJoinOuterAdditionalDataSourceRefs.get(varIndex).getValue()).getVarType(var);
+            }
+            // Set the fieldName in the corresponding matched function
+            // expression, and remember matching subtree.
+            optFuncExpr.setFieldName(funcVarIndex, fieldName);
+            optFuncExpr.setOptimizableSubTree(funcVarIndex, subTree);
+            optFuncExpr.setSourceVar(funcVarIndex, var);
+            optFuncExpr.setLogicalExpr(funcVarIndex, new VariableReferenceExpression(var));
+            setTypeTag(context, subTree, optFuncExpr, funcVarIndex);
+            if (subTree.hasDataSourceScan()) {
+                fillIndexExprs(datasetIndexes, fieldName, fieldType, optFuncExpr, optFuncExprIndex, funcVarIndex,
+                        subTree, analysisCtx);
+            }
+        }
+    }
+
     private void setTypeTag(IOptimizationContext context, OptimizableOperatorSubTree subTree,
             IOptimizableFuncExpr optFuncExpr, int funcVarIndex) throws AlgebricksException {
         // Set the typeTag if the type is not null
@@ -552,7 +585,7 @@ public abstract class AbstractIntroduceAccessMethodRule implements IAlgebraicRew
      * Returns the field name corresponding to the assigned variable at
      * varIndex. Returns null if the expr at varIndex does not yield to a field
      * access function after following a set of allowed functions.
-     * 
+     *
      * @throws AlgebricksException
      */
     protected List<String> getFieldNameFromSubTree(IOptimizableFuncExpr optFuncExpr,
@@ -663,7 +696,7 @@ public abstract class AbstractIntroduceAccessMethodRule implements IAlgebraicRew
                         parentFuncExpr);
 
                 if (parentFieldNames == null) {
-                    //Nested assign was not a field access. 
+                    //Nested assign was not a field access.
                     //We will not use index
                     return null;
                 }

http://git-wip-us.apache.org/repos/asf/incubator-asterixdb/blob/f758b488/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/am/IntroduceJoinAccessMethodRule.java
----------------------------------------------------------------------
diff --git a/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/am/IntroduceJoinAccessMethodRule.java b/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/am/IntroduceJoinAccessMethodRule.java
index 14a2eb3..67b60df 100644
--- a/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/am/IntroduceJoinAccessMethodRule.java
+++ b/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/am/IntroduceJoinAccessMethodRule.java
@@ -23,10 +23,9 @@ import java.util.Iterator;
 import java.util.List;
 import java.util.Map;
 
-import org.apache.commons.lang3.mutable.Mutable;
-
 import org.apache.asterix.metadata.declared.AqlMetadataProvider;
 import org.apache.asterix.metadata.entities.Index;
+import org.apache.commons.lang3.mutable.Mutable;
 import org.apache.hyracks.algebricks.common.exceptions.AlgebricksException;
 import org.apache.hyracks.algebricks.common.utils.Pair;
 import org.apache.hyracks.algebricks.core.algebra.base.ILogicalExpression;
@@ -49,7 +48,9 @@ import org.apache.hyracks.algebricks.core.algebra.util.OperatorPropertiesUtil;
  * Matches the following operator pattern:
  * (join) <-- (select)? <-- (assign | unnest)+ <-- (datasource scan)
  * <-- (select)? <-- (assign | unnest)+ <-- (datasource scan | unnest-map)
- * The order of the join inputs does not matter.
+ * The order of the join inputs matters (left-outer relation, right-inner relation).
+ * This rule tries to utilize an index on the inner relation first.
+ * If that's not possible, it tries to use an index on the outer relation.
  * Replaces the above pattern with the following simplified plan:
  * (select) <-- (assign) <-- (btree search) <-- (sort) <-- (unnest(index search)) <-- (assign) <-- (datasource scan | unnest-map)
  * The sort is optional, and some access methods may choose not to sort.
@@ -134,27 +135,52 @@ public class IntroduceJoinAccessMethodRule extends AbstractIntroduceAccessMethod
         }
         pruneIndexCandidates(analyzedAMs);
 
-        //Remove possibly chosen indexes from left Tree
-        if (isLeftOuterJoin) {
-            Iterator<Map.Entry<IAccessMethod, AccessMethodAnalysisContext>> amIt = analyzedAMs.entrySet().iterator();
-            // Check applicability of indexes by access method type.
-            while (amIt.hasNext()) {
-                Map.Entry<IAccessMethod, AccessMethodAnalysisContext> entry = amIt.next();
-                AccessMethodAnalysisContext amCtx = entry.getValue();
-                Iterator<Map.Entry<Index, List<Pair<Integer, Integer>>>> indexIt = amCtx.indexExprsAndVars.entrySet()
-                        .iterator();
+        // Prioritize the order of index that will be applied. If the right subtree (inner branch) has indexes,
+        // those indexes will be used.
+        String innerDataset = null;
+        if (rightSubTree.dataset != null) {
+            innerDataset = rightSubTree.dataset.getDatasetName();
+        }
+
+        Iterator<Map.Entry<IAccessMethod, AccessMethodAnalysisContext>> amIt = analyzedAMs.entrySet().iterator();
+        while (amIt.hasNext()) {
+            Map.Entry<IAccessMethod, AccessMethodAnalysisContext> entry = amIt.next();
+            AccessMethodAnalysisContext amCtx = entry.getValue();
+            Iterator<Map.Entry<Index, List<Pair<Integer, Integer>>>> indexIt = amCtx.indexExprsAndVars.entrySet()
+                    .iterator();
+
+            // Check whether we can choose the indexes from the inner relations (removing indexes from the outer relations)
+            int totalIndexCount = 0;
+            int indexCountFromTheOuterBranch = 0;
+
+            while (indexIt.hasNext()) {
+                Map.Entry<Index, List<Pair<Integer, Integer>>> indexEntry = indexIt.next();
+
+                Index chosenIndex = indexEntry.getKey();
+                //Count possible indexes that can be removed from left Tree (outer branch)
+                if (!chosenIndex.getDatasetName().equals(innerDataset)) {
+                    indexCountFromTheOuterBranch++;
+                }
+                totalIndexCount++;
+            }
+
+            if (indexCountFromTheOuterBranch < totalIndexCount) {
+                indexIt = amCtx.indexExprsAndVars.entrySet().iterator();
                 while (indexIt.hasNext()) {
                     Map.Entry<Index, List<Pair<Integer, Integer>>> indexEntry = indexIt.next();
 
                     Index chosenIndex = indexEntry.getKey();
-                    if (!chosenIndex.getDatasetName().equals(rightSubTree.dataset.getDatasetName())) {
+                    //Remove possibly chosen indexes from left Tree (outer branch)
+                    if (!chosenIndex.getDatasetName().equals(innerDataset)) {
                         indexIt.remove();
                     }
                 }
             }
         }
 
-        // Choose index to be applied.
+        // For the case of left-outer-join, we have to use indexes from the inner branch.
+        // For the inner-join, we try to use the indexes from the inner branch first.
+        // If no index is available, then we use the indexes from the outer branch.
         Pair<IAccessMethod, Index> chosenIndex = chooseIndex(analyzedAMs);
         if (chosenIndex == null) {
             context.addToDontApplySet(this, join);

http://git-wip-us.apache.org/repos/asf/incubator-asterixdb/blob/f758b488/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/am/OptimizableOperatorSubTree.java
----------------------------------------------------------------------
diff --git a/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/am/OptimizableOperatorSubTree.java b/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/am/OptimizableOperatorSubTree.java
index b2cd400..8d28c62 100644
--- a/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/am/OptimizableOperatorSubTree.java
+++ b/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/am/OptimizableOperatorSubTree.java
@@ -48,7 +48,7 @@ import org.apache.hyracks.algebricks.core.algebra.operators.logical.UnnestMapOpe
 
 /**
  * Operator subtree that matches the following patterns, and provides convenient access to its nodes:
- * (select)? <-- (assign | unnest)* <-- (datasource scan | unnest-map)
+ * (select)? <-- (assign | unnest)* <-- (datasource scan | unnest-map)*
  */
 public class OptimizableOperatorSubTree {
 
@@ -66,10 +66,18 @@ public class OptimizableOperatorSubTree {
     public final List<AbstractLogicalOperator> assignsAndUnnests = new ArrayList<AbstractLogicalOperator>();
     public Mutable<ILogicalOperator> dataSourceRef = null;
     public DataSourceType dataSourceType = DataSourceType.NO_DATASOURCE;
+
     // Dataset and type metadata. Set in setDatasetAndTypeMetadata().
     public Dataset dataset = null;
     public ARecordType recordType = null;
 
+    // Additional datasources can exist if IntroduceJoinAccessMethodRule has been applied.
+    // (E.g. There are index-nested-loop-joins in the plan.)
+    public List<Mutable<ILogicalOperator>> ixJoinOuterAdditionalDataSourceRefs = null;
+    public List<DataSourceType> ixJoinOuterAdditionalDataSourceTypes = null;
+    public List<Dataset> ixJoinOuterAdditionalDatasets = null;
+    public List<ARecordType> ixJoinOuterAdditionalRecordTypes = null;
+
     public boolean initFromSubTree(Mutable<ILogicalOperator> subTreeOpRef) {
         reset();
         rootRef = subTreeOpRef;
@@ -105,6 +113,7 @@ public class OptimizableOperatorSubTree {
 
     private boolean initializeDataSource(Mutable<ILogicalOperator> subTreeOpRef) {
         AbstractLogicalOperator subTreeOp = (AbstractLogicalOperator) subTreeOpRef.getValue();
+
         if (subTreeOp.getOperatorTag() == LogicalOperatorTag.DATASOURCESCAN) {
             dataSourceType = DataSourceType.DATASOURCE_SCAN;
             dataSourceRef = subTreeOpRef;
@@ -118,20 +127,62 @@ public class OptimizableOperatorSubTree {
             dataSourceRef = subTreeOpRef;
             return true;
         } else if (subTreeOp.getOperatorTag() == LogicalOperatorTag.UNNEST_MAP) {
-            UnnestMapOperator unnestMapOp = (UnnestMapOperator) subTreeOp;
-            ILogicalExpression unnestExpr = unnestMapOp.getExpressionRef().getValue();
-            if (unnestExpr.getExpressionTag() == LogicalExpressionTag.FUNCTION_CALL) {
-                AbstractFunctionCallExpression f = (AbstractFunctionCallExpression) unnestExpr;
-                if (f.getFunctionIdentifier().equals(AsterixBuiltinFunctions.INDEX_SEARCH)) {
-                    AccessMethodJobGenParams jobGenParams = new AccessMethodJobGenParams();
-                    jobGenParams.readFromFuncArgs(f.getArguments());
-                    if (jobGenParams.isPrimaryIndex()) {
-                        dataSourceType = DataSourceType.PRIMARY_INDEX_LOOKUP;
-                        dataSourceRef = subTreeOpRef;
-                        return true;
+            // There can be multiple unnest-map or datasource-scan operators
+            // if index-nested-loop-join has been applied by IntroduceJoinAccessMethodRule.
+            // So, we need to traverse the whole path from the subTreeOp.
+            boolean dataSourceFound = false;
+            while (true) {
+                if (subTreeOp.getOperatorTag() == LogicalOperatorTag.UNNEST_MAP) {
+                    UnnestMapOperator unnestMapOp = (UnnestMapOperator) subTreeOp;
+                    ILogicalExpression unnestExpr = unnestMapOp.getExpressionRef().getValue();
+
+                    if (unnestExpr.getExpressionTag() == LogicalExpressionTag.FUNCTION_CALL) {
+                        AbstractFunctionCallExpression f = (AbstractFunctionCallExpression) unnestExpr;
+                        if (f.getFunctionIdentifier().equals(AsterixBuiltinFunctions.INDEX_SEARCH)) {
+                            AccessMethodJobGenParams jobGenParams = new AccessMethodJobGenParams();
+                            jobGenParams.readFromFuncArgs(f.getArguments());
+                            if (jobGenParams.isPrimaryIndex()) {
+                                if (dataSourceRef == null) {
+                                    dataSourceRef = subTreeOpRef;
+                                    dataSourceType = DataSourceType.PRIMARY_INDEX_LOOKUP;
+                                } else {
+                                    // One datasource already exists. This is an additional datasource.
+                                    initializeIxJoinOuterAddtionalDataSourcesIfEmpty();
+                                    ixJoinOuterAdditionalDataSourceTypes.add(DataSourceType.PRIMARY_INDEX_LOOKUP);
+                                    ixJoinOuterAdditionalDataSourceRefs.add(subTreeOpRef);
+                                }
+                                dataSourceFound = true;
+                            }
+                        }
                     }
+                } else if (subTreeOp.getOperatorTag() == LogicalOperatorTag.DATASOURCESCAN) {
+                    initializeIxJoinOuterAddtionalDataSourcesIfEmpty();
+                    ixJoinOuterAdditionalDataSourceTypes.add(DataSourceType.DATASOURCE_SCAN);
+                    ixJoinOuterAdditionalDataSourceRefs.add(subTreeOpRef);
+                    dataSourceFound = true;
+                } else if (subTreeOp.getOperatorTag() == LogicalOperatorTag.EXTERNAL_LOOKUP) {
+                    initializeIxJoinOuterAddtionalDataSourcesIfEmpty();
+                    ixJoinOuterAdditionalDataSourceTypes.add(DataSourceType.EXTERNAL_SCAN);
+                    ixJoinOuterAdditionalDataSourceRefs.add(subTreeOpRef);
+                    dataSourceFound = true;
+                } else if (subTreeOp.getOperatorTag() == LogicalOperatorTag.EMPTYTUPLESOURCE) {
+                    initializeIxJoinOuterAddtionalDataSourcesIfEmpty();
+                    ixJoinOuterAdditionalDataSourceTypes.add(DataSourceType.COLLECTION_SCAN);
+                    ixJoinOuterAdditionalDataSourceRefs.add(subTreeOpRef);
+                }
+
+                // Traverse the subtree while there are operators in the path.
+                if (subTreeOp.hasInputs()) {
+                    subTreeOpRef = subTreeOp.getInputs().get(0);
+                    subTreeOp = (AbstractLogicalOperator) subTreeOpRef.getValue();
+                } else {
+                    break;
                 }
             }
+
+            if (dataSourceFound) {
+                return true;
+            }
         }
 
         return false;
@@ -144,48 +195,93 @@ public class OptimizableOperatorSubTree {
     public boolean setDatasetAndTypeMetadata(AqlMetadataProvider metadataProvider) throws AlgebricksException {
         String dataverseName = null;
         String datasetName = null;
-        switch (dataSourceType) {
-            case DATASOURCE_SCAN:
-                DataSourceScanOperator dataSourceScan = (DataSourceScanOperator) dataSourceRef.getValue();
-                Pair<String, String> datasetInfo = AnalysisUtil.getDatasetInfo(dataSourceScan);
-                dataverseName = datasetInfo.first;
-                datasetName = datasetInfo.second;
-                break;
-            case PRIMARY_INDEX_LOOKUP:
-                AbstractUnnestOperator unnestMapOp = (AbstractUnnestOperator) dataSourceRef.getValue();
-                ILogicalExpression unnestExpr = unnestMapOp.getExpressionRef().getValue();
-                AbstractFunctionCallExpression f = (AbstractFunctionCallExpression) unnestExpr;
-                AccessMethodJobGenParams jobGenParams = new AccessMethodJobGenParams();
-                jobGenParams.readFromFuncArgs(f.getArguments());
-                datasetName = jobGenParams.getDatasetName();
-                dataverseName = jobGenParams.getDataverseName();
-                break;
-            case EXTERNAL_SCAN:
-                ExternalDataLookupOperator externalScan = (ExternalDataLookupOperator) dataSourceRef.getValue();
-                datasetInfo = AnalysisUtil.getDatasetInfo(externalScan);
-                dataverseName = datasetInfo.first;
-                datasetName = datasetInfo.second;
-                break;
-            case COLLECTION_SCAN:
-                return true;
-            case NO_DATASOURCE:
-            default:
-                return false;
-        }
-        if (dataverseName == null || datasetName == null) {
-            return false;
-        }
-        // Find the dataset corresponding to the datasource in the metadata.
-        dataset = metadataProvider.findDataset(dataverseName, datasetName);
-        if (dataset == null) {
-            throw new AlgebricksException("No metadata for dataset " + datasetName);
+
+        Dataset ds = null;
+        ARecordType rType = null;
+
+        List<Mutable<ILogicalOperator>> sourceOpRefs = new ArrayList<Mutable<ILogicalOperator>>();
+        List<DataSourceType> dsTypes = new ArrayList<DataSourceType>();
+
+        sourceOpRefs.add(dataSourceRef);
+        dsTypes.add(dataSourceType);
+
+        // If there are multiple datasources in the subtree, we need to find the dataset for these.
+        if (ixJoinOuterAdditionalDataSourceRefs != null) {
+            for (int i = 0; i < ixJoinOuterAdditionalDataSourceRefs.size(); i++) {
+                sourceOpRefs.add(ixJoinOuterAdditionalDataSourceRefs.get(i));
+                dsTypes.add(ixJoinOuterAdditionalDataSourceTypes.get(i));
+            }
         }
-        // Get the record type for that dataset.
-        IAType itemType = metadataProvider.findType(dataverseName, dataset.getItemTypeName());
-        if (itemType.getTypeTag() != ATypeTag.RECORD) {
-            return false;
+
+        for (int i = 0; i < sourceOpRefs.size(); i++) {
+            switch (dsTypes.get(i)) {
+                case DATASOURCE_SCAN:
+                    DataSourceScanOperator dataSourceScan = (DataSourceScanOperator) sourceOpRefs.get(i).getValue();
+                    Pair<String, String> datasetInfo = AnalysisUtil.getDatasetInfo(dataSourceScan);
+                    dataverseName = datasetInfo.first;
+                    datasetName = datasetInfo.second;
+                    break;
+                case PRIMARY_INDEX_LOOKUP:
+                    AbstractUnnestOperator unnestMapOp = (AbstractUnnestOperator) sourceOpRefs.get(i).getValue();
+                    ILogicalExpression unnestExpr = unnestMapOp.getExpressionRef().getValue();
+                    AbstractFunctionCallExpression f = (AbstractFunctionCallExpression) unnestExpr;
+                    AccessMethodJobGenParams jobGenParams = new AccessMethodJobGenParams();
+                    jobGenParams.readFromFuncArgs(f.getArguments());
+                    datasetName = jobGenParams.getDatasetName();
+                    dataverseName = jobGenParams.getDataverseName();
+                    break;
+                case EXTERNAL_SCAN:
+                    ExternalDataLookupOperator externalScan = (ExternalDataLookupOperator) sourceOpRefs.get(i)
+                            .getValue();
+                    datasetInfo = AnalysisUtil.getDatasetInfo(externalScan);
+                    dataverseName = datasetInfo.first;
+                    datasetName = datasetInfo.second;
+                    break;
+                case COLLECTION_SCAN:
+                    if (i != 0) {
+                        ixJoinOuterAdditionalDatasets.add(null);
+                        ixJoinOuterAdditionalRecordTypes.add(null);
+                    }
+                    continue;
+                case NO_DATASOURCE:
+                default:
+                    return false;
+            }
+            if (dataverseName == null || datasetName == null) {
+                return false;
+            }
+            // Find the dataset corresponding to the datasource in the metadata.
+            ds = metadataProvider.findDataset(dataverseName, datasetName);
+            if (ds == null) {
+                throw new AlgebricksException("No metadata for dataset " + datasetName);
+            }
+            // Get the record type for that dataset.
+            IAType itemType = metadataProvider.findType(dataverseName, ds.getItemTypeName());
+            if (itemType.getTypeTag() != ATypeTag.RECORD) {
+                if (i == 0) {
+                    return false;
+                } else {
+                    ixJoinOuterAdditionalDatasets.add(null);
+                    ixJoinOuterAdditionalRecordTypes.add(null);
+                }
+            }
+            rType = (ARecordType) itemType;
+
+            // First index is always the primary datasource in this subtree.
+            if (i == 0) {
+                dataset = ds;
+                recordType = rType;
+            } else {
+                ixJoinOuterAdditionalDatasets.add(ds);
+                ixJoinOuterAdditionalRecordTypes.add(rType);
+            }
+
+            dataverseName = null;
+            datasetName = null;
+            ds = null;
+            rType = null;
         }
-        recordType = (ARecordType) itemType;
+
         return true;
     }
 
@@ -193,10 +289,34 @@ public class OptimizableOperatorSubTree {
         return dataSourceType != DataSourceType.NO_DATASOURCE;
     }
 
+    public boolean hasIxJoinOuterAdditionalDataSource() {
+        boolean dataSourceFound = false;
+        if (ixJoinOuterAdditionalDataSourceTypes != null) {
+            for (int i = 0; i < ixJoinOuterAdditionalDataSourceTypes.size(); i++) {
+                if (ixJoinOuterAdditionalDataSourceTypes.get(i) != DataSourceType.NO_DATASOURCE) {
+                    dataSourceFound = true;
+                    break;
+                }
+            }
+        }
+        return dataSourceFound;
+    }
+
     public boolean hasDataSourceScan() {
         return dataSourceType == DataSourceType.DATASOURCE_SCAN;
     }
 
+    public boolean hasIxJoinOuterAdditionalDataSourceScan() {
+        if (ixJoinOuterAdditionalDataSourceTypes != null) {
+            for (int i = 0; i < ixJoinOuterAdditionalDataSourceTypes.size(); i++) {
+                if (ixJoinOuterAdditionalDataSourceTypes.get(i) == DataSourceType.DATASOURCE_SCAN) {
+                    return true;
+                }
+            }
+        }
+        return false;
+    }
+
     public void reset() {
         root = null;
         rootRef = null;
@@ -204,8 +324,12 @@ public class OptimizableOperatorSubTree {
         assignsAndUnnests.clear();
         dataSourceRef = null;
         dataSourceType = DataSourceType.NO_DATASOURCE;
+        ixJoinOuterAdditionalDataSourceRefs = null;
+        ixJoinOuterAdditionalDataSourceTypes = null;
         dataset = null;
+        ixJoinOuterAdditionalDatasets = null;
         recordType = null;
+        ixJoinOuterAdditionalRecordTypes = null;
     }
 
     public void getPrimaryKeyVars(List<LogicalVariable> target) throws AlgebricksException {
@@ -243,4 +367,34 @@ public class OptimizableOperatorSubTree {
                 throw new AlgebricksException("The subtree does not have any data source.");
         }
     }
+
+    public List<LogicalVariable> getIxJoinOuterAdditionalDataSourceVariables(int idx) throws AlgebricksException {
+        if (ixJoinOuterAdditionalDataSourceRefs != null && ixJoinOuterAdditionalDataSourceRefs.size() > idx) {
+            switch (ixJoinOuterAdditionalDataSourceTypes.get(idx)) {
+                case DATASOURCE_SCAN:
+                case EXTERNAL_SCAN:
+                case PRIMARY_INDEX_LOOKUP:
+                    AbstractScanOperator scanOp = (AbstractScanOperator) ixJoinOuterAdditionalDataSourceRefs.get(idx)
+                            .getValue();
+                    return scanOp.getVariables();
+                case COLLECTION_SCAN:
+                    return new ArrayList<LogicalVariable>();
+                case NO_DATASOURCE:
+                default:
+                    throw new AlgebricksException("The subtree does not have any additional data sources.");
+            }
+        } else {
+            return null;
+        }
+    }
+
+    public void initializeIxJoinOuterAddtionalDataSourcesIfEmpty() {
+        if (ixJoinOuterAdditionalDataSourceRefs == null) {
+            ixJoinOuterAdditionalDataSourceRefs = new ArrayList<Mutable<ILogicalOperator>>();
+            ixJoinOuterAdditionalDataSourceTypes = new ArrayList<DataSourceType>();
+            ixJoinOuterAdditionalDatasets = new ArrayList<Dataset>();
+            ixJoinOuterAdditionalRecordTypes = new ArrayList<ARecordType>();
+        }
+    }
+
 }

http://git-wip-us.apache.org/repos/asf/incubator-asterixdb/blob/f758b488/asterix-app/data/index-join/results.adm
----------------------------------------------------------------------
diff --git a/asterix-app/data/index-join/results.adm b/asterix-app/data/index-join/results.adm
new file mode 100644
index 0000000..6f04907
--- /dev/null
+++ b/asterix-app/data/index-join/results.adm
@@ -0,0 +1,8 @@
+{ "vertexid": 8, "rank": double("4.3815337979687034E-10d") }
+{ "vertexid": 9, "rank": double("4.3815337979687034E-10d") }
+{ "vertexid": 13, "rank": double("4.3815337979687034E-10d") }
+{ "vertexid": 17, "rank": double("4.3815337979687034E-10d") }
+{ "vertexid": 22, "rank": double("4.3815337979687034E-10d") }
+{ "vertexid": 24, "rank": double("4.3815337979687034E-10d") }
+{ "vertexid": 25, "rank": double("4.3815337979687034E-10d") }
+{ "vertexid": 34, "rank": double("4.3815337979687034E-10d") }
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-asterixdb/blob/f758b488/asterix-app/data/index-join/tw_messages.adm
----------------------------------------------------------------------
diff --git a/asterix-app/data/index-join/tw_messages.adm b/asterix-app/data/index-join/tw_messages.adm
new file mode 100644
index 0000000..445b84c
--- /dev/null
+++ b/asterix-app/data/index-join/tw_messages.adm
@@ -0,0 +1,13 @@
+{"tweetid":1,"user":"AaliyahAckerley@136","sender-location":point("35.65,84.93"),"send-time":datetime("2005-02-14T10:10:00"),"forward-from":-1,"retweet-from":-1,"referred-topics":{{"motorola","customer-service"}},"message-text":" like motorola the customer-service is amazing"}
+{"tweetid":2,"user":"VerniaHerndon_714","sender-location":point("45.37,82.91"),"send-time":datetime("2007-04-27T10:10:00"),"forward-from":-1,"retweet-from":-1,"referred-topics":{{"sprint","network"}},"message-text":" hate sprint its network is horrible:("}
+{"tweetid":3,"user":"VerniaHerndon_714","sender-location":point("31.54,73.22"),"send-time":datetime("2009-12-20T10:10:00"),"forward-from":-1,"retweet-from":-1,"referred-topics":{{"motorola","signal"}},"message-text":" love motorola the signal is good:)"}
+{"tweetid":4,"user":"VerniaHerndon_714","sender-location":point("35.64,71.08"),"send-time":datetime("2006-03-19T10:10:00"),"forward-from":-1,"retweet-from":-1,"referred-topics":{{"at&t","voicemail-service"}},"message-text":" like at&t its voicemail-service is awesome"}
+{"tweetid":5,"user":"VerniaHerndon_714","sender-location":point("40.69,92.3"),"send-time":datetime("2008-04-08T10:10:00"),"forward-from":-1,"retweet-from":-1,"referred-topics":{{"iphone","plan"}},"message-text":" love iphone the plan is amazing:)"}
+{"tweetid":6,"user":"VerniaHerndon_714","sender-location":point("35.53,97.61"),"send-time":datetime("2010-01-05T10:10:00"),"forward-from":-1,"retweet-from":-1,"referred-topics":{{"at&t","network"}},"message-text":" like at&t its network is good"}
+{"tweetid":7,"user":"VerniaHerndon_714","sender-location":point("25.29,85.77"),"send-time":datetime("2008-10-08T10:10:00"),"forward-from":-1,"retweet-from":-1,"referred-topics":{{"at&t","speed"}},"message-text":" can't stand at&t its speed is terrible:("}
+{"tweetid":8,"user":"VerniaHerndon_714","sender-location":point("26.08,83.01"),"send-time":datetime("2007-04-14T10:10:00"),"forward-from":8831723361,"retweet-from":-1,"referred-topics":{{"motorola","network"}},"message-text":" hate motorola the network is horrible"}
+{"tweetid":9,"user":"VerniaHerndon_714","sender-location":point("40.38,73.48"),"send-time":datetime("2006-10-10T10:10:00"),"forward-from":-1,"retweet-from":3258345607,"referred-topics":{{"verizon","shortcut-menu"}},"message-text":" love verizon the shortcut-menu is awesome:)"}
+{"tweetid":10,"user":"VerniaHerndon_714","sender-location":point("47.42,68.06"),"send-time":datetime("2011-11-18T10:10:00"),"forward-from":-1,"retweet-from":-1,"referred-topics":{{"samsung","customization"}},"message-text":" love samsung the customization is mind-blowing"}
+{"tweetid":11,"user":"VerniaHerndon_714","sender-location":point("27.95,95.22"),"send-time":datetime("2006-10-24T10:10:00"),"forward-from":-1,"retweet-from":-1,"referred-topics":{{"at&t","voice-clarity"}},"message-text":" dislike at&t the voice-clarity is bad"}
+{"tweetid":12,"user":"VerniaHerndon_714","sender-location":point("37.68,96.92"),"send-time":datetime("2007-06-28T10:10:00"),"forward-from":-1,"retweet-from":-1,"referred-topics":{{"samsung","customization"}},"message-text":" dislike samsung its customization is OMG:("}
+{"tweetid":13,"user":"VerniaHerndon_714","sender-location":point("33.25,67.32"),"send-time":datetime("2007-08-19T10:10:00"),"forward-from":1231614665,"retweet-from":-1,"referred-topics":{{"sprint","plan"}},"message-text":" love sprint the plan is good"}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-asterixdb/blob/f758b488/asterix-app/data/index-join/tw_users.adm
----------------------------------------------------------------------
diff --git a/asterix-app/data/index-join/tw_users.adm b/asterix-app/data/index-join/tw_users.adm
new file mode 100644
index 0000000..299adc3
--- /dev/null
+++ b/asterix-app/data/index-join/tw_users.adm
@@ -0,0 +1,12 @@
+{"screen-name":"AaliyahAckerley@136","lang":"en","friends_count":6148151,"statuses_count":131,"name":"Aaliyah Ackerley","followers_count":4843658}
+{"screen-name":"AaliyahAckerley_586","lang":"en","friends_count":14590365,"statuses_count":54,"name":"Aaliyah Ackerley","followers_count":1866794}
+{"screen-name":"AaliyahAdams#87","lang":"en","friends_count":3808840,"statuses_count":385,"name":"Aaliyah Adams","followers_count":7511412}
+{"screen-name":"AaliyahAdams#977","lang":"en","friends_count":11060186,"statuses_count":452,"name":"Aaliyah Adams","followers_count":240575}
+{"screen-name":"AaliyahAdams_995","lang":"en","friends_count":12240342,"statuses_count":466,"name":"Aaliyah Adams","followers_count":9042070}
+{"screen-name":"AaliyahAgg#257","lang":"en","friends_count":1787009,"statuses_count":49,"name":"Aaliyah Agg","followers_count":1368140}
+{"screen-name":"AaliyahAgg$544","lang":"en","friends_count":9576364,"statuses_count":14,"name":"Aaliyah Agg","followers_count":11032599}
+{"screen-name":"AaliyahAkers#573","lang":"en","friends_count":871157,"statuses_count":439,"name":"Aaliyah Akers","followers_count":10481754}
+{"screen-name":"AaliyahAlbright@317","lang":"en","friends_count":12759158,"statuses_count":257,"name":"Aaliyah Albright","followers_count":1769056}
+{"screen-name":"VerniaHerndon_714","lang":"en","friends_count":12388156,"statuses_count":448,"name":"Aaliyah Alcocke","followers_count":594008}
+{"screen-name":"AaliyahAlcocke_869","lang":"en","friends_count":2365292,"statuses_count":442,"name":"Aaliyah Alcocke","followers_count":8066644}
+{"screen-name":"AaliyahAlington@332","lang":"en","friends_count":9864476,"statuses_count":153,"name":"Aaliyah Alington","followers_count":3071616}

http://git-wip-us.apache.org/repos/asf/incubator-asterixdb/blob/f758b488/asterix-app/src/test/resources/runtimets/queries/index-join/btree-index-nested-loop-join/btree-index-nested-loop-join.1.ddl.aql
----------------------------------------------------------------------
diff --git a/asterix-app/src/test/resources/runtimets/queries/index-join/btree-index-nested-loop-join/btree-index-nested-loop-join.1.ddl.aql b/asterix-app/src/test/resources/runtimets/queries/index-join/btree-index-nested-loop-join/btree-index-nested-loop-join.1.ddl.aql
new file mode 100644
index 0000000..cc8ba34
--- /dev/null
+++ b/asterix-app/src/test/resources/runtimets/queries/index-join/btree-index-nested-loop-join/btree-index-nested-loop-join.1.ddl.aql
@@ -0,0 +1,60 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT 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 Nested Loop Join on three datasets. Two index nested loop joins should be nested properly.
+ * Success        : Yes
+ */
+
+drop dataverse test if exists;
+create dataverse test;
+
+use dataverse test;
+
+create type TwitterUserType as {
+	screen-name: string,
+	lang: string,
+	friends_count: int32,
+	statuses_count: int32,
+	name: string,
+	followers_count: int32
+}
+
+create type TweetMessageType as {
+	tweetid: int64,
+	user: string,
+	sender-location: point,
+	send-time: datetime,
+	forward-from: int64,
+	retweet-from: int64,
+	referred-topics: {{ string }},
+	message-text: string
+}
+
+create type ResultType as {
+	vertexid: int64,
+	rank: double
+}
+
+create dataset TwitterUsers(TwitterUserType) primary key screen-name;
+
+create dataset TweetMessages(TweetMessageType) primary key tweetid;
+
+create dataset results(ResultType) primary key vertexid;
+
+

http://git-wip-us.apache.org/repos/asf/incubator-asterixdb/blob/f758b488/asterix-app/src/test/resources/runtimets/queries/index-join/btree-index-nested-loop-join/btree-index-nested-loop-join.2.update.aql
----------------------------------------------------------------------
diff --git a/asterix-app/src/test/resources/runtimets/queries/index-join/btree-index-nested-loop-join/btree-index-nested-loop-join.2.update.aql b/asterix-app/src/test/resources/runtimets/queries/index-join/btree-index-nested-loop-join/btree-index-nested-loop-join.2.update.aql
new file mode 100644
index 0000000..a6b4348
--- /dev/null
+++ b/asterix-app/src/test/resources/runtimets/queries/index-join/btree-index-nested-loop-join/btree-index-nested-loop-join.2.update.aql
@@ -0,0 +1,35 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+ /*
+ * Description    : Index Nested Loop Join on three datasets. Two index nested loop joins should be nested properly.
+ * Success        : Yes
+ */
+ use dataverse test;
+
+load dataset TweetMessages
+using "org.apache.asterix.external.dataset.adapter.NCFileSystemAdapter"
+(("path"="nc1://data/index-join/tw_messages.adm"),("format"="adm"));
+
+load dataset TwitterUsers
+using "org.apache.asterix.external.dataset.adapter.NCFileSystemAdapter"
+(("path"="nc1://data/index-join/tw_users.adm"),("format"="adm"));
+
+load dataset results
+using "org.apache.asterix.external.dataset.adapter.NCFileSystemAdapter"
+(("path"="nc1://data/index-join/results.adm"),("format"="adm"));

http://git-wip-us.apache.org/repos/asf/incubator-asterixdb/blob/f758b488/asterix-app/src/test/resources/runtimets/queries/index-join/btree-index-nested-loop-join/btree-index-nested-loop-join.3.query.aql
----------------------------------------------------------------------
diff --git a/asterix-app/src/test/resources/runtimets/queries/index-join/btree-index-nested-loop-join/btree-index-nested-loop-join.3.query.aql b/asterix-app/src/test/resources/runtimets/queries/index-join/btree-index-nested-loop-join/btree-index-nested-loop-join.3.query.aql
new file mode 100644
index 0000000..6356191
--- /dev/null
+++ b/asterix-app/src/test/resources/runtimets/queries/index-join/btree-index-nested-loop-join/btree-index-nested-loop-join.3.query.aql
@@ -0,0 +1,33 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+ /*
+ * Description    : Index Nested Loop Join on three datasets. Two index nested loop joins should be nested properly.
+ * Success        : Yes
+ */
+ use dataverse test;
+
+for $tu in dataset TwitterUsers
+for $tm in dataset TweetMessages
+for $r in dataset results
+    where
+       $r.vertexid /*+ indexnl */ = $tm.tweetid
+       and
+       $tm.user  /*+ indexnl */ = $tu.screen-name
+    order by $tm.tweetid
+	return $tm.tweetid;

http://git-wip-us.apache.org/repos/asf/incubator-asterixdb/blob/f758b488/asterix-app/src/test/resources/runtimets/queries/index-join/btree-secondary-equi-join/btree-secondary-equi-join.1.ddl.aql
----------------------------------------------------------------------
diff --git a/asterix-app/src/test/resources/runtimets/queries/index-join/btree-secondary-equi-join/btree-secondary-equi-join.1.ddl.aql b/asterix-app/src/test/resources/runtimets/queries/index-join/btree-secondary-equi-join/btree-secondary-equi-join.1.ddl.aql
deleted file mode 100644
index 4c03b65..0000000
--- a/asterix-app/src/test/resources/runtimets/queries/index-join/btree-secondary-equi-join/btree-secondary-equi-join.1.ddl.aql
+++ /dev/null
@@ -1,49 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *   http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing,
- * software distributed under the License is distributed on an
- * "AS IS" BASIS, WITHOUT 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    : Equi joins two datasets, DBLP and CSX, based on their title.
- *                  DBLP has a secondary btree index on title, and given the 'indexnl' hint
- *                  we expect the join to be transformed into an indexed nested-loop join.
- * Success        : Yes
- */
-
-drop dataverse test if exists;
-create dataverse test;
-
-use dataverse test;
-
-create type DBLPType as closed {
-  id: int64,
-  dblpid: string,
-  title: string,
-  authors: string,
-  misc: string
-}
-
-create type CSXType as closed {
-  id: int64,
-  csxid: string,
-  title: string,
-  authors: string,
-  misc: string
-}
-
-create dataset DBLP(DBLPType) primary key id;
-create dataset CSX(CSXType) primary key id;
-

http://git-wip-us.apache.org/repos/asf/incubator-asterixdb/blob/f758b488/asterix-app/src/test/resources/runtimets/queries/index-join/btree-secondary-equi-join/btree-secondary-equi-join.2.update.aql
----------------------------------------------------------------------
diff --git a/asterix-app/src/test/resources/runtimets/queries/index-join/btree-secondary-equi-join/btree-secondary-equi-join.2.update.aql b/asterix-app/src/test/resources/runtimets/queries/index-join/btree-secondary-equi-join/btree-secondary-equi-join.2.update.aql
deleted file mode 100644
index 2b3c126..0000000
--- a/asterix-app/src/test/resources/runtimets/queries/index-join/btree-secondary-equi-join/btree-secondary-equi-join.2.update.aql
+++ /dev/null
@@ -1,35 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *   http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing,
- * software distributed under the License is distributed on an
- * "AS IS" BASIS, WITHOUT 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    : Equi joins two datasets, DBLP and CSX, based on their title.
- *                  DBLP has a secondary btree index on title, and given the 'indexnl' hint
- *                  we expect the join to be transformed into an indexed nested-loop join.
- * Success        : Yes
- */
-
-use dataverse test;
-
-load dataset DBLP
-using "org.apache.asterix.external.dataset.adapter.NCFileSystemAdapter"
-(("path"="nc1://data/pub-small/dblp-small-id.txt"),("format"="delimited-text"),("delimiter"=":"));
-
-load dataset CSX
-using "org.apache.asterix.external.dataset.adapter.NCFileSystemAdapter"
-(("path"="nc1://data/pub-small/csx-small-id.txt"),("format"="delimited-text"),("delimiter"=":"),("quote"="\u0000"));
-

http://git-wip-us.apache.org/repos/asf/incubator-asterixdb/blob/f758b488/asterix-app/src/test/resources/runtimets/queries/index-join/btree-secondary-equi-join/btree-secondary-equi-join.3.ddl.aql
----------------------------------------------------------------------
diff --git a/asterix-app/src/test/resources/runtimets/queries/index-join/btree-secondary-equi-join/btree-secondary-equi-join.3.ddl.aql b/asterix-app/src/test/resources/runtimets/queries/index-join/btree-secondary-equi-join/btree-secondary-equi-join.3.ddl.aql
deleted file mode 100644
index 2cca7e7..0000000
--- a/asterix-app/src/test/resources/runtimets/queries/index-join/btree-secondary-equi-join/btree-secondary-equi-join.3.ddl.aql
+++ /dev/null
@@ -1,29 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *   http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing,
- * software distributed under the License is distributed on an
- * "AS IS" BASIS, WITHOUT 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    : Equi joins two datasets, DBLP and CSX, based on their title.
- *                  DBLP has a secondary btree index on title, and given the 'indexnl' hint 
- *                  we expect the join to be transformed into an indexed nested-loop join.
- * Success        : Yes
- */
-
-use dataverse test;
-
-create index title_index on DBLP(authors);
-

http://git-wip-us.apache.org/repos/asf/incubator-asterixdb/blob/f758b488/asterix-app/src/test/resources/runtimets/queries/index-join/btree-secondary-equi-join/btree-secondary-equi-join.4.query.aql
----------------------------------------------------------------------
diff --git a/asterix-app/src/test/resources/runtimets/queries/index-join/btree-secondary-equi-join/btree-secondary-equi-join.4.query.aql b/asterix-app/src/test/resources/runtimets/queries/index-join/btree-secondary-equi-join/btree-secondary-equi-join.4.query.aql
deleted file mode 100644
index 040f319..0000000
--- a/asterix-app/src/test/resources/runtimets/queries/index-join/btree-secondary-equi-join/btree-secondary-equi-join.4.query.aql
+++ /dev/null
@@ -1,32 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *   http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing,
- * software distributed under the License is distributed on an
- * "AS IS" BASIS, WITHOUT 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    : Equi joins two datasets, DBLP and CSX, based on their title.
- *                  DBLP has a secondary btree index on title, and given the 'indexnl' hint 
- *                  we expect the join to be transformed into an indexed nested-loop join.
- * Success        : Yes
- */
-
-use dataverse test;
-
-for $a in dataset('DBLP')
-for $b in dataset('CSX')
-where $a.authors /*+ indexnl */ = $b.authors
-order by $a.id, $b.id
-return {"aid": $a.id, "bid": $b.id, "authors": $a.authors}

http://git-wip-us.apache.org/repos/asf/incubator-asterixdb/blob/f758b488/asterix-app/src/test/resources/runtimets/queries/index-join/btree-secondary-equi-join_01/btree-secondary-equi-join_01.1.ddl.aql
----------------------------------------------------------------------
diff --git a/asterix-app/src/test/resources/runtimets/queries/index-join/btree-secondary-equi-join_01/btree-secondary-equi-join_01.1.ddl.aql b/asterix-app/src/test/resources/runtimets/queries/index-join/btree-secondary-equi-join_01/btree-secondary-equi-join_01.1.ddl.aql
new file mode 100644
index 0000000..672a457
--- /dev/null
+++ b/asterix-app/src/test/resources/runtimets/queries/index-join/btree-secondary-equi-join_01/btree-secondary-equi-join_01.1.ddl.aql
@@ -0,0 +1,49 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+ /*
+ * Description    : Equi joins two datasets, DBLP and CSX, based on their authors.
+ *                  DBLP has a secondary btree index on authors, and given the 'indexnl' hint
+ *                  we expect the join to be transformed into an indexed nested-loop join.
+ * Success        : Yes
+ */
+
+drop dataverse test if exists;
+create dataverse test;
+
+use dataverse test;
+
+create type DBLPType as closed {
+  id: int64,
+  dblpid: string,
+  title: string,
+  authors: string,
+  misc: string
+}
+
+create type CSXType as closed {
+  id: int64,
+  csxid: string,
+  title: string,
+  authors: string,
+  misc: string
+}
+
+create dataset DBLP(DBLPType) primary key id;
+create dataset CSX(CSXType) primary key id;
+

http://git-wip-us.apache.org/repos/asf/incubator-asterixdb/blob/f758b488/asterix-app/src/test/resources/runtimets/queries/index-join/btree-secondary-equi-join_01/btree-secondary-equi-join_01.2.update.aql
----------------------------------------------------------------------
diff --git a/asterix-app/src/test/resources/runtimets/queries/index-join/btree-secondary-equi-join_01/btree-secondary-equi-join_01.2.update.aql b/asterix-app/src/test/resources/runtimets/queries/index-join/btree-secondary-equi-join_01/btree-secondary-equi-join_01.2.update.aql
new file mode 100644
index 0000000..7a2f450
--- /dev/null
+++ b/asterix-app/src/test/resources/runtimets/queries/index-join/btree-secondary-equi-join_01/btree-secondary-equi-join_01.2.update.aql
@@ -0,0 +1,35 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+ /*
+ * Description    : Equi joins two datasets, DBLP and CSX, based on their authors.
+ *                  DBLP has a secondary btree index on authors, and given the 'indexnl' hint
+ *                  we expect the join to be transformed into an indexed nested-loop join.
+ * Success        : Yes
+ */
+
+use dataverse test;
+
+load dataset DBLP
+using "org.apache.asterix.external.dataset.adapter.NCFileSystemAdapter"
+(("path"="nc1://data/pub-small/dblp-small-id.txt"),("format"="delimited-text"),("delimiter"=":"));
+
+load dataset CSX
+using "org.apache.asterix.external.dataset.adapter.NCFileSystemAdapter"
+(("path"="nc1://data/pub-small/csx-small-id.txt"),("format"="delimited-text"),("delimiter"=":"),("quote"="\u0000"));
+

http://git-wip-us.apache.org/repos/asf/incubator-asterixdb/blob/f758b488/asterix-app/src/test/resources/runtimets/queries/index-join/btree-secondary-equi-join_01/btree-secondary-equi-join_01.3.ddl.aql
----------------------------------------------------------------------
diff --git a/asterix-app/src/test/resources/runtimets/queries/index-join/btree-secondary-equi-join_01/btree-secondary-equi-join_01.3.ddl.aql b/asterix-app/src/test/resources/runtimets/queries/index-join/btree-secondary-equi-join_01/btree-secondary-equi-join_01.3.ddl.aql
new file mode 100644
index 0000000..f781be5
--- /dev/null
+++ b/asterix-app/src/test/resources/runtimets/queries/index-join/btree-secondary-equi-join_01/btree-secondary-equi-join_01.3.ddl.aql
@@ -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    : Equi joins two datasets, DBLP and CSX, based on their authors.
+ *                  DBLP has a secondary btree index on authors, and given the 'indexnl' hint
+ *                  we expect the join to be transformed into an indexed nested-loop join.
+ * Success        : Yes
+ */
+
+use dataverse test;
+
+create index authors_index on DBLP(authors);
+

http://git-wip-us.apache.org/repos/asf/incubator-asterixdb/blob/f758b488/asterix-app/src/test/resources/runtimets/queries/index-join/btree-secondary-equi-join_01/btree-secondary-equi-join_01.4.query.aql
----------------------------------------------------------------------
diff --git a/asterix-app/src/test/resources/runtimets/queries/index-join/btree-secondary-equi-join_01/btree-secondary-equi-join_01.4.query.aql b/asterix-app/src/test/resources/runtimets/queries/index-join/btree-secondary-equi-join_01/btree-secondary-equi-join_01.4.query.aql
new file mode 100644
index 0000000..a97a51a
--- /dev/null
+++ b/asterix-app/src/test/resources/runtimets/queries/index-join/btree-secondary-equi-join_01/btree-secondary-equi-join_01.4.query.aql
@@ -0,0 +1,32 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+/*
+ * Description    : Equi joins two datasets, DBLP and CSX, based on their authors.
+ *                  DBLP has a secondary btree index on authors, and given the 'indexnl' hint
+ *                  we expect the join to be transformed into an indexed nested-loop join.
+ * Success        : Yes
+ */
+
+use dataverse test;
+
+for $a in dataset('DBLP')
+for $b in dataset('CSX')
+where $a.authors /*+ indexnl */ = $b.authors
+order by $a.id, $b.id
+return {"aid": $a.id, "bid": $b.id, "authors": $a.authors}

http://git-wip-us.apache.org/repos/asf/incubator-asterixdb/blob/f758b488/asterix-app/src/test/resources/runtimets/queries/index-join/btree-secondary-equi-join_02/btree-secondary-equi-join_02.1.ddl.aql
----------------------------------------------------------------------
diff --git a/asterix-app/src/test/resources/runtimets/queries/index-join/btree-secondary-equi-join_02/btree-secondary-equi-join_02.1.ddl.aql b/asterix-app/src/test/resources/runtimets/queries/index-join/btree-secondary-equi-join_02/btree-secondary-equi-join_02.1.ddl.aql
new file mode 100644
index 0000000..bfc50ec
--- /dev/null
+++ b/asterix-app/src/test/resources/runtimets/queries/index-join/btree-secondary-equi-join_02/btree-secondary-equi-join_02.1.ddl.aql
@@ -0,0 +1,49 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+/*
+ * Description    : Equi joins two datasets, DBLP and CSX, based on their authors.
+ *                  CSX has a secondary btree index on authors, and given the 'indexnl' hint
+ *                  we expect the join to be transformed into an indexed nested-loop join.
+ * Success        : Yes
+ */
+
+drop dataverse test if exists;
+create dataverse test;
+
+use dataverse test;
+
+create type DBLPType as closed {
+  id: int64,
+  dblpid: string,
+  title: string,
+  authors: string,
+  misc: string
+}
+
+create type CSXType as closed {
+  id: int64,
+  csxid: string,
+  title: string,
+  authors: string,
+  misc: string
+}
+
+create dataset DBLP(DBLPType) primary key id;
+create dataset CSX(CSXType) primary key id;
+

http://git-wip-us.apache.org/repos/asf/incubator-asterixdb/blob/f758b488/asterix-app/src/test/resources/runtimets/queries/index-join/btree-secondary-equi-join_02/btree-secondary-equi-join_02.2.update.aql
----------------------------------------------------------------------
diff --git a/asterix-app/src/test/resources/runtimets/queries/index-join/btree-secondary-equi-join_02/btree-secondary-equi-join_02.2.update.aql b/asterix-app/src/test/resources/runtimets/queries/index-join/btree-secondary-equi-join_02/btree-secondary-equi-join_02.2.update.aql
new file mode 100644
index 0000000..384ce3c
--- /dev/null
+++ b/asterix-app/src/test/resources/runtimets/queries/index-join/btree-secondary-equi-join_02/btree-secondary-equi-join_02.2.update.aql
@@ -0,0 +1,35 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+/*
+ * Description    : Equi joins two datasets, DBLP and CSX, based on their authors.
+ *                  CSX has a secondary btree index on authors, and given the 'indexnl' hint
+ *                  we expect the join to be transformed into an indexed nested-loop join.
+ * Success        : Yes
+ */
+
+use dataverse test;
+
+load dataset DBLP
+using "org.apache.asterix.external.dataset.adapter.NCFileSystemAdapter"
+(("path"="nc1://data/pub-small/dblp-small-id.txt"),("format"="delimited-text"),("delimiter"=":"));
+
+load dataset CSX
+using "org.apache.asterix.external.dataset.adapter.NCFileSystemAdapter"
+(("path"="nc1://data/pub-small/csx-small-id.txt"),("format"="delimited-text"),("delimiter"=":"),("quote"="\u0000"));
+

http://git-wip-us.apache.org/repos/asf/incubator-asterixdb/blob/f758b488/asterix-app/src/test/resources/runtimets/queries/index-join/btree-secondary-equi-join_02/btree-secondary-equi-join_02.3.ddl.aql
----------------------------------------------------------------------
diff --git a/asterix-app/src/test/resources/runtimets/queries/index-join/btree-secondary-equi-join_02/btree-secondary-equi-join_02.3.ddl.aql b/asterix-app/src/test/resources/runtimets/queries/index-join/btree-secondary-equi-join_02/btree-secondary-equi-join_02.3.ddl.aql
new file mode 100644
index 0000000..79467ab
--- /dev/null
+++ b/asterix-app/src/test/resources/runtimets/queries/index-join/btree-secondary-equi-join_02/btree-secondary-equi-join_02.3.ddl.aql
@@ -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    : Equi joins two datasets, DBLP and CSX, based on their authors.
+ *                  CSX has a secondary btree index on authors, and given the 'indexnl' hint
+ *                  we expect the join to be transformed into an indexed nested-loop join.
+ * Success        : Yes
+ */
+
+use dataverse test;
+
+create index authors_index on CSX(authors);
+

http://git-wip-us.apache.org/repos/asf/incubator-asterixdb/blob/f758b488/asterix-app/src/test/resources/runtimets/queries/index-join/btree-secondary-equi-join_02/btree-secondary-equi-join_02.4.query.aql
----------------------------------------------------------------------
diff --git a/asterix-app/src/test/resources/runtimets/queries/index-join/btree-secondary-equi-join_02/btree-secondary-equi-join_02.4.query.aql b/asterix-app/src/test/resources/runtimets/queries/index-join/btree-secondary-equi-join_02/btree-secondary-equi-join_02.4.query.aql
new file mode 100644
index 0000000..42423a3
--- /dev/null
+++ b/asterix-app/src/test/resources/runtimets/queries/index-join/btree-secondary-equi-join_02/btree-secondary-equi-join_02.4.query.aql
@@ -0,0 +1,32 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+/*
+ * Description    : Equi joins two datasets, DBLP and CSX, based on their authors.
+ *                  CSX has a secondary btree index on authors, and given the 'indexnl' hint
+ *                  we expect the join to be transformed into an indexed nested-loop join.
+ * Success        : Yes
+ */
+
+use dataverse test;
+
+for $a in dataset('DBLP')
+for $b in dataset('CSX')
+where $a.authors /*+ indexnl */ = $b.authors
+order by $a.id, $b.id
+return {"aid": $a.id, "bid": $b.id, "authors": $a.authors}

http://git-wip-us.apache.org/repos/asf/incubator-asterixdb/blob/f758b488/asterix-app/src/test/resources/runtimets/queries/index-join/btree-secondary-equi-join_03/btree-secondary-equi-join_03.1.ddl.aql
----------------------------------------------------------------------
diff --git a/asterix-app/src/test/resources/runtimets/queries/index-join/btree-secondary-equi-join_03/btree-secondary-equi-join_03.1.ddl.aql b/asterix-app/src/test/resources/runtimets/queries/index-join/btree-secondary-equi-join_03/btree-secondary-equi-join_03.1.ddl.aql
new file mode 100644
index 0000000..c7597c2
--- /dev/null
+++ b/asterix-app/src/test/resources/runtimets/queries/index-join/btree-secondary-equi-join_03/btree-secondary-equi-join_03.1.ddl.aql
@@ -0,0 +1,50 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+/*
+ * Description    : Equi joins two datasets, DBLP and CSX, based on their authors.
+ *                  Both datasets have a secondary btree index on authors. So, given the 'indexnl' hint
+ *                  we expect the join to be transformed into an indexed nested-loop join.
+ *                  (outer relation: the first relation described in the for-loop, inner relation: the other relation)
+ * Success        : Yes
+ */
+
+drop dataverse test if exists;
+create dataverse test;
+
+use dataverse test;
+
+create type DBLPType as closed {
+  id: int64,
+  dblpid: string,
+  title: string,
+  authors: string,
+  misc: string
+}
+
+create type CSXType as closed {
+  id: int64,
+  csxid: string,
+  title: string,
+  authors: string,
+  misc: string
+}
+
+create dataset DBLP(DBLPType) primary key id;
+create dataset CSX(CSXType) primary key id;
+

http://git-wip-us.apache.org/repos/asf/incubator-asterixdb/blob/f758b488/asterix-app/src/test/resources/runtimets/queries/index-join/btree-secondary-equi-join_03/btree-secondary-equi-join_03.2.update.aql
----------------------------------------------------------------------
diff --git a/asterix-app/src/test/resources/runtimets/queries/index-join/btree-secondary-equi-join_03/btree-secondary-equi-join_03.2.update.aql b/asterix-app/src/test/resources/runtimets/queries/index-join/btree-secondary-equi-join_03/btree-secondary-equi-join_03.2.update.aql
new file mode 100644
index 0000000..d6d3ace
--- /dev/null
+++ b/asterix-app/src/test/resources/runtimets/queries/index-join/btree-secondary-equi-join_03/btree-secondary-equi-join_03.2.update.aql
@@ -0,0 +1,36 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+/*
+ * Description    : Equi joins two datasets, DBLP and CSX, based on their authors.
+ *                  Both datasets have a secondary btree index on authors. So, given the 'indexnl' hint
+ *                  we expect the join to be transformed into an indexed nested-loop join.
+ *                  (outer relation: the first relation described in the for-loop, inner relation: the other relation)
+ * Success        : Yes
+ */
+
+use dataverse test;
+
+load dataset DBLP
+using "org.apache.asterix.external.dataset.adapter.NCFileSystemAdapter"
+(("path"="nc1://data/pub-small/dblp-small-id.txt"),("format"="delimited-text"),("delimiter"=":"));
+
+load dataset CSX
+using "org.apache.asterix.external.dataset.adapter.NCFileSystemAdapter"
+(("path"="nc1://data/pub-small/csx-small-id.txt"),("format"="delimited-text"),("delimiter"=":"),("quote"="\u0000"));
+

http://git-wip-us.apache.org/repos/asf/incubator-asterixdb/blob/f758b488/asterix-app/src/test/resources/runtimets/queries/index-join/btree-secondary-equi-join_03/btree-secondary-equi-join_03.3.ddl.aql
----------------------------------------------------------------------
diff --git a/asterix-app/src/test/resources/runtimets/queries/index-join/btree-secondary-equi-join_03/btree-secondary-equi-join_03.3.ddl.aql b/asterix-app/src/test/resources/runtimets/queries/index-join/btree-secondary-equi-join_03/btree-secondary-equi-join_03.3.ddl.aql
new file mode 100644
index 0000000..776bd87
--- /dev/null
+++ b/asterix-app/src/test/resources/runtimets/queries/index-join/btree-secondary-equi-join_03/btree-secondary-equi-join_03.3.ddl.aql
@@ -0,0 +1,31 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+/*
+ * Description    : Equi joins two datasets, DBLP and CSX, based on their authors.
+ *                  Both datasets have a secondary btree index on authors. So, given the 'indexnl' hint
+ *                  we expect the join to be transformed into an indexed nested-loop join.
+ *                  (outer relation: the first relation described in the for-loop, inner relation: the other relation)
+ * Success        : Yes
+ */
+
+use dataverse test;
+
+create index CSX_authors_index on CSX(authors);
+create index DBLP_authors_index on DBLP(authors);
+

http://git-wip-us.apache.org/repos/asf/incubator-asterixdb/blob/f758b488/asterix-app/src/test/resources/runtimets/queries/index-join/btree-secondary-equi-join_03/btree-secondary-equi-join_03.4.query.aql
----------------------------------------------------------------------
diff --git a/asterix-app/src/test/resources/runtimets/queries/index-join/btree-secondary-equi-join_03/btree-secondary-equi-join_03.4.query.aql b/asterix-app/src/test/resources/runtimets/queries/index-join/btree-secondary-equi-join_03/btree-secondary-equi-join_03.4.query.aql
new file mode 100644
index 0000000..ed3cd7e
--- /dev/null
+++ b/asterix-app/src/test/resources/runtimets/queries/index-join/btree-secondary-equi-join_03/btree-secondary-equi-join_03.4.query.aql
@@ -0,0 +1,33 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+/*
+ * Description    : Equi joins two datasets, DBLP and CSX, based on their authors.
+ *                  Both datasets have a secondary btree index on authors. So, given the 'indexnl' hint
+ *                  we expect the join to be transformed into an indexed nested-loop join.
+ *                  (outer relation: the first relation described in the for-loop, inner relation: the other relation)
+ * Success        : Yes
+ */
+
+use dataverse test;
+
+for $a in dataset('DBLP')
+for $b in dataset('CSX')
+where $a.authors /*+ indexnl */ = $b.authors
+order by $a.id, $b.id
+return {"aid": $a.id, "bid": $b.id, "authors": $a.authors}

http://git-wip-us.apache.org/repos/asf/incubator-asterixdb/blob/f758b488/asterix-app/src/test/resources/runtimets/results/index-join/btree-index-nested-loop-join/btree-index-nested-loop-join.1.adm
----------------------------------------------------------------------
diff --git a/asterix-app/src/test/resources/runtimets/results/index-join/btree-index-nested-loop-join/btree-index-nested-loop-join.1.adm b/asterix-app/src/test/resources/runtimets/results/index-join/btree-index-nested-loop-join/btree-index-nested-loop-join.1.adm
new file mode 100644
index 0000000..4f7a33a
--- /dev/null
+++ b/asterix-app/src/test/resources/runtimets/results/index-join/btree-index-nested-loop-join/btree-index-nested-loop-join.1.adm
@@ -0,0 +1,4 @@
+[ 8
+, 9
+, 13
+ ]

http://git-wip-us.apache.org/repos/asf/incubator-asterixdb/blob/f758b488/asterix-app/src/test/resources/runtimets/results/index-join/btree-secondary-equi-join/btree-secondary-equi-join.1.adm
----------------------------------------------------------------------
diff --git a/asterix-app/src/test/resources/runtimets/results/index-join/btree-secondary-equi-join/btree-secondary-equi-join.1.adm b/asterix-app/src/test/resources/runtimets/results/index-join/btree-secondary-equi-join/btree-secondary-equi-join.1.adm
deleted file mode 100644
index f203147..0000000
--- a/asterix-app/src/test/resources/runtimets/results/index-join/btree-secondary-equi-join/btree-secondary-equi-join.1.adm
+++ /dev/null
@@ -1,6 +0,0 @@
-[ { "aid": 5, "bid": 98, "authors": "Umeshwar Dayal Eric N. Hanson Jennifer Widom" }
-, { "aid": 34, "bid": 57, "authors": "" }
-, { "aid": 54, "bid": 91, "authors": "Lynn Andrea Stein Henry Lieberman David Ungar" }
-, { "aid": 68, "bid": 57, "authors": "" }
-, { "aid": 69, "bid": 57, "authors": "" }
- ]

http://git-wip-us.apache.org/repos/asf/incubator-asterixdb/blob/f758b488/asterix-app/src/test/resources/runtimets/results/index-join/btree-secondary-equi-join_01/btree-secondary-equi-join_01.1.adm
----------------------------------------------------------------------
diff --git a/asterix-app/src/test/resources/runtimets/results/index-join/btree-secondary-equi-join_01/btree-secondary-equi-join_01.1.adm b/asterix-app/src/test/resources/runtimets/results/index-join/btree-secondary-equi-join_01/btree-secondary-equi-join_01.1.adm
new file mode 100644
index 0000000..f203147
--- /dev/null
+++ b/asterix-app/src/test/resources/runtimets/results/index-join/btree-secondary-equi-join_01/btree-secondary-equi-join_01.1.adm
@@ -0,0 +1,6 @@
+[ { "aid": 5, "bid": 98, "authors": "Umeshwar Dayal Eric N. Hanson Jennifer Widom" }
+, { "aid": 34, "bid": 57, "authors": "" }
+, { "aid": 54, "bid": 91, "authors": "Lynn Andrea Stein Henry Lieberman David Ungar" }
+, { "aid": 68, "bid": 57, "authors": "" }
+, { "aid": 69, "bid": 57, "authors": "" }
+ ]

http://git-wip-us.apache.org/repos/asf/incubator-asterixdb/blob/f758b488/asterix-app/src/test/resources/runtimets/results/index-join/btree-secondary-equi-join_02/btree-secondary-equi-join_02.1.adm
----------------------------------------------------------------------
diff --git a/asterix-app/src/test/resources/runtimets/results/index-join/btree-secondary-equi-join_02/btree-secondary-equi-join_02.1.adm b/asterix-app/src/test/resources/runtimets/results/index-join/btree-secondary-equi-join_02/btree-secondary-equi-join_02.1.adm
new file mode 100644
index 0000000..f203147
--- /dev/null
+++ b/asterix-app/src/test/resources/runtimets/results/index-join/btree-secondary-equi-join_02/btree-secondary-equi-join_02.1.adm
@@ -0,0 +1,6 @@
+[ { "aid": 5, "bid": 98, "authors": "Umeshwar Dayal Eric N. Hanson Jennifer Widom" }
+, { "aid": 34, "bid": 57, "authors": "" }
+, { "aid": 54, "bid": 91, "authors": "Lynn Andrea Stein Henry Lieberman David Ungar" }
+, { "aid": 68, "bid": 57, "authors": "" }
+, { "aid": 69, "bid": 57, "authors": "" }
+ ]

http://git-wip-us.apache.org/repos/asf/incubator-asterixdb/blob/f758b488/asterix-app/src/test/resources/runtimets/results/index-join/btree-secondary-equi-join_03/btree-secondary-equi-join_03.1.adm
----------------------------------------------------------------------
diff --git a/asterix-app/src/test/resources/runtimets/results/index-join/btree-secondary-equi-join_03/btree-secondary-equi-join_03.1.adm b/asterix-app/src/test/resources/runtimets/results/index-join/btree-secondary-equi-join_03/btree-secondary-equi-join_03.1.adm
new file mode 100644
index 0000000..f203147
--- /dev/null
+++ b/asterix-app/src/test/resources/runtimets/results/index-join/btree-secondary-equi-join_03/btree-secondary-equi-join_03.1.adm
@@ -0,0 +1,6 @@
+[ { "aid": 5, "bid": 98, "authors": "Umeshwar Dayal Eric N. Hanson Jennifer Widom" }
+, { "aid": 34, "bid": 57, "authors": "" }
+, { "aid": 54, "bid": 91, "authors": "Lynn Andrea Stein Henry Lieberman David Ungar" }
+, { "aid": 68, "bid": 57, "authors": "" }
+, { "aid": 69, "bid": 57, "authors": "" }
+ ]

http://git-wip-us.apache.org/repos/asf/incubator-asterixdb/blob/f758b488/asterix-app/src/test/resources/runtimets/testsuite.xml
----------------------------------------------------------------------
diff --git a/asterix-app/src/test/resources/runtimets/testsuite.xml b/asterix-app/src/test/resources/runtimets/testsuite.xml
index d706feb..ba471cb 100644
--- a/asterix-app/src/test/resources/runtimets/testsuite.xml
+++ b/asterix-app/src/test/resources/runtimets/testsuite.xml
@@ -2479,13 +2479,28 @@
     </test-group> -->
     <test-group name="index-join">
         <test-case FilePath="index-join">
+            <compilation-unit name="btree-index-nested-loop-join">
+                <output-dir compare="Text">btree-index-nested-loop-join</output-dir>
+            </compilation-unit>
+        </test-case>
+        <test-case FilePath="index-join">
             <compilation-unit name="btree-primary-equi-join">
                 <output-dir compare="Text">btree-primary-equi-join</output-dir>
             </compilation-unit>
         </test-case>
         <test-case FilePath="index-join">
-            <compilation-unit name="btree-secondary-equi-join">
-                <output-dir compare="Text">btree-secondary-equi-join</output-dir>
+            <compilation-unit name="btree-secondary-equi-join_01">
+                <output-dir compare="Text">btree-secondary-equi-join_01</output-dir>
+            </compilation-unit>
+        </test-case>
+        <test-case FilePath="index-join">
+            <compilation-unit name="btree-secondary-equi-join_02">
+                <output-dir compare="Text">btree-secondary-equi-join_02</output-dir>
+            </compilation-unit>
+        </test-case>
+        <test-case FilePath="index-join">
+            <compilation-unit name="btree-secondary-equi-join_03">
+                <output-dir compare="Text">btree-secondary-equi-join_03</output-dir>
             </compilation-unit>
         </test-case>
         <test-case FilePath="index-join">