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 2017/03/19 22:55:28 UTC

[2/2] asterixdb git commit: Index-only plan step 3: Top-down Select and Join transformation rule

Index-only plan step 3: Top-down Select and Join transformation rule

 - Converted IntroduceSelectAccessMethodRule and IntroduceJoinAccessMethodRule
   from bottom-up approach to top-down approach from the root operator.
 - Index-only plan needs to verify the variables that are live in the select or join condition
   are the only variables to be used afterwards unless a variable is generated after the select or join operator.
 - In order to keep this information, top-down approach needs to be introduced.

Change-Id: I60a2a61eb46851d4c16c8f17447e3ac9b0aca778
Reviewed-on: https://asterix-gerrit.ics.uci.edu/1551
Tested-by: Jenkins <je...@fulliautomatix.ics.uci.edu>
BAD: Jenkins <je...@fulliautomatix.ics.uci.edu>
Integration-Tests: Jenkins <je...@fulliautomatix.ics.uci.edu>
Reviewed-by: Yingyi Bu <bu...@gmail.com>


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

Branch: refs/heads/master
Commit: c4eb65649b251f6a3403fb7301a40d0dab47418c
Parents: d8b5972
Author: Taewoo Kim <wa...@yahoo.com>
Authored: Fri Mar 17 19:16:11 2017 -0700
Committer: Taewoo Kim <wa...@gmail.com>
Committed: Sun Mar 19 15:55:01 2017 -0700

----------------------------------------------------------------------
 .../am/AbstractIntroduceAccessMethodRule.java   |  40 +-
 .../rules/am/AccessMethodAnalysisContext.java   |  65 +++-
 .../optimizer/rules/am/AccessMethodUtils.java   |  61 +--
 .../optimizer/rules/am/BTreeAccessMethod.java   |  24 +-
 .../optimizer/rules/am/IAccessMethod.java       |   2 +-
 .../rules/am/IntroduceJoinAccessMethodRule.java | 370 +++++++++++++------
 .../am/IntroduceLSMComponentFilterRule.java     |   9 +-
 .../am/IntroduceSelectAccessMethodRule.java     | 248 ++++++++-----
 .../rules/am/InvertedIndexAccessMethod.java     |  34 +-
 .../rules/am/OptimizableOperatorSubTree.java    |  34 +-
 .../optimizer/rules/am/RTreeAccessMethod.java   |  10 +-
 .../asterix/common/exceptions/ErrorCode.java    |   8 +
 .../main/resources/asx_errormsg/en.properties   |   8 +
 .../hyracks/api/exceptions/ErrorCode.java       |   2 +-
 14 files changed, 606 insertions(+), 309 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/asterixdb/blob/c4eb6564/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/am/AbstractIntroduceAccessMethodRule.java
----------------------------------------------------------------------
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 d7fb0ac..5179adc 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
@@ -73,7 +73,7 @@ import com.google.common.collect.ImmutableSet;
  */
 public abstract class AbstractIntroduceAccessMethodRule implements IAlgebraicRewriteRule {
 
-    private MetadataProvider metadataProvider;
+    protected MetadataProvider metadataProvider;
 
     // Function Identifier sets that retain the original field variable through each function's arguments
     private final ImmutableSet<FunctionIdentifier> funcIDSetThatRetainFieldName =
@@ -100,7 +100,8 @@ public abstract class AbstractIntroduceAccessMethodRule implements IAlgebraicRew
     }
 
     @Override
-    public boolean rewritePre(Mutable<ILogicalOperator> opRef, IOptimizationContext context) {
+    public boolean rewritePre(Mutable<ILogicalOperator> opRef, IOptimizationContext context)
+            throws AlgebricksException {
         return false;
     }
 
@@ -132,7 +133,7 @@ public abstract class AbstractIntroduceAccessMethodRule implements IAlgebraicRew
             pruneIndexCandidates(entry.getKey(), amCtx, context, typeEnvironment);
             // Remove access methods for which there are definitely no
             // applicable indexes.
-            if (amCtx.indexExprsAndVars.isEmpty()) {
+            if (amCtx.isIndexExprsAndVarsEmpty()) {
                 amIt.remove();
             }
         }
@@ -143,7 +144,7 @@ public abstract class AbstractIntroduceAccessMethodRule implements IAlgebraicRew
      * process by making it more systematic.
      */
     protected Pair<IAccessMethod, Index> chooseBestIndex(Map<IAccessMethod, AccessMethodAnalysisContext> analyzedAMs) {
-        List<Pair<IAccessMethod, Index>> list = chooseAllIndex(analyzedAMs);
+        List<Pair<IAccessMethod, Index>> list = chooseAllIndexes(analyzedAMs);
         return list.isEmpty() ? null : list.get(0);
     }
 
@@ -155,7 +156,7 @@ public abstract class AbstractIntroduceAccessMethodRule implements IAlgebraicRew
      * [InvertedIndexAccessMethod, IndexType.SINGLE_PARTITION_WORD_INVIX || SINGLE_PARTITION_NGRAM_INVIX ||
      * LENGTH_PARTITIONED_WORD_INVIX || LENGTH_PARTITIONED_NGRAM_INVIX]
      */
-    protected List<Pair<IAccessMethod, Index>> chooseAllIndex(
+    protected List<Pair<IAccessMethod, Index>> chooseAllIndexes(
             Map<IAccessMethod, AccessMethodAnalysisContext> analyzedAMs) {
         List<Pair<IAccessMethod, Index>> result = new ArrayList<>();
         // Use variables (fields) to the index types map to check which type of indexes are applied for the vars.
@@ -165,7 +166,7 @@ public abstract class AbstractIntroduceAccessMethodRule implements IAlgebraicRew
             Map.Entry<IAccessMethod, AccessMethodAnalysisContext> amEntry = amIt.next();
             AccessMethodAnalysisContext analysisCtx = amEntry.getValue();
             Iterator<Map.Entry<Index, List<Pair<Integer, Integer>>>> indexIt =
-                    analysisCtx.indexExprsAndVars.entrySet().iterator();
+                    analysisCtx.getIteratorForIndexExprsAndVars();
             while (indexIt.hasNext()) {
                 Map.Entry<Index, List<Pair<Integer, Integer>>> indexEntry = indexIt.next();
                 IAccessMethod chosenAccessMethod = amEntry.getKey();
@@ -209,7 +210,7 @@ public abstract class AbstractIntroduceAccessMethodRule implements IAlgebraicRew
     public void pruneIndexCandidates(IAccessMethod accessMethod, AccessMethodAnalysisContext analysisCtx,
             IOptimizationContext context, IVariableTypeEnvironment typeEnvironment) throws AlgebricksException {
         Iterator<Map.Entry<Index, List<Pair<Integer, Integer>>>> indexExprAndVarIt =
-                analysisCtx.indexExprsAndVars.entrySet().iterator();
+                analysisCtx.getIteratorForIndexExprsAndVars();
         // Used to keep track of matched expressions (added for prefix search)
         int numMatchedKeys = 0;
         ArrayList<Integer> matchedExpressions = new ArrayList<>();
@@ -229,7 +230,7 @@ public abstract class AbstractIntroduceAccessMethodRule implements IAlgebraicRew
                 Iterator<Pair<Integer, Integer>> exprsAndVarIter = indexExprAndVarEntry.getValue().iterator();
                 while (exprsAndVarIter.hasNext()) {
                     final Pair<Integer, Integer> exprAndVarIdx = exprsAndVarIter.next();
-                    final IOptimizableFuncExpr optFuncExpr = analysisCtx.matchedFuncExprs.get(exprAndVarIdx.first);
+                    final IOptimizableFuncExpr optFuncExpr = analysisCtx.getMatchedFuncExpr(exprAndVarIdx.first);
                     // If expr is not optimizable by concrete index then remove
                     // expr and continue.
                     if (!accessMethod.exprIsOptimizable(index, optFuncExpr)) {
@@ -357,7 +358,7 @@ public abstract class AbstractIntroduceAccessMethodRule implements IAlgebraicRew
                     continue;
                 }
             }
-            analysisCtx.indexNumMatchedKeys.put(index, new Integer(numMatchedKeys));
+            analysisCtx.putNumberOfMatchedKeys(index, Integer.valueOf(numMatchedKeys));
         }
     }
 
@@ -381,13 +382,13 @@ public abstract class AbstractIntroduceAccessMethodRule implements IAlgebraicRew
      *
      * @throws AlgebricksException
      */
-    protected boolean analyzeCondition(ILogicalExpression cond, List<AbstractLogicalOperator> assignsAndUnnests,
+    protected boolean analyzeSelectOrJoinOpConditionAndUpdateAnalyzedAM(ILogicalExpression cond,
+            List<AbstractLogicalOperator> assignsAndUnnests,
             Map<IAccessMethod, AccessMethodAnalysisContext> analyzedAMs, IOptimizationContext context,
             IVariableTypeEnvironment typeEnvironment) throws AlgebricksException {
         AbstractFunctionCallExpression funcExpr = (AbstractFunctionCallExpression) cond;
         FunctionIdentifier funcIdent = funcExpr.getFunctionIdentifier();
-        // Don't consider optimizing a disjunctive condition with an index (too
-        // complicated for now).
+        // TODO: We don't consider a disjunctive condition with an index yet since it's complex.
         if (funcIdent == AlgebricksBuiltinFunctions.OR) {
             return false;
         } else if (funcIdent == AlgebricksBuiltinFunctions.AND) {
@@ -400,14 +401,15 @@ public abstract class AbstractIntroduceAccessMethodRule implements IAlgebraicRew
                     continue;
                 }
                 AbstractFunctionCallExpression argFuncExpr = (AbstractFunctionCallExpression) argExpr;
-                boolean matchFound =
-                        analyzeFunctionExpr(argFuncExpr, assignsAndUnnests, analyzedAMs, context, typeEnvironment);
+                boolean matchFound = analyzeFunctionExprAndUpdateAnalyzedAM(argFuncExpr, assignsAndUnnests, analyzedAMs,
+                        context, typeEnvironment);
                 found = found || matchFound;
             }
             return found;
         } else {
             // For single function or "NOT" case:
-            return analyzeFunctionExpr(funcExpr, assignsAndUnnests, analyzedAMs, context, typeEnvironment);
+            return analyzeFunctionExprAndUpdateAnalyzedAM(funcExpr, assignsAndUnnests, analyzedAMs, context,
+                    typeEnvironment);
         }
     }
 
@@ -418,7 +420,7 @@ public abstract class AbstractIntroduceAccessMethodRule implements IAlgebraicRew
      *
      * @throws AlgebricksException
      */
-    protected boolean analyzeFunctionExpr(AbstractFunctionCallExpression funcExpr,
+    protected boolean analyzeFunctionExprAndUpdateAnalyzedAM(AbstractFunctionCallExpression funcExpr,
             List<AbstractLogicalOperator> assignsAndUnnests,
             Map<IAccessMethod, AccessMethodAnalysisContext> analyzedAMs, IOptimizationContext context,
             IVariableTypeEnvironment typeEnvironment) throws AlgebricksException {
@@ -443,8 +445,8 @@ public abstract class AbstractIntroduceAccessMethodRule implements IAlgebraicRew
             }
             // Analyzes the funcExpr's arguments to see if the accessMethod is
             // truly applicable.
-            boolean matchFound = accessMethod.analyzeFuncExprArgs(funcExpr, assignsAndUnnests, analysisCtx, context,
-                    typeEnvironment);
+            boolean matchFound = accessMethod.analyzeFuncExprArgsAndUpdateAnalysisCtx(funcExpr, assignsAndUnnests,
+                    analysisCtx, context, typeEnvironment);
             if (matchFound) {
                 // If we've used the current new context placeholder, replace it
                 // with a new one.
@@ -510,7 +512,7 @@ public abstract class AbstractIntroduceAccessMethodRule implements IAlgebraicRew
                 datasetRecordVar = datasetVars.get(datasetVars.size() - 1);
             }
         }
-        for (IOptimizableFuncExpr optFuncExpr : analysisCtx.matchedFuncExprs) {
+        for (IOptimizableFuncExpr optFuncExpr : analysisCtx.getMatchedFuncExprs()) {
             // Try to match variables from optFuncExpr to assigns or unnests.
             for (int assignOrUnnestIndex = 0; assignOrUnnestIndex < subTree.getAssignsAndUnnests()
                     .size(); assignOrUnnestIndex++) {

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/c4eb6564/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/am/AccessMethodAnalysisContext.java
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/am/AccessMethodAnalysisContext.java b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/am/AccessMethodAnalysisContext.java
index ca5da98..16ee6d1 100644
--- a/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/am/AccessMethodAnalysisContext.java
+++ b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/am/AccessMethodAnalysisContext.java
@@ -19,14 +19,14 @@
 package org.apache.asterix.optimizer.rules.am;
 
 import java.util.ArrayList;
+import java.util.Iterator;
 import java.util.List;
 import java.util.Map;
 import java.util.TreeMap;
 
-import org.apache.commons.lang3.mutable.Mutable;
-
 import org.apache.asterix.metadata.entities.Dataset;
 import org.apache.asterix.metadata.entities.Index;
+import org.apache.commons.lang3.mutable.Mutable;
 import org.apache.hyracks.algebricks.common.utils.Pair;
 import org.apache.hyracks.algebricks.core.algebra.base.ILogicalOperator;
 import org.apache.hyracks.algebricks.core.algebra.expressions.ScalarFunctionCallExpression;
@@ -36,37 +36,70 @@ import org.apache.hyracks.algebricks.core.algebra.expressions.ScalarFunctionCall
  */
 public class AccessMethodAnalysisContext {
 
-    public List<IOptimizableFuncExpr> matchedFuncExprs = new ArrayList<IOptimizableFuncExpr>();
+    private List<IOptimizableFuncExpr> matchedFuncExprs = new ArrayList<IOptimizableFuncExpr>();
 
-    // Contains candidate indexes and a list of (integer,integer) tuples that index into matchedFuncExprs and matched variable inside this expr.
-    // We are mapping from candidate indexes to a list of function expressions
+    // Contains candidate indexes and a list of (integer,integer) tuples that index into matchedFuncExprs and
+    // matched variable inside this expr. We are mapping from candidate indexes to a list of function expressions
     // that match one of the index's expressions.
-    public Map<Index, List<Pair<Integer, Integer>>> indexExprsAndVars = new TreeMap<Index, List<Pair<Integer, Integer>>>();
+    private Map<Index, List<Pair<Integer, Integer>>> indexExprsAndVars =
+            new TreeMap<Index, List<Pair<Integer, Integer>>>();
 
     // Maps from index to the dataset it is indexing.
-    public Map<Index, Dataset> indexDatasetMap = new TreeMap<Index, Dataset>();
+    private Map<Index, Dataset> indexDatasetMap = new TreeMap<Index, Dataset>();
 
     // Maps from an index to the number of matched fields in the query plan (for performing prefix search)
-    public Map<Index, Integer> indexNumMatchedKeys = new TreeMap<Index, Integer>();
+    private Map<Index, Integer> indexNumMatchedKeys = new TreeMap<Index, Integer>();
 
     // variables for resetting null placeholder for left-outer-join
     private Mutable<ILogicalOperator> lojGroupbyOpRef = null;
     private ScalarFunctionCallExpression lojIsNullFuncInGroupBy = null;
 
     public void addIndexExpr(Dataset dataset, Index index, Integer exprIndex, Integer varIndex) {
-        List<Pair<Integer, Integer>> exprs = indexExprsAndVars.get(index);
+        List<Pair<Integer, Integer>> exprs = getIndexExprsFromIndexExprsAndVars(index);
         if (exprs == null) {
             exprs = new ArrayList<Pair<Integer, Integer>>();
-            indexExprsAndVars.put(index, exprs);
+            putIndexExprToIndexExprsAndVars(index, exprs);
         }
         exprs.add(new Pair<Integer, Integer>(exprIndex, varIndex));
-        indexDatasetMap.put(index, dataset);
+        putDatasetIntoIndexDatasetMap(index, dataset);
+    }
+
+    public List<IOptimizableFuncExpr> getMatchedFuncExprs() {
+        return matchedFuncExprs;
+    }
+
+    public IOptimizableFuncExpr getMatchedFuncExpr(int index) {
+        return matchedFuncExprs.get(index);
+    }
+
+    public void addMatchedFuncExpr(IOptimizableFuncExpr optFuncExpr) {
+        matchedFuncExprs.add(optFuncExpr);
     }
 
-    public List<Pair<Integer, Integer>> getIndexExprs(Index index) {
+    public Iterator<Map.Entry<Index, List<Pair<Integer, Integer>>>> getIteratorForIndexExprsAndVars() {
+        return indexExprsAndVars.entrySet().iterator();
+    }
+
+    public boolean isIndexExprsAndVarsEmpty() {
+        return indexExprsAndVars.isEmpty();
+    }
+
+    public List<Pair<Integer, Integer>> getIndexExprsFromIndexExprsAndVars(Index index) {
         return indexExprsAndVars.get(index);
     }
 
+    public void putIndexExprToIndexExprsAndVars(Index index, List<Pair<Integer, Integer>> exprs) {
+        indexExprsAndVars.put(index, exprs);
+    }
+
+    public Integer getNumberOfMatchedKeys(Index index) {
+        return indexNumMatchedKeys.get(index);
+    }
+
+    public void putNumberOfMatchedKeys(Index index, Integer numMatchedKeys) {
+        indexNumMatchedKeys.put(index, numMatchedKeys);
+    }
+
     public void setLOJGroupbyOpRef(Mutable<ILogicalOperator> opRef) {
         lojGroupbyOpRef = opRef;
     }
@@ -83,4 +116,12 @@ public class AccessMethodAnalysisContext {
         return lojIsNullFuncInGroupBy;
     }
 
+    public Dataset getDatasetFromIndexDatasetMap(Index idx) {
+        return indexDatasetMap.get(idx);
+    }
+
+    public void putDatasetIntoIndexDatasetMap(Index idx, Dataset ds) {
+        indexDatasetMap.put(idx, ds);
+    }
+
 }

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/c4eb6564/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/am/AccessMethodUtils.java
----------------------------------------------------------------------
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 7743d9f..e059574 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
@@ -129,9 +129,9 @@ public class AccessMethodUtils {
         return ConstantExpressionUtil.getBooleanConstant(expr.getValue());
     }
 
-    public static boolean analyzeFuncExprArgsForOneConstAndVar(AbstractFunctionCallExpression funcExpr,
-            AccessMethodAnalysisContext analysisCtx, IOptimizationContext context,
-            IVariableTypeEnvironment typeEnvironment) throws AlgebricksException {
+    public static boolean analyzeFuncExprArgsForOneConstAndVarAndUpdateAnalysisCtx(
+            AbstractFunctionCallExpression funcExpr, AccessMethodAnalysisContext analysisCtx,
+            IOptimizationContext context, IVariableTypeEnvironment typeEnvironment) throws AlgebricksException {
         ILogicalExpression constExpression = null;
         IAType constantExpressionType = null;
         LogicalVariable fieldVar = null;
@@ -180,16 +180,38 @@ public class AccessMethodUtils {
         } else {
             return false;
         }
-        OptimizableFuncExpr newOptFuncExpr = new OptimizableFuncExpr(funcExpr, fieldVar, constExpression,
-                constantExpressionType);
-        for (IOptimizableFuncExpr optFuncExpr : analysisCtx.matchedFuncExprs) {
+
+        // Updates the given Analysis Context by adding a new optimizable function expression.
+        constructNewOptFuncExprAndAddToAnalysisCtx(funcExpr, fieldVar, constExpression, constantExpressionType,
+                analysisCtx);
+        return true;
+    }
+
+    private static void constructNewOptFuncExprAndAddToAnalysisCtx(AbstractFunctionCallExpression funcExpr,
+            LogicalVariable fieldVar, ILogicalExpression expression, IAType expressionType,
+            AccessMethodAnalysisContext analysisCtx) {
+        OptimizableFuncExpr newOptFuncExpr =
+                new OptimizableFuncExpr(funcExpr, fieldVar, expression, expressionType);
+        addNewOptFuncExprToAnalysisCtx(funcExpr, newOptFuncExpr, analysisCtx);
+    }
+
+    private static void constructNewOptFuncExprAndAddToAnalysisCtx(AbstractFunctionCallExpression funcExpr,
+            LogicalVariable[] fieldVars, ILogicalExpression[] expressions, IAType[] expressionTypes,
+            AccessMethodAnalysisContext analysisCtx) {
+        OptimizableFuncExpr newOptFuncExpr = new OptimizableFuncExpr(funcExpr, fieldVars, expressions, expressionTypes);
+        addNewOptFuncExprToAnalysisCtx(funcExpr, newOptFuncExpr, analysisCtx);
+
+    }
+
+    private static void addNewOptFuncExprToAnalysisCtx(AbstractFunctionCallExpression funcExpr,
+            OptimizableFuncExpr newOptFuncExpr, AccessMethodAnalysisContext analysisCtx) {
+        for (IOptimizableFuncExpr optFuncExpr : analysisCtx.getMatchedFuncExprs()) {
             //avoid additional optFuncExpressions in case of a join
             if (optFuncExpr.getFuncExpr().equals(funcExpr)) {
-                return true;
+                return;
             }
         }
-        analysisCtx.matchedFuncExprs.add(newOptFuncExpr);
-        return true;
+        analysisCtx.addMatchedFuncExpr(newOptFuncExpr);
     }
 
     /**
@@ -247,7 +269,7 @@ public class AccessMethodUtils {
         }
     }
 
-    public static boolean analyzeFuncExprArgsForTwoVars(AbstractFunctionCallExpression funcExpr,
+    public static boolean analyzeFuncExprArgsForTwoVarsAndUpdateAnalysisCtx(AbstractFunctionCallExpression funcExpr,
             AccessMethodAnalysisContext analysisCtx) {
         LogicalVariable fieldVar1 = null;
         LogicalVariable fieldVar2 = null;
@@ -260,15 +282,10 @@ public class AccessMethodUtils {
         } else {
             return false;
         }
-        OptimizableFuncExpr newOptFuncExpr = new OptimizableFuncExpr(funcExpr,
-                new LogicalVariable[] { fieldVar1, fieldVar2 }, new ILogicalExpression[0], new IAType[0]);
-        for (IOptimizableFuncExpr optFuncExpr : analysisCtx.matchedFuncExprs) {
-            //avoid additional optFuncExpressions in case of a join
-            if (optFuncExpr.getFuncExpr().equals(funcExpr)) {
-                return true;
-            }
-        }
-        analysisCtx.matchedFuncExprs.add(newOptFuncExpr);
+
+        // Updates the given Analysis Context by adding a new optimizable function expression.
+        constructNewOptFuncExprAndAddToAnalysisCtx(funcExpr, new LogicalVariable[] { fieldVar1, fieldVar2 },
+                new ILogicalExpression[0], new IAType[0], analysisCtx);
         return true;
     }
 
@@ -451,13 +468,13 @@ public class AccessMethodUtils {
      */
     public static IOptimizableFuncExpr chooseFirstOptFuncExpr(Index chosenIndex,
             AccessMethodAnalysisContext analysisCtx) {
-        List<Pair<Integer, Integer>> indexExprs = analysisCtx.getIndexExprs(chosenIndex);
+        List<Pair<Integer, Integer>> indexExprs = analysisCtx.getIndexExprsFromIndexExprsAndVars(chosenIndex);
         int firstExprIndex = indexExprs.get(0).first;
-        return analysisCtx.matchedFuncExprs.get(firstExprIndex);
+        return analysisCtx.getMatchedFuncExpr(firstExprIndex);
     }
 
     public static int chooseFirstOptFuncVar(Index chosenIndex, AccessMethodAnalysisContext analysisCtx) {
-        List<Pair<Integer, Integer>> indexExprs = analysisCtx.getIndexExprs(chosenIndex);
+        List<Pair<Integer, Integer>> indexExprs = analysisCtx.getIndexExprsFromIndexExprsAndVars(chosenIndex);
         return indexExprs.get(0).second;
     }
 

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/c4eb6564/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/am/BTreeAccessMethod.java
----------------------------------------------------------------------
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 3acf1f5..0f6012f 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
@@ -99,14 +99,14 @@ public class BTreeAccessMethod implements IAccessMethod {
     }
 
     @Override
-    public boolean analyzeFuncExprArgs(AbstractFunctionCallExpression funcExpr,
+    public boolean analyzeFuncExprArgsAndUpdateAnalysisCtx(AbstractFunctionCallExpression funcExpr,
             List<AbstractLogicalOperator> assignsAndUnnests, AccessMethodAnalysisContext analysisCtx,
             IOptimizationContext context, IVariableTypeEnvironment typeEnvironment) throws AlgebricksException {
         boolean matches =
-                AccessMethodUtils.analyzeFuncExprArgsForOneConstAndVar(
+                AccessMethodUtils.analyzeFuncExprArgsForOneConstAndVarAndUpdateAnalysisCtx(
                         funcExpr, analysisCtx, context, typeEnvironment);
         if (!matches) {
-            matches = AccessMethodUtils.analyzeFuncExprArgsForTwoVars(funcExpr, analysisCtx);
+            matches = AccessMethodUtils.analyzeFuncExprArgsForTwoVarsAndUpdateAnalysisCtx(funcExpr, analysisCtx);
         }
         return matches;
     }
@@ -175,7 +175,7 @@ public class BTreeAccessMethod implements IAccessMethod {
         Mutable<ILogicalExpression> conditionRef = joinOp.getCondition();
         // Determine if the index is applicable on the left or right side
         // (if both, we arbitrarily prefer the left side).
-        Dataset dataset = analysisCtx.indexDatasetMap.get(chosenIndex);
+        Dataset dataset = analysisCtx.getDatasetFromIndexDatasetMap(chosenIndex);
         OptimizableOperatorSubTree indexSubTree;
         OptimizableOperatorSubTree probeSubTree;
         // We assume that the left subtree is the outer branch and the right subtree is the inner branch.
@@ -225,17 +225,16 @@ public class BTreeAccessMethod implements IAccessMethod {
     @Override
     public ILogicalOperator createSecondaryToPrimaryPlan(Mutable<ILogicalExpression> conditionRef,
             OptimizableOperatorSubTree indexSubTree, OptimizableOperatorSubTree probeSubTree, Index chosenIndex,
-            AccessMethodAnalysisContext analysisCtx, boolean retainInput, boolean retainNull,
-            boolean requiresBroadcast, IOptimizationContext context) throws AlgebricksException {
+            AccessMethodAnalysisContext analysisCtx, boolean retainInput, boolean retainNull, boolean requiresBroadcast,
+            IOptimizationContext context) throws AlgebricksException {
         Dataset dataset = indexSubTree.getDataset();
         ARecordType recordType = indexSubTree.getRecordType();
         ARecordType metaRecordType = indexSubTree.getMetaRecordType();
         // we made sure indexSubTree has datasource scan
         AbstractDataSourceOperator dataSourceOp =
                 (AbstractDataSourceOperator) indexSubTree.getDataSourceRef().getValue();
-        List<Pair<Integer, Integer>> exprAndVarList = analysisCtx.indexExprsAndVars.get(chosenIndex);
-        List<IOptimizableFuncExpr> matchedFuncExprs = analysisCtx.matchedFuncExprs;
-        int numSecondaryKeys = analysisCtx.indexNumMatchedKeys.get(chosenIndex);
+        List<Pair<Integer, Integer>> exprAndVarList = analysisCtx.getIndexExprsFromIndexExprsAndVars(chosenIndex);
+        int numSecondaryKeys = analysisCtx.getNumberOfMatchedKeys(chosenIndex);
         // List of function expressions that will be replaced by the secondary-index search.
         // These func exprs will be removed from the select condition at the very end of this method.
         Set<ILogicalExpression> replacedFuncExprs = new HashSet<>();
@@ -267,15 +266,14 @@ public class BTreeAccessMethod implements IAccessMethod {
 
         for (Pair<Integer, Integer> exprIndex : exprAndVarList) {
             // Position of the field of matchedFuncExprs.get(exprIndex) in the chosen index's indexed exprs.
-            IOptimizableFuncExpr optFuncExpr = matchedFuncExprs.get(exprIndex.first);
+            IOptimizableFuncExpr optFuncExpr = analysisCtx.getMatchedFuncExpr(exprIndex.first);
             int keyPos = indexOf(optFuncExpr.getFieldName(0), chosenIndex.getKeyFieldNames());
             if (keyPos < 0 && optFuncExpr.getNumLogicalVars() > 1) {
                 // If we are optimizing a join, the matching field may be the second field name.
                 keyPos = indexOf(optFuncExpr.getFieldName(1), chosenIndex.getKeyFieldNames());
             }
             if (keyPos < 0) {
-                throw new AlgebricksException(
-                        "Could not match optimizable function expression to any index field name.");
+                throw CompilationException.create(ErrorCode.NO_INDEX_FIELD_NAME_FOR_GIVEN_FUNC_EXPR);
             }
             Pair<ILogicalExpression, Boolean> returnedSearchKeyExpr =
                     AccessMethodUtils.createSearchKeyExpr(optFuncExpr, indexSubTree, probeSubTree);
@@ -420,7 +418,7 @@ public class BTreeAccessMethod implements IAccessMethod {
             }
             if (!couldntFigureOut) {
                 // Remember to remove this funcExpr later.
-                replacedFuncExprs.add(matchedFuncExprs.get(exprIndex.first).getFuncExpr());
+                replacedFuncExprs.add(analysisCtx.getMatchedFuncExpr(exprIndex.first).getFuncExpr());
             }
             if (doneWithExprs) {
                 break;

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/c4eb6564/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/am/IAccessMethod.java
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/am/IAccessMethod.java b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/am/IAccessMethod.java
index d249b96..870b425 100644
--- a/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/am/IAccessMethod.java
+++ b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/am/IAccessMethod.java
@@ -57,7 +57,7 @@ public interface IAccessMethod extends Comparable<IAccessMethod> {
      *         otherwise
      * @throws AlgebricksException
      */
-    boolean analyzeFuncExprArgs(AbstractFunctionCallExpression funcExpr,
+    boolean analyzeFuncExprArgsAndUpdateAnalysisCtx(AbstractFunctionCallExpression funcExpr,
             List<AbstractLogicalOperator> assignsAndUnnests, AccessMethodAnalysisContext analysisCtx,
             IOptimizationContext context, IVariableTypeEnvironment typeEnvironment) throws AlgebricksException;
 

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/c4eb6564/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/am/IntroduceJoinAccessMethodRule.java
----------------------------------------------------------------------
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 73bbbfc..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
@@ -23,7 +23,8 @@ import java.util.Iterator;
 import java.util.List;
 import java.util.Map;
 
-import org.apache.asterix.metadata.declared.MetadataProvider;
+import org.apache.asterix.algebra.operators.CommitOperator;
+import org.apache.asterix.metadata.entities.Dataset;
 import org.apache.asterix.metadata.entities.Index;
 import org.apache.commons.lang3.mutable.Mutable;
 import org.apache.hyracks.algebricks.common.exceptions.AlgebricksException;
@@ -39,6 +40,7 @@ import org.apache.hyracks.algebricks.core.algebra.expressions.ScalarFunctionCall
 import org.apache.hyracks.algebricks.core.algebra.functions.FunctionIdentifier;
 import org.apache.hyracks.algebricks.core.algebra.operators.logical.AbstractBinaryJoinOperator;
 import org.apache.hyracks.algebricks.core.algebra.operators.logical.AbstractLogicalOperator;
+import org.apache.hyracks.algebricks.core.algebra.operators.logical.DelegateOperator;
 import org.apache.hyracks.algebricks.core.algebra.operators.logical.GroupByOperator;
 import org.apache.hyracks.algebricks.core.algebra.operators.logical.InnerJoinOperator;
 import org.apache.hyracks.algebricks.core.algebra.operators.logical.LeftOuterJoinOperator;
@@ -53,7 +55,8 @@ import org.apache.hyracks.algebricks.core.algebra.util.OperatorPropertiesUtil;
  * This rule tries to utilize an index on the inner relation.
  * If that's not possible, it stops transforming the given join into an index-nested-loop join.
  * Replaces the above pattern with the following simplified plan:
- * (select) <-- (assign) <-- (btree search) <-- (sort) <-- (unnest(index search)) <-- (assign) <-- (datasource scan | unnest-map)
+ * (select) <-- (assign) <-- (btree search) <-- (sort) <-- (unnest(index search)) <-- (assign) <-- (A)
+ * (A) <-- (datasource scan | unnest-map)
  * The sort is optional, and some access methods may choose not to sort.
  * Note that for some index-based optimizations we do not remove the triggering
  * condition from the join, since the secondary index may only act as a filter, and the
@@ -76,7 +79,7 @@ import org.apache.hyracks.algebricks.core.algebra.util.OperatorPropertiesUtil;
 public class IntroduceJoinAccessMethodRule extends AbstractIntroduceAccessMethodRule {
 
     protected Mutable<ILogicalOperator> joinRef = null;
-    protected AbstractBinaryJoinOperator join = null;
+    protected AbstractBinaryJoinOperator joinOp = null;
     protected AbstractFunctionCallExpression joinCond = null;
     protected final OptimizableOperatorSubTree leftSubTree = new OptimizableOperatorSubTree();
     protected final OptimizableOperatorSubTree rightSubTree = new OptimizableOperatorSubTree();
@@ -85,7 +88,7 @@ public class IntroduceJoinAccessMethodRule extends AbstractIntroduceAccessMethod
     protected boolean hasGroupBy = true;
 
     // Register access methods.
-    protected static Map<FunctionIdentifier, List<IAccessMethod>> accessMethods = new HashMap<FunctionIdentifier, List<IAccessMethod>>();
+    protected static Map<FunctionIdentifier, List<IAccessMethod>> accessMethods = new HashMap<>();
 
     static {
         registerAccessMethod(BTreeAccessMethod.INSTANCE, accessMethods);
@@ -93,90 +96,61 @@ public class IntroduceJoinAccessMethodRule extends AbstractIntroduceAccessMethod
         registerAccessMethod(InvertedIndexAccessMethod.INSTANCE, accessMethods);
     }
 
+    /**
+     * Recursively check the given plan from the root operator to transform a plan
+     * with JOIN or LEFT-OUTER-JOIN operator into an index-utilized plan.
+     */
+
     @Override
-    public boolean rewritePost(Mutable<ILogicalOperator> opRef, IOptimizationContext context)
+    public boolean rewritePre(Mutable<ILogicalOperator> opRef, IOptimizationContext context)
             throws AlgebricksException {
         clear();
         setMetadataDeclarations(context);
 
-        // Match operator pattern and initialize optimizable sub trees.
-        if (!matchesOperatorPattern(opRef, context)) {
-            return false;
-        }
-        // Analyze condition on those optimizable subtrees that have a datasource scan.
-        Map<IAccessMethod, AccessMethodAnalysisContext> analyzedAMs = new HashMap<IAccessMethod, AccessMethodAnalysisContext>();
-        boolean matchInLeftSubTree = false;
-        boolean matchInRightSubTree = false;
-        if (leftSubTree.hasDataSource()) {
-            matchInLeftSubTree = analyzeCondition(joinCond, leftSubTree.getAssignsAndUnnests(), analyzedAMs, context,
-                    typeEnvironment);
-        }
-        if (rightSubTree.hasDataSource()) {
-            matchInRightSubTree = analyzeCondition(joinCond, rightSubTree.getAssignsAndUnnests(), analyzedAMs, context,
-                    typeEnvironment);
-        }
-        if (!matchInLeftSubTree && !matchInRightSubTree) {
+        AbstractLogicalOperator op = (AbstractLogicalOperator) opRef.getValue();
+
+        // Already checked?
+        if (context.checkIfInDontApplySet(this, op)) {
             return false;
         }
 
-        // Set dataset and type metadata.
-        MetadataProvider metadataProvider = (MetadataProvider) context.getMetadataProvider();
-        boolean checkLeftSubTreeMetadata = false;
-        boolean checkRightSubTreeMetadata = false;
-        if (matchInLeftSubTree) {
-            checkLeftSubTreeMetadata = leftSubTree.setDatasetAndTypeMetadata(metadataProvider);
-        }
-        if (matchInRightSubTree) {
-            checkRightSubTreeMetadata = rightSubTree.setDatasetAndTypeMetadata(metadataProvider);
-        }
-        if (!checkLeftSubTreeMetadata && !checkRightSubTreeMetadata) {
+        // Check whether this operator is the root, which is DISTRIBUTE_RESULT or SINK since
+        // we start the process from the root operator.
+        if (op.getOperatorTag() != LogicalOperatorTag.DISTRIBUTE_RESULT
+                && op.getOperatorTag() != LogicalOperatorTag.SINK
+                && op.getOperatorTag() != LogicalOperatorTag.DELEGATE_OPERATOR) {
             return false;
         }
-        if (checkLeftSubTreeMetadata) {
-            fillSubTreeIndexExprs(leftSubTree, analyzedAMs, context);
-        }
-        if (checkRightSubTreeMetadata) {
-            fillSubTreeIndexExprs(rightSubTree, analyzedAMs, context);
-        }
-        pruneIndexCandidates(analyzedAMs, context, typeEnvironment);
 
-        // We only consider indexes from the inner branch (right subTree).
-        // If no index is available, then we stop this optimization.
-        removeIndexCandidatesFromOuterBranch(analyzedAMs);
-
-        // Choose an index from the inner branch that will be used.
-        Pair<IAccessMethod, Index> chosenIndex = chooseBestIndex(analyzedAMs);
-        if (chosenIndex == null) {
-            context.addToDontApplySet(this, join);
+        if (op.getOperatorTag() == LogicalOperatorTag.DELEGATE_OPERATOR
+                && !(((DelegateOperator) op).getDelegate() instanceof CommitOperator)) {
             return false;
         }
 
-        // Apply plan transformation using chosen index.
-        AccessMethodAnalysisContext analysisCtx = analyzedAMs.get(chosenIndex.first);
+        // Recursively check the given plan whether the desired pattern exists in it.
+        // If so, try to optimize the plan.
+        boolean planTransformed = checkAndApplyJoinTransformation(opRef, context);
 
-        //For LOJ with GroupBy, prepare objects to reset LOJ nullPlaceHolderVariable in GroupByOp
-        if (isLeftOuterJoin && hasGroupBy) {
-            analysisCtx.setLOJGroupbyOpRef(opRef);
-            ScalarFunctionCallExpression isNullFuncExpr = AccessMethodUtils
-                    .findLOJIsMissingFuncInGroupBy((GroupByOperator) opRef.getValue());
-            analysisCtx.setLOJIsNullFuncInGroupBy(isNullFuncExpr);
+        if (joinOp != null) {
+            // We found an optimization here. Don't need to optimize this operator again.
+            context.addToDontApplySet(this, joinOp);
         }
 
-        // At this point, we are sure that only an index from the inner branch is going to be used.
-        // So, the left subtree is the outer branch and the right subtree is the inner branch.
-        boolean res = chosenIndex.first.applyJoinPlanTransformation(joinRef, leftSubTree, rightSubTree,
-                chosenIndex.second, analysisCtx, context, isLeftOuterJoin, hasGroupBy);
-        if (res) {
+        if (!planTransformed) {
+            return false;
+        } else {
             OperatorPropertiesUtil.typeOpRec(opRef, context);
         }
-        context.addToDontApplySet(this, join);
-        return res;
+
+        return planTransformed;
     }
 
     /**
-     * Removes indexes from the optimizer's consideration for this rule.
+     * Removes indexes from the outer branch from the optimizer's consideration for this rule,
+     * since we only use indexes from the inner branch.
      */
-    protected void removeIndexCandidatesFromOuterBranch(Map<IAccessMethod, AccessMethodAnalysisContext> analyzedAMs) {
+    protected void pruneIndexCandidatesFromOuterBranch(Map<IAccessMethod, AccessMethodAnalysisContext> analyzedAMs) {
+        // Inner branch is the right side branch of the given JOIN operator.
         String innerDataset = null;
         if (rightSubTree.getDataset() != null) {
             innerDataset = rightSubTree.getDataset().getDatasetName();
@@ -188,8 +162,7 @@ public class IntroduceJoinAccessMethodRule extends AbstractIntroduceAccessMethod
             AccessMethodAnalysisContext amCtx = entry.getValue();
 
             // Fetch index, expression, and variables.
-            Iterator<Map.Entry<Index, List<Pair<Integer, Integer>>>> indexIt = amCtx.indexExprsAndVars.entrySet()
-                    .iterator();
+            Iterator<Map.Entry<Index, List<Pair<Integer, Integer>>>> indexIt = amCtx.getIteratorForIndexExprsAndVars();
 
             while (indexIt.hasNext()) {
                 Map.Entry<Index, List<Pair<Integer, Integer>>> indexExprAndVarEntry = indexIt.next();
@@ -198,10 +171,10 @@ public class IntroduceJoinAccessMethodRule extends AbstractIntroduceAccessMethod
 
                 while (exprsAndVarIter.hasNext()) {
                     Pair<Integer, Integer> exprAndVarIdx = exprsAndVarIter.next();
-                    IOptimizableFuncExpr optFuncExpr = amCtx.matchedFuncExprs.get(exprAndVarIdx.first);
+                    IOptimizableFuncExpr optFuncExpr = amCtx.getMatchedFuncExpr(exprAndVarIdx.first);
 
-                    // Does this index come from the inner branch?
-                    // We check the dataset name and the subtree to make sure the index is applicable.
+                    // We check the dataset name and the subtree to make sure
+                    // that this index come from the inner branch.
                     if (indexExprAndVarEntry.getKey().getDatasetName().equals(innerDataset)) {
                         if (optFuncExpr.getOperatorSubTree(exprAndVarIdx.second).equals(rightSubTree)) {
                             indexFromInnerBranch = true;
@@ -209,8 +182,8 @@ public class IntroduceJoinAccessMethodRule extends AbstractIntroduceAccessMethod
                     }
                 }
 
-                // If the index does not come from the inner branch,
-                // We do not consider this index.
+                // If the given index does not come from the inner branch,
+                // prune this index so that the optimizer doesn't consider this index in this rule.
                 if (!indexFromInnerBranch) {
                     indexIt.remove();
                 }
@@ -218,50 +191,11 @@ public class IntroduceJoinAccessMethodRule extends AbstractIntroduceAccessMethod
         }
     }
 
-    protected boolean matchesOperatorPattern(Mutable<ILogicalOperator> opRef, IOptimizationContext context)
-            throws AlgebricksException {
-        // First check that the operator is a join and its condition is a function call.
-        AbstractLogicalOperator op1 = (AbstractLogicalOperator) opRef.getValue();
-        if (context.checkIfInDontApplySet(this, op1)) {
-            return false;
-        }
-
-        boolean isInnerJoin = isInnerJoin(op1);
-        isLeftOuterJoin = isLeftOuterJoin(op1);
-
-        if (!isInnerJoin && !isLeftOuterJoin) {
-            return false;
-        }
-
-        // Set and analyze select.
-        if (isInnerJoin) {
-            joinRef = opRef;
-            join = (InnerJoinOperator) op1;
-        } else {
-            joinRef = op1.getInputs().get(0);
-            join = (LeftOuterJoinOperator) joinRef.getValue();
-        }
-
-        typeEnvironment = context.getOutputTypeEnvironment(join);
-        // Check that the select's condition is a function call.
-        ILogicalExpression condExpr = join.getCondition().getValue();
-        if (condExpr.getExpressionTag() != LogicalExpressionTag.FUNCTION_CALL) {
-            return false;
-        }
-        joinCond = (AbstractFunctionCallExpression) condExpr;
-        boolean leftSubTreeInitialized = leftSubTree.initFromSubTree(join.getInputs().get(0));
-        boolean rightSubTreeInitialized = rightSubTree.initFromSubTree(join.getInputs().get(1));
-        if (!leftSubTreeInitialized || !rightSubTreeInitialized) {
-            return false;
-        }
-
-        // One of the subtrees must have a datasource scan.
-        if (leftSubTree.hasDataSourceScan() || rightSubTree.hasDataSourceScan()) {
-            return true;
-        }
-        return false;
-    }
-
+    /**
+     * Checks whether the given operator is LEFTOUTERJOIN.
+     * If so, also checks that GROUPBY is placed after LEFTOUTERJOIN.
+     */
+    // Check whether (Groupby)? <-- Leftouterjoin
     private boolean isLeftOuterJoin(AbstractLogicalOperator op1) {
         if (op1.getInputs().size() != 1) {
             return false;
@@ -277,6 +211,9 @@ public class IntroduceJoinAccessMethodRule extends AbstractIntroduceAccessMethod
         return true;
     }
 
+    /**
+     * Checks whether the given operator is INNERJOIN.
+     */
     private boolean isInnerJoin(AbstractLogicalOperator op1) {
         return op1.getOperatorTag() == LogicalOperatorTag.INNERJOIN;
     }
@@ -288,8 +225,207 @@ public class IntroduceJoinAccessMethodRule extends AbstractIntroduceAccessMethod
 
     private void clear() {
         joinRef = null;
-        join = null;
+        joinOp = null;
         joinCond = null;
         isLeftOuterJoin = false;
     }
+
+    /**
+     * Recursively traverse the given plan and check whether a INNERJOIN or LEFTOUTERJOIN operator exists.
+     * If one is found, maintain the path from the root to the given join operator and
+     * optimize the path from the given join operator to the EMPTY_TUPLE_SOURCE operator
+     * if it is not already optimized.
+     */
+    protected boolean checkAndApplyJoinTransformation(Mutable<ILogicalOperator> opRef, IOptimizationContext context)
+            throws AlgebricksException {
+        AbstractLogicalOperator op = (AbstractLogicalOperator) opRef.getValue();
+        boolean joinFoundAndOptimizationApplied;
+
+        // Check the current operator pattern to see whether it is a JOIN or not.
+        boolean isThisOpInnerJoin = isInnerJoin(op);
+        boolean isThisOpLeftOuterJoin = isLeftOuterJoin(op);
+        boolean isParentOpGroupBy = hasGroupBy;
+
+        Mutable<ILogicalOperator> joinRefFromThisOp = null;
+        AbstractBinaryJoinOperator joinOpFromThisOp = null;
+
+        if (isThisOpInnerJoin) {
+            // Set join operator.
+            joinRef = opRef;
+            joinOp = (InnerJoinOperator) op;
+            joinRefFromThisOp = opRef;
+            joinOpFromThisOp = (InnerJoinOperator) op;
+        } else if (isThisOpLeftOuterJoin) {
+            // Set left-outer-join op.
+            // The current operator is GROUP and the child of this op is LEFTOUERJOIN.
+            joinRef = op.getInputs().get(0);
+            joinOp = (LeftOuterJoinOperator) joinRef.getValue();
+            joinRefFromThisOp = op.getInputs().get(0);
+            joinOpFromThisOp = (LeftOuterJoinOperator) joinRefFromThisOp.getValue();
+        }
+
+        // Recursively check the plan and try to optimize it. We first check the children of the given operator
+        // to make sure an earlier join in the path is optimized first.
+        for (Mutable<ILogicalOperator> inputOpRef : op.getInputs()) {
+            joinFoundAndOptimizationApplied = checkAndApplyJoinTransformation(inputOpRef, context);
+            if (joinFoundAndOptimizationApplied) {
+                return true;
+            }
+        }
+
+        // For a JOIN case, try to transform the given plan.
+        if (isThisOpInnerJoin || isThisOpLeftOuterJoin) {
+            // Restore the information from this operator since it might have been be set to null
+            // if there are other join operators in the earlier path.
+            joinRef = joinRefFromThisOp;
+            joinOp = joinOpFromThisOp;
+
+            boolean continueCheck = true;
+
+            // Already checked? If not, this operator may be optimized.
+            if (context.checkIfInDontApplySet(this, joinOp)) {
+                continueCheck = false;
+            }
+
+            // For each access method, this contains the information about
+            // whether an available index can be applicable or not.
+            Map<IAccessMethod, AccessMethodAnalysisContext> analyzedAMs = null;
+            if (continueCheck) {
+                analyzedAMs = new HashMap<>();
+            }
+
+            // Check the condition of JOIN operator is a function call since only function call can be transformed
+            // using available indexes. If so, initialize the subtree information that will be used later to decide
+            // whether the given plan is truly optimizable or not.
+            if (continueCheck && !checkJoinOpConditionAndInitSubTree(context)) {
+                continueCheck = false;
+            }
+
+            // Analyze the condition of SELECT operator and initialize analyzedAMs.
+            // Check whether the function in the SELECT operator can be truly transformed.
+            boolean matchInLeftSubTree = false;
+            boolean matchInRightSubTree = false;
+            if (continueCheck) {
+                if (leftSubTree.hasDataSource()) {
+                    matchInLeftSubTree = analyzeSelectOrJoinOpConditionAndUpdateAnalyzedAM(joinCond,
+                            leftSubTree.getAssignsAndUnnests(), analyzedAMs, context, typeEnvironment);
+                }
+                if (rightSubTree.hasDataSource()) {
+                    matchInRightSubTree = analyzeSelectOrJoinOpConditionAndUpdateAnalyzedAM(joinCond,
+                            rightSubTree.getAssignsAndUnnests(), analyzedAMs, context, typeEnvironment);
+                }
+            }
+
+            // Find the dataset from the data-source and the record type of the dataset from the metadata.
+            // This will be used to find an applicable index on the dataset.
+            boolean checkLeftSubTreeMetadata = false;
+            boolean checkRightSubTreeMetadata = false;
+            if (continueCheck && (matchInLeftSubTree || matchInRightSubTree)) {
+                // Set dataset and type metadata.
+                if (matchInLeftSubTree) {
+                    checkLeftSubTreeMetadata = leftSubTree.setDatasetAndTypeMetadata(metadataProvider);
+                }
+                if (matchInRightSubTree) {
+                    checkRightSubTreeMetadata = rightSubTree.setDatasetAndTypeMetadata(metadataProvider);
+                }
+            }
+
+            if (continueCheck && (checkLeftSubTreeMetadata || checkRightSubTreeMetadata)) {
+                // Map variables to the applicable indexes and find the field name and type.
+                // Then find the applicable indexes for the variables used in the JOIN condition.
+                if (checkLeftSubTreeMetadata) {
+                    fillSubTreeIndexExprs(leftSubTree, analyzedAMs, context);
+                }
+                if (checkRightSubTreeMetadata) {
+                    fillSubTreeIndexExprs(rightSubTree, analyzedAMs, context);
+                }
+
+                // Prune the access methods based on the function expression and access methods.
+                pruneIndexCandidates(analyzedAMs, context, typeEnvironment);
+
+                // If the right subtree (inner branch) has indexes, one of those indexes will be used.
+                // Remove the indexes from the outer branch in the optimizer's consideration list for this rule.
+                pruneIndexCandidatesFromOuterBranch(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);
+                if (chosenIndex == null) {
+                    context.addToDontApplySet(this, joinOp);
+                    continueCheck = false;
+                }
+
+                if (continueCheck) {
+                    // Apply plan transformation using chosen index.
+                    AccessMethodAnalysisContext analysisCtx = analyzedAMs.get(chosenIndex.first);
+
+                    // For LOJ with GroupBy, prepare objects to reset LOJ nullPlaceHolderVariable
+                    // in GroupByOp.
+                    if (isThisOpLeftOuterJoin && isParentOpGroupBy) {
+                        analysisCtx.setLOJGroupbyOpRef(opRef);
+                        ScalarFunctionCallExpression isNullFuncExpr = AccessMethodUtils
+                                .findLOJIsMissingFuncInGroupBy((GroupByOperator) opRef.getValue());
+                        analysisCtx.setLOJIsNullFuncInGroupBy(isNullFuncExpr);
+                    }
+
+                    Dataset indexDataset = analysisCtx.getDatasetFromIndexDatasetMap(chosenIndex.second);
+
+                    // We assume that the left subtree is the outer branch and the right subtree
+                    // is the inner branch. This assumption holds true since we only use an index
+                    // from the right subtree. The following is just a sanity check.
+                    if (!rightSubTree.hasDataSourceScan()
+                            && !indexDataset.getDatasetName().equals(rightSubTree.getDataset().getDatasetName())) {
+                        return false;
+                    }
+
+                    // Finally, try to apply plan transformation using chosen index.
+                    boolean res = chosenIndex.first.applyJoinPlanTransformation(joinRef, leftSubTree, rightSubTree,
+                            chosenIndex.second, analysisCtx, context, isThisOpLeftOuterJoin, isParentOpGroupBy);
+
+                    // If the plan transformation is successful, we don't need to traverse the plan
+                    // any more, since if there are more JOIN operators, the next trigger on this plan
+                    // will find them.
+                    if (res) {
+                        return res;
+                    }
+                }
+            }
+
+            joinRef = null;
+            joinOp = null;
+        }
+
+        return false;
+    }
+
+    /**
+     * After the pattern is matched, check the condition and initialize the data sources from the both sub trees.
+     *
+     * @throws AlgebricksException
+     */
+    protected boolean checkJoinOpConditionAndInitSubTree(IOptimizationContext context) throws AlgebricksException {
+
+        typeEnvironment = context.getOutputTypeEnvironment(joinOp);
+
+        // Check that the join's condition is a function call.
+        ILogicalExpression condExpr = joinOp.getCondition().getValue();
+        if (condExpr.getExpressionTag() != LogicalExpressionTag.FUNCTION_CALL) {
+            return false;
+        }
+        joinCond = (AbstractFunctionCallExpression) condExpr;
+
+        boolean leftSubTreeInitialized = leftSubTree.initFromSubTree(joinOp.getInputs().get(0));
+        boolean rightSubTreeInitialized = rightSubTree.initFromSubTree(joinOp.getInputs().get(1));
+
+        if (!leftSubTreeInitialized || !rightSubTreeInitialized) {
+            return false;
+        }
+
+        // One of the subtrees must have a datasource scan.
+        if (leftSubTree.hasDataSourceScan() || rightSubTree.hasDataSourceScan()) {
+            return true;
+        }
+        return false;
+    }
+
 }

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/c4eb6564/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/am/IntroduceLSMComponentFilterRule.java
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/am/IntroduceLSMComponentFilterRule.java b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/am/IntroduceLSMComponentFilterRule.java
index 5f9b4ab..18641e5 100644
--- a/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/am/IntroduceLSMComponentFilterRule.java
+++ b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/am/IntroduceLSMComponentFilterRule.java
@@ -83,7 +83,7 @@ public class IntroduceLSMComponentFilterRule implements IAlgebraicRewriteRule {
         typeEnvironment = context.getOutputTypeEnvironment(op);
         ILogicalExpression condExpr = ((SelectOperator) op).getCondition().getValue();
         AccessMethodAnalysisContext analysisCtx = analyzeCondition(condExpr, context, typeEnvironment);
-        if (analysisCtx.matchedFuncExprs.isEmpty()) {
+        if (analysisCtx.getMatchedFuncExprs().isEmpty()) {
             return false;
         }
 
@@ -106,8 +106,8 @@ public class IntroduceLSMComponentFilterRule implements IAlgebraicRewriteRule {
 
         List<IOptimizableFuncExpr> optFuncExprs = new ArrayList<>();
 
-        for (int i = 0; i < analysisCtx.matchedFuncExprs.size(); i++) {
-            IOptimizableFuncExpr optFuncExpr = analysisCtx.matchedFuncExprs.get(i);
+        for (int i = 0; i < analysisCtx.getMatchedFuncExprs().size(); i++) {
+            IOptimizableFuncExpr optFuncExpr = analysisCtx.getMatchedFuncExpr(i);
             boolean found = findMacthedExprFieldName(optFuncExpr, op, dataset, recType, datasetIndexes, context);
             if (found && optFuncExpr.getFieldName(0).equals(filterFieldName)) {
                 optFuncExprs.add(optFuncExpr);
@@ -299,7 +299,8 @@ public class IntroduceLSMComponentFilterRule implements IAlgebraicRewriteRule {
         if (funcIdent == AlgebricksBuiltinFunctions.LE || funcIdent == AlgebricksBuiltinFunctions.GE
                 || funcIdent == AlgebricksBuiltinFunctions.LT || funcIdent == AlgebricksBuiltinFunctions.GT
                 || funcIdent == AlgebricksBuiltinFunctions.EQ) {
-            AccessMethodUtils.analyzeFuncExprArgsForOneConstAndVar(funcExpr, analysisCtx, context, typeEnvironment);
+            AccessMethodUtils.analyzeFuncExprArgsForOneConstAndVarAndUpdateAnalysisCtx(funcExpr, analysisCtx, context,
+                    typeEnvironment);
         }
     }
 

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/c4eb6564/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/am/IntroduceSelectAccessMethodRule.java
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/am/IntroduceSelectAccessMethodRule.java b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/am/IntroduceSelectAccessMethodRule.java
index 1d332b6..d95b278 100644
--- a/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/am/IntroduceSelectAccessMethodRule.java
+++ b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/am/IntroduceSelectAccessMethodRule.java
@@ -89,7 +89,7 @@ public class IntroduceSelectAccessMethodRule extends AbstractIntroduceAccessMeth
     protected List<Mutable<ILogicalOperator>> afterSelectRefs = null;
 
     // Register access methods.
-    protected static Map<FunctionIdentifier, List<IAccessMethod>> accessMethods = new HashMap<FunctionIdentifier, List<IAccessMethod>>();
+    protected static Map<FunctionIdentifier, List<IAccessMethod>> accessMethods = new HashMap<>();
 
     static {
         registerAccessMethod(BTreeAccessMethod.INSTANCE, accessMethods);
@@ -102,32 +102,35 @@ public class IntroduceSelectAccessMethodRule extends AbstractIntroduceAccessMeth
      * with SELECT operator into an index-utilized plan.
      */
     @Override
-    public boolean rewritePost(Mutable<ILogicalOperator> opRef, IOptimizationContext context)
+    public boolean rewritePre(Mutable<ILogicalOperator> opRef, IOptimizationContext context)
             throws AlgebricksException {
         clear();
         setMetadataDeclarations(context);
 
         AbstractLogicalOperator op = (AbstractLogicalOperator) opRef.getValue();
+
+        // Already checked?
         if (context.checkIfInDontApplySet(this, op)) {
             return false;
         }
 
-        //We start at the top of the plan
+        // We start at the top of the plan. Thus, check whether this operator is the root,
+        // which is DISTRIBUTE_RESULT, SINK, or COMMIT since we start the process from the root operator.
         if (op.getOperatorTag() != LogicalOperatorTag.DISTRIBUTE_RESULT
                 && op.getOperatorTag() != LogicalOperatorTag.SINK
                 && op.getOperatorTag() != LogicalOperatorTag.DELEGATE_OPERATOR) {
             return false;
         }
+
         if (op.getOperatorTag() == LogicalOperatorTag.DELEGATE_OPERATOR
                 && !(((DelegateOperator) op).getDelegate() instanceof CommitOperator)) {
             return false;
         }
 
         afterSelectRefs = new ArrayList<>();
-
         // Recursively check the given plan whether the desired pattern exists in it.
         // If so, try to optimize the plan.
-        boolean planTransformed = checkAndApplyTheSelectTransformationRule(opRef, context);
+        boolean planTransformed = checkAndApplyTheSelectTransformation(opRef, context);
 
         if (selectOp != null) {
             // We found an optimization here. Don't need to optimize this operator again.
@@ -135,78 +138,44 @@ public class IntroduceSelectAccessMethodRule extends AbstractIntroduceAccessMeth
         }
 
         if (!planTransformed) {
-            // We found an optimization here. Don't need to optimize this operator again.
             return false;
         } else {
             OperatorPropertiesUtil.typeOpRec(opRef, context);
-
         }
 
         return planTransformed;
     }
 
-    protected boolean checkAndApplyTheSelectTransformationRule(Mutable<ILogicalOperator> opRef,
-            IOptimizationContext context) throws AlgebricksException {
-        AbstractLogicalOperator op = (AbstractLogicalOperator) opRef.getValue();
-
-        // Match operator pattern and initialize operator members.
-        if (matchesOperatorPattern(opRef, context)) {
-            // Analyze select condition.
-            Map<IAccessMethod, AccessMethodAnalysisContext> analyzedAMs = new TreeMap<>();
-            if (!analyzeCondition(selectCond, subTree.getAssignsAndUnnests(), analyzedAMs, context, typeEnvironment)) {
-                return false;
-            }
-
-            // Set dataset and type metadata.
-            if (!subTree.setDatasetAndTypeMetadata((MetadataProvider) context.getMetadataProvider())) {
-                return false;
-            }
-
-            fillSubTreeIndexExprs(subTree, analyzedAMs, context);
-            pruneIndexCandidates(analyzedAMs, context, typeEnvironment);
-
-            // Choose index to be applied.
-            List<Pair<IAccessMethod, Index>> chosenIndexes = chooseAllIndex(analyzedAMs);
-            if (chosenIndexes == null || chosenIndexes.isEmpty()) {
-                context.addToDontApplySet(this, selectOp);
-                return false;
-            }
-
-            // Apply plan transformation using chosen index.
-            boolean res = intersectAllSecondaryIndexes(chosenIndexes, analyzedAMs, context);
-
-            context.addToDontApplySet(this, selectOp);
-            if (res) {
-                OperatorPropertiesUtil.typeOpRec(opRef, context);
-                return res;
-            }
-            selectRef = null;
-            selectOp = null;
-            afterSelectRefs.add(opRef);
-
-        } else {
-            // This is not a SELECT operator. Remember operators
-            afterSelectRefs.add(opRef);
-
-        }
-        // Recursively check the plan and try to optimize it.
-        boolean selectFoundAndOptimizationApplied = false;
-        for (Mutable<ILogicalOperator> inputOpRef : op.getInputs()) {
-            boolean foundHere = checkAndApplyTheSelectTransformationRule(inputOpRef, context);
-            if (foundHere) {
-                selectFoundAndOptimizationApplied = true;
-            }
+    /**
+     * Check that the given SELECT condition is a function call.
+     * Call initSubTree() to initialize the optimizable subtree that collects information from
+     * the operators below the given SELECT operator.
+     * In order to transform the given plan, a datasource should be configured
+     * since we are going to transform a datasource into an unnest-map operator.
+     */
+    protected boolean checkSelectOpConditionAndInitSubTree(IOptimizationContext context) throws AlgebricksException {
+        // Set and analyze select.
+        ILogicalExpression condExpr = selectOp.getCondition().getValue();
+        typeEnvironment = context.getOutputTypeEnvironment(selectOp);
+        if (condExpr.getExpressionTag() != LogicalExpressionTag.FUNCTION_CALL) {
+            return false;
         }
+        selectCond = (AbstractFunctionCallExpression) condExpr;
 
-        // Clean the path after SELECT operator by removing the current operator in the list.
-        afterSelectRefs.remove(opRef);
-        return selectFoundAndOptimizationApplied;
-
+        // Initialize the subtree information.
+        // Match and put assign, unnest, and datasource information.
+        boolean res = subTree.initFromSubTree(selectOp.getInputs().get(0));
+        return res && subTree.hasDataSourceScan();
     }
 
+    /**
+     * Construct all applicable secondary index-based access paths in the given selection plan and
+     * intersect them using INTERSECT operator to guide to the common primary index search.
+     * In case where the applicable index is one, we only construct one path.
+     */
     private boolean intersectAllSecondaryIndexes(List<Pair<IAccessMethod, Index>> chosenIndexes,
             Map<IAccessMethod, AccessMethodAnalysisContext> analyzedAMs, IOptimizationContext context)
-                    throws AlgebricksException {
+            throws AlgebricksException {
         Pair<IAccessMethod, Index> chosenIndex = null;
         Optional<Pair<IAccessMethod, Index>> primaryIndex = chosenIndexes.stream()
                 .filter(pair -> pair.second.isPrimaryIndex()).findFirst();
@@ -235,22 +204,26 @@ public class IntroduceSelectAccessMethodRule extends AbstractIntroduceAccessMeth
                             .getExecutionMode() == ExecutionMode.UNPARTITIONED,
                     context));
         }
-        ILogicalOperator primaryUnnest = connectAll2ndarySearchPlanWithIntersect(subRoots, context);
+        // Connect each secondary index utilization plan to a common intersect operator.
+        ILogicalOperator primaryUnnestOp = connectAll2ndarySearchPlanWithIntersect(subRoots, context);
 
-        subTree.getDataSourceRef().setValue(primaryUnnest);
-        return primaryUnnest != null;
+        subTree.getDataSourceRef().setValue(primaryUnnestOp);
+        return primaryUnnestOp != null;
     }
 
+    /**
+     * Connect each secondary index utilization plan to a common INTERSECT operator.
+     */
     private ILogicalOperator connectAll2ndarySearchPlanWithIntersect(List<ILogicalOperator> subRoots,
             IOptimizationContext context) throws AlgebricksException {
         ILogicalOperator lop = subRoots.get(0);
         List<List<LogicalVariable>> inputVars = new ArrayList<>(subRoots.size());
         for (int i = 0; i < subRoots.size(); i++) {
             if (lop.getOperatorTag() != subRoots.get(i).getOperatorTag()) {
-                throw new AlgebricksException("The data source root should have the same operator type");
+                throw new AlgebricksException("The data source root should have the same operator type.");
             }
             if (lop.getInputs().size() != 1) {
-                throw new AlgebricksException("The primary search has multiple input");
+                throw new AlgebricksException("The primary search has multiple inputs.");
             }
 
             ILogicalOperator curRoot = subRoots.get(i);
@@ -259,7 +232,8 @@ public class IntroduceSelectAccessMethodRule extends AbstractIntroduceAccessMeth
             for (Pair<OrderOperator.IOrder, Mutable<ILogicalExpression>> orderExpression : order
                     .getOrderExpressions()) {
                 if (orderExpression.second.getValue().getExpressionTag() != LogicalExpressionTag.VARIABLE) {
-                    throw new AlgebricksException("It should not happen, the order by expression is not variables");
+                    throw new AlgebricksException(
+                            "The order by expression should be variables, but they aren't variables.");
                 }
                 VariableReferenceExpression orderedVar = (VariableReferenceExpression) orderExpression.second
                         .getValue();
@@ -278,31 +252,128 @@ public class IntroduceSelectAccessMethodRule extends AbstractIntroduceAccessMeth
         return lop;
     }
 
-    protected boolean matchesOperatorPattern(Mutable<ILogicalOperator> opRef, IOptimizationContext context)
-            throws AlgebricksException {
-        // First check that the operator is a select and its condition is a function call.
-        AbstractLogicalOperator op1 = (AbstractLogicalOperator) opRef.getValue();
-        if (context.checkIfInDontApplySet(this, op1)) {
-            return false;
+    /**
+     * Recursively traverse the given plan and check whether a SELECT operator exists.
+     * If one is found, maintain the path from the root to SELECT operator and
+     * optimize the path from the SELECT operator to the EMPTY_TUPLE_SOURCE operator
+     * if it is not already optimized.
+     */
+    protected boolean checkAndApplyTheSelectTransformation(Mutable<ILogicalOperator> opRef,
+            IOptimizationContext context) throws AlgebricksException {
+        AbstractLogicalOperator op = (AbstractLogicalOperator) opRef.getValue();
+        boolean selectFoundAndOptimizationApplied;
+        boolean isSelectOp = false;
+
+        Mutable<ILogicalOperator> selectRefFromThisOp = null;
+        SelectOperator selectOpFromThisOp = null;
+
+        // Check the current operator pattern to see whether it is a JOIN or not.
+        if (op.getOperatorTag() == LogicalOperatorTag.SELECT) {
+            selectRef = opRef;
+            selectOp = (SelectOperator) op;
+            selectRefFromThisOp = opRef;
+            selectOpFromThisOp = (SelectOperator) op;
+            isSelectOp = true;
+        } else {
+            // This is not a SELECT operator. Remember this operator.
+            afterSelectRefs.add(opRef);
         }
-        if (op1.getOperatorTag() != LogicalOperatorTag.SELECT) {
-            return false;
+
+        // Recursively check the plan and try to optimize it. We first check the children of the given operator
+        // to make sure an earlier select in the path is optimized first.
+        for (Mutable<ILogicalOperator> inputOpRef : op.getInputs()) {
+            selectFoundAndOptimizationApplied = checkAndApplyTheSelectTransformation(inputOpRef, context);
+            if (selectFoundAndOptimizationApplied) {
+                return true;
+            }
         }
-        // Set and analyze select.
-        selectRef = opRef;
-        selectOp = (SelectOperator) op1;
 
-        typeEnvironment = context.getOutputTypeEnvironment(op1);
-        // Check that the select's condition is a function call.
-        ILogicalExpression condExpr = selectOp.getCondition().getValue();
-        if (condExpr.getExpressionTag() != LogicalExpressionTag.FUNCTION_CALL) {
-            return false;
+        // Traverse the plan until we find a SELECT operator.
+        if (isSelectOp) {
+            // Restore the information from this operator since it might have been be set to null
+            // if there are other select operators in the earlier path.
+            selectRef = selectRefFromThisOp;
+            selectOp = selectOpFromThisOp;
+
+            // Decides the plan transformation check needs to be continued.
+            // This variable is needed since we can't just return false
+            // in order to keep this operator in the afterSelectRefs list.
+            boolean continueCheck = true;
+
+            // Already checked this SELECT operator? If not, this operator may be optimized.
+            if (context.checkIfInDontApplySet(this, selectOp)) {
+                continueCheck = false;
+            }
+
+            // For each access method, contains the information about
+            // whether an available index can be applicable or not.
+            Map<IAccessMethod, AccessMethodAnalysisContext> analyzedAMs = null;
+            if (continueCheck) {
+                analyzedAMs = new TreeMap<>();
+            }
+
+            // Check the condition of SELECT operator is a function call since
+            // only function call can be transformed using available indexes.
+            // If so, initialize the subtree information that will be used later to decide whether
+            // the given plan is truly optimizable or not.
+            if (continueCheck && !checkSelectOpConditionAndInitSubTree(context)) {
+                continueCheck = false;
+            }
+
+            // Analyze the condition of SELECT operator and initialize analyzedAMs.
+            // Check whether the function in the SELECT operator can be truly transformed.
+            if (continueCheck && !analyzeSelectOrJoinOpConditionAndUpdateAnalyzedAM(selectCond,
+                    subTree.getAssignsAndUnnests(), analyzedAMs, context, typeEnvironment)) {
+                continueCheck = false;
+            }
+
+            // Find the dataset from the data-source and
+            // the record type of the dataset from the metadata.
+            // This will be used to find an applicable index on the dataset.
+            if (continueCheck && !subTree.setDatasetAndTypeMetadata((MetadataProvider) context.getMetadataProvider())) {
+                continueCheck = false;
+            }
+
+            if (continueCheck) {
+                // Map variables to the applicable indexes and find the field name and type.
+                // Then find the applicable indexes for the variables used in the SELECT condition.
+                fillSubTreeIndexExprs(subTree, analyzedAMs, context);
+
+                // Prune the access methods based on the function expression and access methods.
+                pruneIndexCandidates(analyzedAMs, context, typeEnvironment);
+
+                // Choose all indexes that will be applied.
+                List<Pair<IAccessMethod, Index>> chosenIndexes = chooseAllIndexes(analyzedAMs);
+
+                if (chosenIndexes == null || chosenIndexes.isEmpty()) {
+                    // We can't apply any index for this SELECT operator
+                    context.addToDontApplySet(this, selectRef.getValue());
+                    return false;
+                }
+
+                // Apply plan transformation using chosen index.
+                boolean res = intersectAllSecondaryIndexes(chosenIndexes, analyzedAMs, context);
+                context.addToDontApplySet(this, selectOp);
+
+                if (res) {
+                    OperatorPropertiesUtil.typeOpRec(opRef, context);
+                    return res;
+                }
+            }
+
+            selectRef = null;
+            selectOp = null;
+            afterSelectRefs.add(opRef);
         }
-        selectCond = (AbstractFunctionCallExpression) condExpr;
-        boolean res = subTree.initFromSubTree(op1.getInputs().get(0));
-        return res && subTree.hasDataSourceScan();
+
+        // Clean the path after SELECT operator by removing the current operator in the list.
+        afterSelectRefs.remove(opRef);
+
+        return false;
+
     }
 
+
     @Override
     public Map<FunctionIdentifier, List<IAccessMethod>> getAccessMethods() {
         return accessMethods;
@@ -313,5 +384,6 @@ public class IntroduceSelectAccessMethodRule extends AbstractIntroduceAccessMeth
         selectRef = null;
         selectOp = null;
         selectCond = null;
+        subTree.reset();
     }
 }

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/c4eb6564/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/am/InvertedIndexAccessMethod.java
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/am/InvertedIndexAccessMethod.java b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/am/InvertedIndexAccessMethod.java
index 7328f97..a02679d 100644
--- a/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/am/InvertedIndexAccessMethod.java
+++ b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/am/InvertedIndexAccessMethod.java
@@ -28,6 +28,8 @@ import java.util.Map;
 import org.apache.asterix.common.annotations.SkipSecondaryIndexSearchExpressionAnnotation;
 import org.apache.asterix.common.config.DatasetConfig.IndexType;
 import org.apache.asterix.common.exceptions.AsterixException;
+import org.apache.asterix.common.exceptions.CompilationException;
+import org.apache.asterix.common.exceptions.ErrorCode;
 import org.apache.asterix.dataflow.data.common.ExpressionTypeComputer;
 import org.apache.asterix.formats.nontagged.BinaryTokenizerFactoryProvider;
 import org.apache.asterix.lang.common.util.FunctionUtil;
@@ -135,17 +137,17 @@ public class InvertedIndexAccessMethod implements IAccessMethod {
     }
 
     @Override
-    public boolean analyzeFuncExprArgs(AbstractFunctionCallExpression funcExpr,
+    public boolean analyzeFuncExprArgsAndUpdateAnalysisCtx(AbstractFunctionCallExpression funcExpr,
             List<AbstractLogicalOperator> assignsAndUnnests, AccessMethodAnalysisContext analysisCtx,
             IOptimizationContext context, IVariableTypeEnvironment typeEnvironment) throws AlgebricksException {
 
         if (funcExpr.getFunctionIdentifier() == BuiltinFunctions.STRING_CONTAINS
                 || funcExpr.getFunctionIdentifier() == BuiltinFunctions.FULLTEXT_CONTAINS
                 || funcExpr.getFunctionIdentifier() == BuiltinFunctions.FULLTEXT_CONTAINS_WO_OPTION) {
-            boolean matches = AccessMethodUtils.analyzeFuncExprArgsForOneConstAndVar(funcExpr, analysisCtx, context,
-                    typeEnvironment);
+            boolean matches = AccessMethodUtils.analyzeFuncExprArgsForOneConstAndVarAndUpdateAnalysisCtx(funcExpr,
+                    analysisCtx, context, typeEnvironment);
             if (!matches) {
-                matches = AccessMethodUtils.analyzeFuncExprArgsForTwoVars(funcExpr, analysisCtx);
+                matches = AccessMethodUtils.analyzeFuncExprArgsForTwoVarsAndUpdateAnalysisCtx(funcExpr, analysisCtx);
             }
             return matches;
         }
@@ -265,13 +267,13 @@ public class InvertedIndexAccessMethod implements IAccessMethod {
         OptimizableFuncExpr newOptFuncExpr = new OptimizableFuncExpr(funcExpr,
                 new LogicalVariable[] { fieldVarExpr1, fieldVarExpr2 }, new ILogicalExpression[] { arg3 },
                 new IAType[] { (IAType) ExpressionTypeComputer.INSTANCE.getType(arg3, null, null) });
-        for (IOptimizableFuncExpr optFuncExpr : analysisCtx.matchedFuncExprs) {
+        for (IOptimizableFuncExpr optFuncExpr : analysisCtx.getMatchedFuncExprs()) {
             //avoid additional optFuncExpressions in case of a join
             if (optFuncExpr.getFuncExpr().equals(funcExpr)) {
                 return true;
             }
         }
-        analysisCtx.matchedFuncExprs.add(newOptFuncExpr);
+        analysisCtx.addMatchedFuncExpr(newOptFuncExpr);
         return true;
     }
 
@@ -315,13 +317,13 @@ public class InvertedIndexAccessMethod implements IAccessMethod {
                 new ILogicalExpression[] { constArg, arg3 },
                 new IAType[] { (IAType) ExpressionTypeComputer.INSTANCE.getType(constArg, null, null),
                         (IAType) ExpressionTypeComputer.INSTANCE.getType(arg3, null, null) });
-        for (IOptimizableFuncExpr optFuncExpr : analysisCtx.matchedFuncExprs) {
+        for (IOptimizableFuncExpr optFuncExpr : analysisCtx.getMatchedFuncExprs()) {
             //avoid additional optFuncExpressions in case of a join
             if (optFuncExpr.getFuncExpr().equals(funcExpr)) {
                 return true;
             }
         }
-        analysisCtx.matchedFuncExprs.add(newOptFuncExpr);
+        analysisCtx.addMatchedFuncExpr(newOptFuncExpr);
         return true;
     }
 
@@ -466,7 +468,7 @@ public class InvertedIndexAccessMethod implements IAccessMethod {
             AccessMethodAnalysisContext analysisCtx, IOptimizationContext context, boolean isLeftOuterJoin,
             boolean hasGroupBy) throws AlgebricksException {
         // Figure out if the index is applicable on the left or right side (if both, we arbitrarily prefer the left side).
-        Dataset dataset = analysisCtx.indexDatasetMap.get(chosenIndex);
+        Dataset dataset = analysisCtx.getDatasetFromIndexDatasetMap(chosenIndex);
         OptimizableOperatorSubTree indexSubTree;
         OptimizableOperatorSubTree probeSubTree;
 
@@ -595,7 +597,7 @@ public class InvertedIndexAccessMethod implements IAccessMethod {
             ILogicalExpression joinCond, IOptimizableFuncExpr optFuncExpr, List<LogicalVariable> originalSubTreePKs,
             List<LogicalVariable> surrogateSubTreePKs, IOptimizationContext context) throws AlgebricksException {
 
-        probeSubTree.getPrimaryKeyVars(originalSubTreePKs);
+        probeSubTree.getPrimaryKeyVars(null, originalSubTreePKs);
 
         // Create two copies of the original probe subtree.
         // The first copy, which becomes the new probe subtree, will retain the primary-key and secondary-search key variables,
@@ -775,7 +777,7 @@ public class InvertedIndexAccessMethod implements IAccessMethod {
                 break;
             }
             default: {
-                throw new AlgebricksException("Only strings, ordered and unordered list types supported.");
+                throw CompilationException.create(ErrorCode.NO_SUPPORTED_TYPE);
             }
         }
 
@@ -828,7 +830,7 @@ public class InvertedIndexAccessMethod implements IAccessMethod {
                 typeTag = ((AUnionType) type).getActualType().getTypeTag();
             }
             if (typeTag != ATypeTag.ORDEREDLIST && typeTag != ATypeTag.STRING && typeTag != ATypeTag.UNORDEREDLIST) {
-                throw new AlgebricksException("Only ordered lists, string, and unordered lists types supported.");
+                throw CompilationException.create(ErrorCode.NO_SUPPORTED_TYPE);
             }
         }
         jobGenParams.setSearchKeyType(typeTag);
@@ -1226,7 +1228,7 @@ public class InvertedIndexAccessMethod implements IAccessMethod {
                         index.getGramLength(), prePost, false);
             }
             default: {
-                throw new AlgebricksException("Tokenizer not applicable to index kind '" + index.getIndexType() + "'.");
+                throw CompilationException.create(ErrorCode.NO_TOKENIZER_FOR_TYPE, index.getIndexType());
             }
         }
     }
@@ -1271,12 +1273,12 @@ public class InvertedIndexAccessMethod implements IAccessMethod {
                         }
                     }
                     default: {
-                        throw new AlgebricksException("Incompatible search modifier '" + searchModifierType
-                                + "' for index type '" + index.getIndexType() + "'");
+                        throw CompilationException.create(ErrorCode.INCOMPATIBLE_SEARCH_MODIFIER,
+                                searchModifierType, index.getIndexType());
                     }
                 }
             default:
-                throw new AlgebricksException("Unknown search modifier type '" + searchModifierType + "'.");
+                throw CompilationException.create(ErrorCode.UNKNOWN_SEARCH_MODIFIER, searchModifierType);
         }
     }