You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@drill.apache.org by am...@apache.org on 2018/10/25 23:08:59 UTC

[drill] 03/08: DRILL-6381: (Part 3) Planner and Execution implementation to support Secondary Indexes

This is an automated email from the ASF dual-hosted git repository.

amansinha pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/drill.git

commit 12f57f0ca0ea9651ad763a5c01eb5708ba3f245a
Author: rebase <bu...@mapr.com>
AuthorDate: Fri Mar 16 14:25:26 2018 -0700

    DRILL-6381: (Part 3) Planner and Execution implementation to support Secondary Indexes
    
      1. Index Planning Rules and Plan generators
        - DbScanToIndexScanRule: Top level physical planning rule that drives index planning for several relational algebra patterns.
        - DbScanSortRemovalRule: Physical planning rule for index planning for Sort-based operations.
        - Plan Generators: Covering, Non-Covering and Intersect physical plan generators.
        - Support planning with functional indexes such as CAST functions.
        - Enhance PlannerSettings with several configuration options for indexes.
      2. Index Selection and Statistics
        - An IndexSelector that support cost-based index selection of covering and non-covering indexes using statistics and collation properties.
        - Costing of index intersection for comparison with single-index plans.
      3. Planning and execution operators
        - Support RangePartitioning physical operator during query planning and execution.
        - Support RowKeyJoin physical operator during query planning and execution.
        - HashTable and HashJoin changes to support RowKeyJoin and Index Intersection.
        - Enhance Materializer to keep track of subscan association with a particular rowkey join.
      4. Index Planning utilities
        - Utility classes to perform RexNode analysis, including conversion to and from SchemaPath.
        - Utility class to analyze filter condition and an input collation to determine output collation.
        - Helper classes to maintain index contexts for logical and physical planning phase.
        - IndexPlanUtils utility class for various helper methods.
      5. Miscellaneous
        - Separate physical rel for DirectScan.
        - Modify LimitExchangeTranspose rule to handle SingleMergeExchange.
        - MD-3880: Return correct status from RangePartitionRecordBatch setupNewSchema
    
    Co-authored-by: Aman Sinha <as...@maprtech.com>
    Co-authored-by: chunhui-shi <cs...@maprtech.com>
    Co-authored-by: Gautam Parai <gp...@maprtech.com>
    Co-authored-by: Padma Penumarthy <pp...@yahoo.com>
    Co-authored-by: Hanumath Rao Maduri <hm...@maprtech.com>
    
    Conflicts:
    	exec/java-exec/src/main/java/org/apache/drill/exec/physical/config/HashJoinPOP.java
    	exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/ScanBatch.java
    	exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/common/HashPartition.java
    	exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/common/HashTable.java
    	exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/common/HashTableTemplate.java
    	exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/join/HashJoinBatch.java
    	exec/java-exec/src/main/java/org/apache/drill/exec/planner/common/DrillRelOptUtil.java
    	exec/java-exec/src/main/java/org/apache/drill/exec/planner/fragment/Materializer.java
    	exec/java-exec/src/main/java/org/apache/drill/exec/planner/logical/DrillMergeProjectRule.java
    	exec/java-exec/src/main/java/org/apache/drill/exec/planner/logical/DrillOptiq.java
    	exec/java-exec/src/main/java/org/apache/drill/exec/planner/logical/DrillPushProjectIntoScanRule.java
    	exec/java-exec/src/main/java/org/apache/drill/exec/planner/logical/DrillScanRel.java
    	exec/java-exec/src/main/java/org/apache/drill/exec/planner/physical/BroadcastExchangePrel.java
    	exec/java-exec/src/main/java/org/apache/drill/exec/planner/physical/DrillDistributionTrait.java
    	exec/java-exec/src/main/java/org/apache/drill/exec/planner/physical/HashJoinPrel.java
    	exec/java-exec/src/main/java/org/apache/drill/exec/planner/physical/PrelUtil.java
    	exec/java-exec/src/main/java/org/apache/drill/exec/server/options/SystemOptionManager.java
    	exec/java-exec/src/main/java/org/apache/drill/exec/store/parquet/ParquetPushDownFilter.java
    	exec/java-exec/src/main/resources/drill-module.conf
    	logical/src/main/java/org/apache/drill/common/logical/StoragePluginConfig.java
    
    Resolve merge comflicts and compilation issues.
---
 .../drill/exec/planner/index/MapRDBStatistics.java |   2 +-
 .../store/mapr/db/MapRDBPushFilterIntoScan.java    |   4 +-
 .../store/mapr/db/MapRDBPushLimitIntoScan.java     |   2 +-
 .../store/mapr/db/MapRDBPushProjectIntoScan.java   |  15 +-
 .../exec/store/hbase/HBasePushFilterIntoScan.java  |   2 +-
 ...ertHiveMapRDBJsonScanToDrillMapRDBJsonScan.java |   2 +-
 .../store/kafka/KafkaPushDownFilterIntoScan.java   |   3 +-
 .../store/mongo/MongoPushDownFilterForScan.java    |   3 +-
 .../physical/base/AbstractPhysicalVisitor.java     |   6 +
 .../drill/exec/physical/base/PhysicalVisitor.java  |   2 +
 .../drill/exec/physical/config/HashJoinPOP.java    |  74 +-
 .../exec/physical/config/HashPartitionSender.java  |  20 +-
 .../physical/config/RangePartitionExchange.java    |  60 ++
 .../drill/exec/physical/config/RowKeyJoinPOP.java  |  96 +++
 .../apache/drill/exec/physical/impl/ScanBatch.java | 139 ++--
 .../exec/physical/impl/common/HashPartition.java   |   5 +
 .../drill/exec/physical/impl/common/HashTable.java |   3 +
 .../exec/physical/impl/common/HashTableConfig.java |  45 +-
 .../physical/impl/common/HashTableTemplate.java    |  23 +
 .../exec/physical/impl/join/HashJoinBatch.java     | 100 ++-
 .../physical/impl/join/HashJoinBatchCreator.java   |  11 +-
 .../exec/physical/impl/join/HashJoinHelper.java    |  11 +-
 .../physical/impl/join/HashJoinProbeTemplate.java  |   7 +
 .../exec/physical/impl/join/RowKeyJoinBatch.java   | 284 +++++++
 ...tchCreator.java => RowKeyJoinBatchCreator.java} |  18 +-
 .../impl/mergereceiver/MergingRecordBatch.java     |   5 +-
 .../physical/impl/partitionsender/Partitioner.java |   2 +
 .../RangePartitionRecordBatch.java                 | 186 +++++
 .../RangePartitionSenderCreator.java               |  46 ++
 .../apache/drill/exec/planner/PlannerPhase.java    |  34 +-
 .../exec/planner/common/DrillJoinRelBase.java      |   4 +
 .../drill/exec/planner/common/DrillRelOptUtil.java | 137 ++++
 .../exec/planner/common/DrillScanRelBase.java      |  39 +-
 .../drill/exec/planner/common/JoinControl.java     |  53 ++
 .../cost/DrillDefaultRelMetadataProvider.java      |   1 +
 .../exec/planner/cost/DrillRelMdSelectivity.java   |  90 +++
 .../drill/exec/planner/fragment/Materializer.java  |  37 +-
 .../apache/drill/exec/planner/index/ExprToRex.java | 107 +++
 .../planner/index/FindFiltersForCollation.java     | 199 +++++
 .../exec/planner/index/FunctionalIndexHelper.java  | 215 ++++++
 .../exec/planner/index/IndexConditionInfo.java     | 250 ++++++
 .../exec/planner/index/IndexDiscoverFactory.java   |  75 ++
 .../planner/index/IndexLogicalPlanCallContext.java | 178 +++++
 .../index/IndexPhysicalPlanCallContext.java        | 193 +++++
 .../drill/exec/planner/index/IndexPlanUtils.java   | 855 +++++++++++++++++++++
 .../drill/exec/planner/index/IndexProperties.java  |   7 +-
 .../drill/exec/planner/index/IndexSelector.java    | 766 ++++++++++++++++++
 .../exec/planner/index/IndexableExprMarker.java    |   2 +-
 .../drill/exec/planner/index/PathInExpr.java       | 147 ++++
 .../drill/exec/planner/index/RexSeparator.java     | 116 +++
 .../drill/exec/planner/index/RexToExpression.java  | 165 ++++
 .../drill/exec/planner/index/SimpleRexRemap.java   | 300 ++++++++
 .../drill/exec/planner/index/Statistics.java       |   3 +-
 .../generators/AbstractIndexPlanGenerator.java     | 259 +++++++
 .../generators/CoveringIndexPlanGenerator.java     | 208 +++++
 .../generators/CoveringPlanNoFilterGenerator.java  | 125 +++
 .../generators/IndexIntersectPlanGenerator.java    | 338 ++++++++
 .../generators/NonCoveringIndexPlanGenerator.java  | 335 ++++++++
 .../planner/index/rules/AbstractMatchFunction.java |  58 ++
 .../planner/index/rules/DbScanSortRemovalRule.java | 226 ++++++
 .../index/rules/DbScanToIndexScanPrule.java        | 533 +++++++++++++
 .../rules/MatchFunction.java}                      |  20 +-
 .../planner/logical/DrillMergeProjectRule.java     |  24 +
 .../drill/exec/planner/logical/DrillOptiq.java     |  43 +-
 .../logical/DrillPushProjectIntoScanRule.java      | 171 +----
 .../drill/exec/planner/logical/DrillScanRel.java   |  37 +-
 .../physical/AbstractRangePartitionFunction.java   |  44 +-
 .../planner/physical/BroadcastExchangePrel.java    |   4 +-
 .../planner/physical/ConvertCountToDirectScan.java |   5 +-
 .../{ScanPrel.java => DirectScanPrel.java}         | 146 ++--
 .../exec/planner/physical/DirectScanPrule.java     |  10 +-
 .../planner/physical/DrillDistributionTrait.java   |  31 +-
 .../physical/DrillDistributionTraitDef.java        |   5 +-
 .../drill/exec/planner/physical/FilterPrel.java    |   2 +-
 .../drill/exec/planner/physical/HashJoinPrel.java  |  26 +-
 ...seRule.java => LimitExchangeTransposeRule.java} |  63 +-
 .../exec/planner/physical/PlannerSettings.java     |  70 ++
 .../drill/exec/planner/physical/PrelFactories.java |  52 ++
 .../drill/exec/planner/physical/PrelUtil.java      | 232 +-----
 .../physical/RangePartitionExchangePrel.java       | 130 ++++
 .../exec/planner/physical/RowKeyJoinPrel.java      | 117 +++
 .../drill/exec/planner/physical/ScanPrel.java      |  45 +-
 .../drill/exec/planner/physical/ScanPrule.java     |   2 +-
 .../exec/planner/physical/SubsetTransformer.java   |  13 +
 .../physical/visitor/SwapHashJoinVisitor.java      |   3 +-
 .../exec/server/options/SystemOptionManager.java   |  19 +-
 .../drill/exec/store/dfs/FileSystemConfig.java     |   8 +-
 .../InfoSchemaPushFilterIntoRecordGenerator.java   |   2 +-
 .../exec/store/parquet/ParquetPushDownFilter.java  |   2 +-
 .../java-exec/src/main/resources/drill-module.conf |  16 +
 .../drill/common/expression/CastExpression.java    |  23 +
 .../apache/drill/common/expression/SchemaPath.java |   3 +
 .../drill/common/logical/StoragePluginConfig.java  |   9 +-
 93 files changed, 7895 insertions(+), 713 deletions(-)

diff --git a/contrib/format-maprdb/src/main/java/org/apache/drill/exec/planner/index/MapRDBStatistics.java b/contrib/format-maprdb/src/main/java/org/apache/drill/exec/planner/index/MapRDBStatistics.java
index dc89a4d..3b8de34 100644
--- a/contrib/format-maprdb/src/main/java/org/apache/drill/exec/planner/index/MapRDBStatistics.java
+++ b/contrib/format-maprdb/src/main/java/org/apache/drill/exec/planner/index/MapRDBStatistics.java
@@ -185,7 +185,7 @@ public class MapRDBStatistics implements Statistics {
    *  @return approximate rows satisfying the leading filter
    */
   @Override
-  public double getLeadingRowCount(RexNode condition, String tabIdxName, RelNode scanRel) {
+  public double getLeadingRowCount(RexNode condition, String tabIdxName, DrillScanRelBase scanRel) {
     String conditionAsStr = nullConditionAsString;
     Map<String, StatisticsPayload> payloadMap;
     if ((scanRel instanceof DrillScanRel && ((DrillScanRel)scanRel).getGroupScan() instanceof DbGroupScan)
diff --git a/contrib/format-maprdb/src/main/java/org/apache/drill/exec/store/mapr/db/MapRDBPushFilterIntoScan.java b/contrib/format-maprdb/src/main/java/org/apache/drill/exec/store/mapr/db/MapRDBPushFilterIntoScan.java
index cf49714..511a111 100644
--- a/contrib/format-maprdb/src/main/java/org/apache/drill/exec/store/mapr/db/MapRDBPushFilterIntoScan.java
+++ b/contrib/format-maprdb/src/main/java/org/apache/drill/exec/store/mapr/db/MapRDBPushFilterIntoScan.java
@@ -140,7 +140,7 @@ public abstract class MapRDBPushFilterIntoScan extends StoragePluginOptimizerRul
     final JsonTableGroupScan newGroupsScan = (JsonTableGroupScan) groupScan.clone(newScanSpec);
     newGroupsScan.setFilterPushedDown(true);
 
-    final ScanPrel newScanPrel = new ScanPrel(scan, filter.getTraitSet(), newGroupsScan, scan.getRowType());
+    final ScanPrel newScanPrel = new ScanPrel(scan, filter.getTraitSet(), newGroupsScan, scan.getRowType(), scan.getTable());
 
     // Depending on whether is a project in the middle, assign either scan or copy of project to childRel.
     final RelNode childRel = project == null ? newScanPrel : project.copy(project.getTraitSet(), ImmutableList.of((RelNode)newScanPrel));
@@ -186,7 +186,7 @@ public abstract class MapRDBPushFilterIntoScan extends StoragePluginOptimizerRul
                                                                         groupScan.getTableStats());
     newGroupsScan.setFilterPushedDown(true);
 
-    final ScanPrel newScanPrel = new ScanPrel(scan, filter.getTraitSet(), newGroupsScan, scan.getRowType());
+    final ScanPrel newScanPrel = new ScanPrel(scan, filter.getTraitSet(), newGroupsScan, scan.getRowType(), scan.getTable());
 
     // Depending on whether is a project in the middle, assign either scan or copy of project to childRel.
     final RelNode childRel = project == null ? newScanPrel : project.copy(project.getTraitSet(), ImmutableList.of((RelNode)newScanPrel));;
diff --git a/contrib/format-maprdb/src/main/java/org/apache/drill/exec/store/mapr/db/MapRDBPushLimitIntoScan.java b/contrib/format-maprdb/src/main/java/org/apache/drill/exec/store/mapr/db/MapRDBPushLimitIntoScan.java
index 79eec12..1f4b8c9 100644
--- a/contrib/format-maprdb/src/main/java/org/apache/drill/exec/store/mapr/db/MapRDBPushLimitIntoScan.java
+++ b/contrib/format-maprdb/src/main/java/org/apache/drill/exec/store/mapr/db/MapRDBPushLimitIntoScan.java
@@ -119,7 +119,7 @@ public abstract class MapRDBPushLimitIntoScan extends StoragePluginOptimizerRule
         return;
       }
       final ScanPrel newScan = new ScanPrel(scan.getCluster(), scan.getTraitSet(), newGroupScan,
-          scan.getRowType());
+          scan.getRowType(), scan.getTable());
       final RelNode newChild;
       if (project != null) {
         final ProjectPrel newProject = new ProjectPrel(project.getCluster(), project.getTraitSet(),
diff --git a/contrib/format-maprdb/src/main/java/org/apache/drill/exec/store/mapr/db/MapRDBPushProjectIntoScan.java b/contrib/format-maprdb/src/main/java/org/apache/drill/exec/store/mapr/db/MapRDBPushProjectIntoScan.java
index 8d4f549..2eb84e7 100644
--- a/contrib/format-maprdb/src/main/java/org/apache/drill/exec/store/mapr/db/MapRDBPushProjectIntoScan.java
+++ b/contrib/format-maprdb/src/main/java/org/apache/drill/exec/store/mapr/db/MapRDBPushProjectIntoScan.java
@@ -28,6 +28,7 @@ import org.apache.calcite.rel.type.RelDataType;
 import org.apache.calcite.rel.type.RelDataTypeField;
 import org.apache.calcite.rex.RexNode;
 import org.apache.drill.common.exceptions.DrillRuntimeException;
+import org.apache.drill.exec.planner.common.DrillRelOptUtil;
 import org.apache.drill.exec.planner.logical.RelOptHelper;
 import org.apache.drill.exec.planner.physical.Prel;
 import org.apache.drill.exec.planner.physical.PrelUtil;
@@ -36,6 +37,7 @@ import org.apache.drill.exec.planner.physical.ScanPrel;
 import org.apache.drill.exec.store.StoragePluginOptimizerRule;
 import org.apache.drill.exec.store.mapr.db.binary.BinaryTableGroupScan;
 import org.apache.drill.exec.store.mapr.db.json.JsonTableGroupScan;
+import org.apache.drill.exec.util.Utilities;
 
 import java.util.List;
 
@@ -82,9 +84,10 @@ public abstract class MapRDBPushProjectIntoScan extends StoragePluginOptimizerRu
       ProjectPrel project, ScanPrel scan, MapRDBGroupScan groupScan) {
     try {
 
-      PrelUtil.ProjectPushInfo columnInfo = PrelUtil.getColumns(scan.getRowType(), project.getProjects());
-      if (columnInfo == null || columnInfo.isStarQuery() //
-          || !groupScan.canPushdownProjects(columnInfo.columns)) {
+      DrillRelOptUtil.ProjectPushInfo columnInfo =
+          DrillRelOptUtil.getFieldsInformation(scan.getRowType(), project.getProjects());
+      if (columnInfo == null || Utilities.isStarQuery(columnInfo.getFields()) //
+          || !groupScan.canPushdownProjects(columnInfo.getFields())) {
         return;
       }
       RelTraitSet newTraits = call.getPlanner().emptyTraitSet();
@@ -95,12 +98,12 @@ public abstract class MapRDBPushProjectIntoScan extends StoragePluginOptimizerRu
         }
       }
       final ScanPrel newScan = new ScanPrel(scan.getCluster(), newTraits.plus(Prel.DRILL_PHYSICAL),
-          groupScan.clone(columnInfo.columns),
-          columnInfo.createNewRowType(project.getInput().getCluster().getTypeFactory()));
+          groupScan.clone(columnInfo.getFields()),
+          columnInfo.createNewRowType(project.getInput().getCluster().getTypeFactory()), scan.getTable());
 
       List<RexNode> newProjects = Lists.newArrayList();
       for (RexNode n : project.getChildExps()) {
-        newProjects.add(n.accept(columnInfo.getInputRewriter()));
+        newProjects.add(n.accept(columnInfo.getInputReWriter()));
       }
 
       final ProjectPrel newProj =
diff --git a/contrib/storage-hbase/src/main/java/org/apache/drill/exec/store/hbase/HBasePushFilterIntoScan.java b/contrib/storage-hbase/src/main/java/org/apache/drill/exec/store/hbase/HBasePushFilterIntoScan.java
index 736b36b..91ca787 100644
--- a/contrib/storage-hbase/src/main/java/org/apache/drill/exec/store/hbase/HBasePushFilterIntoScan.java
+++ b/contrib/storage-hbase/src/main/java/org/apache/drill/exec/store/hbase/HBasePushFilterIntoScan.java
@@ -122,7 +122,7 @@ public abstract class HBasePushFilterIntoScan extends StoragePluginOptimizerRule
         newScanSpec, groupScan.getColumns());
     newGroupsScan.setFilterPushedDown(true);
 
-    final ScanPrel newScanPrel = new ScanPrel(scan, filter.getTraitSet(), newGroupsScan, scan.getRowType());
+    final ScanPrel newScanPrel = new ScanPrel(scan, filter.getTraitSet(), newGroupsScan, scan.getRowType(), scan.getTable());
 
     // Depending on whether is a project in the middle, assign either scan or copy of project to childRel.
     final RelNode childRel = project == null ? newScanPrel : project.copy(project.getTraitSet(), ImmutableList.of(newScanPrel));
diff --git a/contrib/storage-hive/core/src/main/java/org/apache/drill/exec/planner/sql/logical/ConvertHiveMapRDBJsonScanToDrillMapRDBJsonScan.java b/contrib/storage-hive/core/src/main/java/org/apache/drill/exec/planner/sql/logical/ConvertHiveMapRDBJsonScanToDrillMapRDBJsonScan.java
index 9eb43a2..4994a72 100644
--- a/contrib/storage-hive/core/src/main/java/org/apache/drill/exec/planner/sql/logical/ConvertHiveMapRDBJsonScanToDrillMapRDBJsonScan.java
+++ b/contrib/storage-hive/core/src/main/java/org/apache/drill/exec/planner/sql/logical/ConvertHiveMapRDBJsonScanToDrillMapRDBJsonScan.java
@@ -123,7 +123,7 @@ public class ConvertHiveMapRDBJsonScanToDrillMapRDBJsonScan extends StoragePlugi
     HiveScan hiveScan = (HiveScan) hiveScanRel.getGroupScan();
     Map<String, String> parameters = hiveScan.getHiveReadEntry().getHiveTableWrapper().getParameters();
 
-    JsonScanSpec scanSpec = new JsonScanSpec(parameters.get(MAPRDB_TABLE_NAME), null);
+    JsonScanSpec scanSpec = new JsonScanSpec(parameters.get(MAPRDB_TABLE_NAME), null, null);
     List<SchemaPath> hiveScanCols = hiveScanRel.getColumns().stream()
         .map(colNameSchemaPath -> replaceOverriddenSchemaPath(parameters, colNameSchemaPath))
         .collect(Collectors.toList());
diff --git a/contrib/storage-kafka/src/main/java/org/apache/drill/exec/store/kafka/KafkaPushDownFilterIntoScan.java b/contrib/storage-kafka/src/main/java/org/apache/drill/exec/store/kafka/KafkaPushDownFilterIntoScan.java
index 14abadf..019a67e 100644
--- a/contrib/storage-kafka/src/main/java/org/apache/drill/exec/store/kafka/KafkaPushDownFilterIntoScan.java
+++ b/contrib/storage-kafka/src/main/java/org/apache/drill/exec/store/kafka/KafkaPushDownFilterIntoScan.java
@@ -66,7 +66,8 @@ public class KafkaPushDownFilterIntoScan extends StoragePluginOptimizerRule {
 
     logger.info("Partitions ScanSpec after pushdown: " + newScanSpec);
     GroupScan newGroupScan = groupScan.cloneWithNewSpec(newScanSpec);
-    final ScanPrel newScanPrel = new ScanPrel(scan, filter.getTraitSet(), newGroupScan, scan.getRowType());
+    final ScanPrel newScanPrel = 
+      new ScanPrel(scan, filter.getTraitSet(), newGroupScan, scan.getRowType(), scan.getTable());
     call.transformTo(filter.copy(filter.getTraitSet(), ImmutableList.of(newScanPrel)));
   }
 
diff --git a/contrib/storage-mongo/src/main/java/org/apache/drill/exec/store/mongo/MongoPushDownFilterForScan.java b/contrib/storage-mongo/src/main/java/org/apache/drill/exec/store/mongo/MongoPushDownFilterForScan.java
index afe32f2..8ad84c1 100644
--- a/contrib/storage-mongo/src/main/java/org/apache/drill/exec/store/mongo/MongoPushDownFilterForScan.java
+++ b/contrib/storage-mongo/src/main/java/org/apache/drill/exec/store/mongo/MongoPushDownFilterForScan.java
@@ -78,7 +78,8 @@ public class MongoPushDownFilterForScan extends StoragePluginOptimizerRule {
     newGroupsScan.setFilterPushedDown(true);
 
     final ScanPrel newScanPrel = new ScanPrel(scan, filter.getTraitSet(),
-        newGroupsScan, scan.getRowType());
+        newGroupsScan, scan.getRowType(), scan.getTable());
+
     if (mongoFilterBuilder.isAllExpressionsConverted()) {
       /*
        * Since we could convert the entire filter condition expression into an
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/base/AbstractPhysicalVisitor.java b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/base/AbstractPhysicalVisitor.java
index ca82ca6..262e7e5 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/base/AbstractPhysicalVisitor.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/base/AbstractPhysicalVisitor.java
@@ -31,6 +31,7 @@ import org.apache.drill.exec.physical.config.OrderedPartitionSender;
 import org.apache.drill.exec.physical.config.ProducerConsumer;
 import org.apache.drill.exec.physical.config.Project;
 import org.apache.drill.exec.physical.config.RangePartitionSender;
+import org.apache.drill.exec.physical.config.RowKeyJoinPOP;
 import org.apache.drill.exec.physical.config.Screen;
 import org.apache.drill.exec.physical.config.SingleSender;
 import org.apache.drill.exec.physical.config.Sort;
@@ -139,6 +140,11 @@ public abstract class AbstractPhysicalVisitor<T, X, E extends Throwable> impleme
   }
 
   @Override
+  public T visitRowKeyJoin(RowKeyJoinPOP join, X value) throws E {
+    return visitOp(join, value);
+  }
+
+  @Override
   public T visitHashPartitionSender(HashPartitionSender op, X value) throws E {
     return visitSender(op, value);
   }
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/base/PhysicalVisitor.java b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/base/PhysicalVisitor.java
index 1bb1545..77a87c2 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/base/PhysicalVisitor.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/base/PhysicalVisitor.java
@@ -31,6 +31,7 @@ import org.apache.drill.exec.physical.config.OrderedPartitionSender;
 import org.apache.drill.exec.physical.config.ProducerConsumer;
 import org.apache.drill.exec.physical.config.Project;
 import org.apache.drill.exec.physical.config.RangePartitionSender;
+import org.apache.drill.exec.physical.config.RowKeyJoinPOP;
 import org.apache.drill.exec.physical.config.Screen;
 import org.apache.drill.exec.physical.config.SingleSender;
 import org.apache.drill.exec.physical.config.Sort;
@@ -64,6 +65,7 @@ public interface PhysicalVisitor<RETURN, EXTRA, EXCEP extends Throwable> {
   public RETURN visitLimit(Limit limit, EXTRA value) throws EXCEP;
   public RETURN visitFlatten(FlattenPOP flatten, EXTRA value) throws EXCEP;
   public RETURN visitSender(Sender sender, EXTRA value) throws EXCEP;
+  public RETURN visitRowKeyJoin(RowKeyJoinPOP join, EXTRA value) throws EXCEP;
   public RETURN visitReceiver(Receiver receiver, EXTRA value) throws EXCEP;
   public RETURN visitStreamingAggregate(StreamingAggregate agg, EXTRA value) throws EXCEP;
   public RETURN visitHashAggregate(HashAggregate agg, EXTRA value) throws EXCEP;
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/config/HashJoinPOP.java b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/config/HashJoinPOP.java
index 131d9b9..21d6092 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/config/HashJoinPOP.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/config/HashJoinPOP.java
@@ -24,6 +24,8 @@ import org.apache.drill.common.logical.data.JoinCondition;
 import org.apache.drill.exec.ExecConstants;
 import org.apache.drill.exec.ops.QueryContext;
 import org.apache.drill.exec.physical.base.PhysicalOperator;
+import org.apache.drill.exec.physical.base.SubScan;
+import org.apache.drill.exec.planner.common.JoinControl;
 import org.apache.drill.exec.proto.UserBitShared.CoreOperatorType;
 import org.apache.calcite.rel.core.JoinRelType;
 
@@ -43,44 +45,92 @@ public class HashJoinPOP extends AbstractJoinPop {
 
   private RuntimeFilterDef runtimeFilterDef;
 
+  private final boolean isRowKeyJoin;
+  private final int joinControl;
+  @JsonProperty("subScanForRowKeyJoin")
+  private SubScan subScanForRowKeyJoin;
+
+  /*
+  public HashJoinPOP(
+      @JsonProperty("left") PhysicalOperator left,
+      @JsonProperty("right") PhysicalOperator right,
+      @JsonProperty("conditions") List<JoinCondition> conditions,
+      @JsonProperty("joinType") JoinRelType joinType
+  ) {
+    this(left, right, conditions, joinType, false, JoinControl.DEFAULT);
+  }
+*/
   @JsonCreator
   public HashJoinPOP(@JsonProperty("left") PhysicalOperator left, @JsonProperty("right") PhysicalOperator right,
                      @JsonProperty("conditions") List<JoinCondition> conditions,
                      @JsonProperty("joinType") JoinRelType joinType,
-                     @JsonProperty("runtimeFilterDef") RuntimeFilterDef runtimeFilterDef) {
+                     @JsonProperty("runtimeFilterDef") RuntimeFilterDef runtimeFilterDef,
+                     @JsonProperty("isRowKeyJoin") boolean isRowKeyJoin,
+                     @JsonProperty("joinControl") int joinControl) {
     super(left, right, joinType, null, conditions);
     Preconditions.checkArgument(joinType != null, "Join type is missing for HashJoin Pop");
     this.runtimeFilterDef = runtimeFilterDef;
+    this.isRowKeyJoin = isRowKeyJoin;
+    this.subScanForRowKeyJoin = null;
+    this.joinControl = joinControl;
   }
 
   @VisibleForTesting
   public HashJoinPOP(PhysicalOperator left, PhysicalOperator right,
                      List<JoinCondition> conditions,
                      JoinRelType joinType) {
-    super(left, right, joinType, null, conditions);
-    Preconditions.checkArgument(joinType != null, "Join type is missing for HashJoin Pop");
+    this(left, right, conditions, joinType, null, false, JoinControl.DEFAULT);
+    // super(left, right, joinType, null, conditions);
+    // Preconditions.checkArgument(joinType != null, "Join type is missing for HashJoin Pop");
   }
 
+  @VisibleForTesting
+  public HashJoinPOP(PhysicalOperator left, PhysicalOperator right,
+                     List<JoinCondition> conditions,
+                     JoinRelType joinType,
+                     RuntimeFilterDef runtimeFilterDef) {
+    this(left, right, conditions, joinType, runtimeFilterDef, false, JoinControl.DEFAULT);
+  }
 
   @Override
   public PhysicalOperator getNewWithChildren(List<PhysicalOperator> children) {
         Preconditions.checkArgument(children.size() == 2);
 
-        HashJoinPOP newHashJoin = new HashJoinPOP(children.get(0), children.get(1), conditions, joinType, runtimeFilterDef);
+        HashJoinPOP newHashJoin = new HashJoinPOP(children.get(0), children.get(1), conditions, joinType, runtimeFilterDef,
+              isRowKeyJoin, joinControl);
         newHashJoin.setMaxAllocation(getMaxAllocation());
+        newHashJoin.setSubScanForRowKeyJoin(this.getSubScanForRowKeyJoin());
         return newHashJoin;
-  }
+    }
+
+    @JsonProperty("isRowKeyJoin")
+    public boolean isRowKeyJoin() {
+        return isRowKeyJoin;
+    }
+
+    @JsonProperty("joinControl")
+    public int getJoinControl() {
+        return joinControl;
+    }
+
+    @JsonProperty("subScanForRowKeyJoin")
+    public SubScan getSubScanForRowKeyJoin() {
+        return subScanForRowKeyJoin;
+    }
 
+    public void setSubScanForRowKeyJoin(SubScan subScan) {
+        this.subScanForRowKeyJoin = subScan;
+    }
 
   public HashJoinPOP flipIfRight() {
       if (joinType == JoinRelType.RIGHT) {
-            List<JoinCondition> flippedConditions = Lists.newArrayList();
-            for (JoinCondition c : conditions) {
-                flippedConditions.add(c.flip());
-            }
-            return new HashJoinPOP(right, left, flippedConditions, JoinRelType.LEFT, runtimeFilterDef);
-        } else {
-            return this;
+        List<JoinCondition> flippedConditions = Lists.newArrayList();
+        for (JoinCondition c : conditions) {
+          flippedConditions.add(c.flip());
+        }
+        return new HashJoinPOP(right, left, flippedConditions, JoinRelType.LEFT, runtimeFilterDef, isRowKeyJoin, joinControl);
+      } else {
+        return this;
       }
   }
 
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/config/HashPartitionSender.java b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/config/HashPartitionSender.java
index e7151ad..a586bbe 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/config/HashPartitionSender.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/config/HashPartitionSender.java
@@ -24,6 +24,7 @@ import org.apache.drill.exec.physical.MinorFragmentEndpoint;
 import org.apache.drill.exec.physical.base.AbstractSender;
 import org.apache.drill.exec.physical.base.PhysicalOperator;
 import org.apache.drill.exec.physical.base.PhysicalVisitor;
+import org.apache.drill.exec.physical.impl.partitionsender.Partitioner;
 import org.apache.drill.exec.proto.UserBitShared.CoreOperatorType;
 
 import com.fasterxml.jackson.annotation.JsonCreator;
@@ -35,25 +36,40 @@ public class HashPartitionSender extends AbstractSender {
   static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(HashPartitionSender.class);
 
   private final LogicalExpression expr;
+  private final int outgoingBatchSize;
 
   @JsonCreator
   public HashPartitionSender(@JsonProperty("receiver-major-fragment") int oppositeMajorFragmentId,
                              @JsonProperty("child") PhysicalOperator child,
                              @JsonProperty("expr") LogicalExpression expr,
-                             @JsonProperty("destinations") List<MinorFragmentEndpoint> endpoints) {
+                             @JsonProperty("destinations") List<MinorFragmentEndpoint> endpoints,
+                             @JsonProperty("outgoingBatchSize") int outgoingBatchSize) {
     super(oppositeMajorFragmentId, child, endpoints);
     this.expr = expr;
+    this.outgoingBatchSize = outgoingBatchSize;
+  }
+
+  public HashPartitionSender(int oppositeMajorFragmentId,
+                             PhysicalOperator child,
+                             LogicalExpression expr,
+                             List<MinorFragmentEndpoint> endpoints) {
+    this(oppositeMajorFragmentId, child, expr, endpoints, Partitioner.DEFAULT_RECORD_BATCH_SIZE);
   }
 
   @Override
   protected PhysicalOperator getNewWithChild(PhysicalOperator child) {
-    return new HashPartitionSender(oppositeMajorFragmentId, child, expr, destinations);
+    return new HashPartitionSender(oppositeMajorFragmentId, child, expr,
+        destinations, outgoingBatchSize);
   }
 
   public LogicalExpression getExpr() {
     return expr;
   }
 
+  public int getOutgoingBatchSize() {
+    return outgoingBatchSize;
+  }
+
   @Override
   public <T, X, E extends Throwable> T accept(PhysicalVisitor<T, X, E> physicalVisitor, X value) throws E {
     return physicalVisitor.visitHashPartitionSender(this, value);
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/config/RangePartitionExchange.java b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/config/RangePartitionExchange.java
new file mode 100644
index 0000000..a822d34
--- /dev/null
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/config/RangePartitionExchange.java
@@ -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.
+ */
+package org.apache.drill.exec.physical.config;
+
+import org.apache.drill.exec.physical.base.AbstractExchange;
+import org.apache.drill.exec.physical.base.PhysicalOperator;
+import org.apache.drill.exec.physical.base.PhysicalOperatorUtil;
+import org.apache.drill.exec.physical.base.Receiver;
+import org.apache.drill.exec.physical.base.Sender;
+import org.apache.drill.exec.planner.physical.PartitionFunction;
+
+import com.fasterxml.jackson.annotation.JsonCreator;
+import com.fasterxml.jackson.annotation.JsonProperty;
+import com.fasterxml.jackson.annotation.JsonTypeName;
+
+@JsonTypeName("range-partition-exchange")
+public class RangePartitionExchange extends AbstractExchange {
+  static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(RangePartitionExchange.class);
+
+  private final PartitionFunction partitionFunction;
+
+  @JsonCreator
+  public RangePartitionExchange(@JsonProperty("child") PhysicalOperator child,
+      @JsonProperty("partition-function") PartitionFunction partitionFunction) {
+    super(child);
+    this.partitionFunction = partitionFunction;
+  }
+
+  @Override
+  public Sender getSender(int minorFragmentId, PhysicalOperator child) {
+    return new RangePartitionSender(receiverMajorFragmentId, child,
+        PhysicalOperatorUtil.getIndexOrderedEndpoints(receiverLocations),
+        partitionFunction);
+  }
+
+  @Override
+  public Receiver getReceiver(int minorFragmentId) {
+    return new UnorderedReceiver(senderMajorFragmentId, PhysicalOperatorUtil.getIndexOrderedEndpoints(senderLocations), false);
+  }
+
+  @Override
+  protected PhysicalOperator getNewWithChild(PhysicalOperator child) {
+    return new RangePartitionExchange(child, partitionFunction);
+  }
+}
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/config/RowKeyJoinPOP.java b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/config/RowKeyJoinPOP.java
new file mode 100644
index 0000000..1b0980d
--- /dev/null
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/config/RowKeyJoinPOP.java
@@ -0,0 +1,96 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.drill.exec.physical.config;
+
+import java.util.Iterator;
+import java.util.List;
+
+import org.apache.drill.exec.physical.base.AbstractBase;
+import org.apache.drill.exec.physical.base.PhysicalOperator;
+import org.apache.drill.exec.physical.base.PhysicalVisitor;
+import org.apache.drill.exec.physical.base.SubScan;
+import org.apache.drill.exec.proto.UserBitShared.CoreOperatorType;
+
+import com.fasterxml.jackson.annotation.JsonCreator;
+import com.fasterxml.jackson.annotation.JsonProperty;
+import com.fasterxml.jackson.annotation.JsonTypeName;
+import org.apache.drill.shaded.guava.com.google.common.base.Preconditions;
+import org.apache.drill.shaded.guava.com.google.common.collect.Iterators;
+
+@JsonTypeName("rowkey-join")
+public class RowKeyJoinPOP extends AbstractBase {
+  static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(RowKeyJoinPOP.class);
+
+
+  private final PhysicalOperator left;
+  private final PhysicalOperator right;
+
+  @JsonProperty("subScanForRowKeyJoin")
+  private SubScan subScanForRowKeyJoin;
+
+  @JsonCreator
+  public RowKeyJoinPOP(
+      @JsonProperty("left") PhysicalOperator left,
+      @JsonProperty("right") PhysicalOperator right
+      ) {
+    this.left = left;
+    this.right = right;
+    this.subScanForRowKeyJoin = null;
+  }
+
+  @Override
+  public <T, X, E extends Throwable> T accept(PhysicalVisitor<T, X, E> physicalVisitor, X value) throws E {
+    return physicalVisitor.visitRowKeyJoin(this, value);
+  }
+
+  @Override
+  public PhysicalOperator getNewWithChildren(List<PhysicalOperator> children) {
+    Preconditions.checkArgument(children.size() == 2);
+    RowKeyJoinPOP rj = new RowKeyJoinPOP(children.get(0), children.get(1));
+    rj.setSubScanForRowKeyJoin(this.getSubScanForRowKeyJoin());
+    return rj;
+  }
+
+  @Override
+  public Iterator<PhysicalOperator> iterator() {
+    return Iterators.forArray(left, right);
+  }
+
+  public PhysicalOperator getLeft() {
+    return left;
+  }
+
+  public PhysicalOperator getRight() {
+    return right;
+  }
+
+  @JsonProperty("subScanForRowKeyJoin")
+  public SubScan getSubScanForRowKeyJoin() {
+    return subScanForRowKeyJoin;
+  }
+
+  public void setSubScanForRowKeyJoin(SubScan subScan) {
+    this.subScanForRowKeyJoin = subScan;
+  }
+
+  @Override
+  public int getOperatorType() {
+    return CoreOperatorType.ROWKEY_JOIN_VALUE;
+  }
+}
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/ScanBatch.java b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/ScanBatch.java
index 5ccf1c0..70b2852 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/ScanBatch.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/ScanBatch.java
@@ -174,55 +174,102 @@ public class ScanBatch implements CloseableRecordBatch {
     }
   }
 
-  @Override
-  public IterOutcome next() {
-    if (done) {
-      lastOutcome = IterOutcome.NONE;
-      return lastOutcome;
-    }
-    oContext.getStats().startProcessing();
-    try {
-      while (true) {
-        if (currentReader == null && !getNextReaderIfHas()) {
-          releaseAssets(); // All data has been read. Release resource.
-          done = true;
-          lastOutcome = IterOutcome.NONE;
-          return lastOutcome;
-        }
-        injector.injectChecked(context.getExecutionControls(), "next-allocate", OutOfMemoryException.class);
-        currentReader.allocate(mutator.fieldVectorMap());
+  /**
+   * This method is to perform scan specific actions when the scan needs to clean/reset readers and return NONE status
+   * @return NONE
+   */
+  private IterOutcome cleanAndReturnNone() {
+    if(isRepeatableScan) {
+      readers = readerList.iterator();
+      return IterOutcome.NONE;
+    } 
+    else {
+      releaseAssets(); // All data has been read. Release resource.
+      done = true;
+      return IterOutcome.NONE;}
+  }
 
-        recordCount = currentReader.next();
-        Preconditions.checkArgument(recordCount >= 0, "recordCount from RecordReader.next() should not be negative");
-        boolean isNewSchema = mutator.isNewSchema();
-        populateImplicitVectorsAndSetCount();
-        oContext.getStats().batchReceived(0, recordCount, isNewSchema);
-        logRecordBatchStats();
+  /**
+   * When receive zero record from current reader, update reader accordingly,
+   * and return the decision whether the iteration should continue
+   * @return whether we could continue iteration
+   * @throws Exception
+   */
+  private boolean shouldContinueAfterNoRecords() throws Exception {
+    logger.trace("scan got 0 record.");
+    if(isRepeatableScan) {
+      if (!currentReader.hasNext()) {
+        currentReader = null;
+        readers = readerList.iterator();
+        return false;
+      }
+      return true;
+    }
+    else {// Regular scan
+      currentReader.close();
+      currentReader = null;
+      return true;// In regular case, we always continue the iteration, if no more reader, we will break out at the head of loop
+    }
+  }
 
-        if (recordCount == 0) {
-          currentReader.close();
-          currentReader = null; // indicate currentReader is complete,
-                                // and fetch next reader in next loop iterator if required.
-        }
+  private IterOutcome internalNext() throws Exception {
+    while (true) {
+      if (currentReader == null && !getNextReaderIfHas()) {
+        logger.trace("currentReader is null");
+        return cleanAndReturnNone();
+      }
+      injector.injectChecked(context.getExecutionControls(), "next-allocate", OutOfMemoryException.class);
+      currentReader.allocate(mutator.fieldVectorMap());
+
+      recordCount = currentReader.next();
+      logger.trace("currentReader.next return recordCount={}", recordCount);
+      Preconditions.checkArgument(recordCount >= 0, "recordCount from RecordReader.next() should not be negative");
+      boolean isNewSchema = mutator.isNewSchema();
+      populateImplicitVectorsAndSetCount();
+      oContext.getStats().batchReceived(0, recordCount, isNewSchema);
+
+      boolean toContinueIter = true;
+      if (recordCount == 0) {
+        // If we got 0 record, we may need to clean and exit, but we may need to return a new schema in below code block,
+        // so we use toContinueIter to mark the decision whether we should continue the iteration
+        toContinueIter = shouldContinueAfterNoRecords();
+      }
 
-        if (isNewSchema) {
-          // Even when recordCount = 0, we should return return OK_NEW_SCHEMA if current reader presents a new schema.
-          // This could happen when data sources have a non-trivial schema with 0 row.
-          container.buildSchema(SelectionVectorMode.NONE);
-          schema = container.getSchema();
-          lastOutcome = IterOutcome.OK_NEW_SCHEMA;
-          return lastOutcome;
-        }
+      if (isNewSchema) {
+        // Even when recordCount = 0, we should return return OK_NEW_SCHEMA if current reader presents a new schema.
+        // This could happen when data sources have a non-trivial schema with 0 row.
+        container.buildSchema(SelectionVectorMode.NONE);
+        schema = container.getSchema();
+        lastOutcome = IterOutcome.OK_NEW_SCHEMA;
+        return lastOutcome;
+      }
 
-        // Handle case of same schema.
-        if (recordCount == 0) {
-            continue; // Skip to next loop iteration if reader returns 0 row and has same schema.
+      // Handle case of same schema.
+      if (recordCount == 0) {
+        if (toContinueIter) {
+          continue; // Skip to next loop iteration if reader returns 0 row and has same schema.
         } else {
-          // return OK if recordCount > 0 && ! isNewSchema
-          lastOutcome = IterOutcome.OK;
+          // Return NONE if recordCount == 0 && !isNewSchema
+          lastOutcome = IterOutcome.NONE;
           return lastOutcome;
         }
+      } else {
+        // return OK if recordCount > 0 && ! isNewSchema
+        lastOutcome = IterOutcome.OK;
+        return lastOutcome;
       }
+    }
+  }
+
+  @Override
+  public IterOutcome next() {
+    if (done) {
+      lastOutcome = IterOutcome.NONE;
+      return lastOutcome;
+    }
+    oContext.getStats().startProcessing();
+    try {
+      return internalNext();
     } catch (OutOfMemoryException ex) {
       clearFieldVectorMap();
       lastOutcome = IterOutcome.STOP;
@@ -236,7 +283,7 @@ public class ScanBatch implements CloseableRecordBatch {
         }
       }
       lastOutcome = IterOutcome.STOP;
-      throw UserException.internalError(e)
+      throw UserException.systemError(e)
           .addContext("Setup failed for", currentReaderClassName)
           .build(logger);
     } catch (UserException ex) {
@@ -244,7 +291,7 @@ public class ScanBatch implements CloseableRecordBatch {
       throw ex;
     } catch (Exception ex) {
       lastOutcome = IterOutcome.STOP;
-      throw UserException.internalError(ex).build(logger);
+      throw UserException.systemError(ex).build(logger);
     } finally {
       oContext.getStats().stopProcessing();
     }
@@ -287,9 +334,9 @@ public class ScanBatch implements CloseableRecordBatch {
       }
     } catch(SchemaChangeException e) {
       // No exception should be thrown here.
-      throw UserException.internalError(e)
-        .addContext("Failure while allocating implicit vectors")
-        .build(logger);
+      throw UserException.systemError(e)
+          .addContext("Failure while allocating implicit vectors")
+          .build(logger);
     }
   }
 
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/common/HashPartition.java b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/common/HashPartition.java
index 53655da..1f7da38 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/common/HashPartition.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/common/HashPartition.java
@@ -19,6 +19,7 @@ package org.apache.drill.exec.physical.impl.common;
 
 import org.apache.drill.shaded.guava.com.google.common.base.Preconditions;
 import org.apache.drill.shaded.guava.com.google.common.collect.Lists;
+import org.apache.commons.lang3.tuple.Pair;
 import org.apache.drill.common.exceptions.RetryAfterSpillException;
 import org.apache.drill.common.exceptions.UserException;
 import org.apache.drill.exec.cache.VectorSerializer;
@@ -417,6 +418,10 @@ public class HashPartition implements HashJoinMemoryCalculator.PartitionStat {
     hashTable.updateBatches();
   }
 
+  public Pair<VectorContainer, Integer> nextBatch() {
+    return hashTable.nextBatch();
+  }
+
   @Override
   public List<HashJoinMemoryCalculator.BatchStat> getInMemoryBatches() {
     return inMemoryBatchStats;
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/common/HashTable.java b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/common/HashTable.java
index 3bf4b86..5732458 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/common/HashTable.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/common/HashTable.java
@@ -17,6 +17,7 @@
  */
 package org.apache.drill.exec.physical.impl.common;
 
+import org.apache.commons.lang3.tuple.Pair;
 import org.apache.drill.exec.compile.TemplateClassDefinition;
 import org.apache.drill.exec.exception.SchemaChangeException;
 import org.apache.drill.exec.memory.BufferAllocator;
@@ -150,6 +151,8 @@ public interface HashTable {
   void setTargetBatchRowCount(int batchRowCount);
 
   int getTargetBatchRowCount();
+
+  Pair<VectorContainer, Integer> nextBatch();
 }
 
 
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/common/HashTableConfig.java b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/common/HashTableConfig.java
index b689390..fe329cc 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/common/HashTableConfig.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/common/HashTableConfig.java
@@ -22,6 +22,7 @@ import org.apache.drill.common.logical.data.NamedExpression;
 import com.fasterxml.jackson.annotation.JsonCreator;
 import com.fasterxml.jackson.annotation.JsonProperty;
 import com.fasterxml.jackson.annotation.JsonTypeName;
+import org.apache.drill.exec.planner.common.JoinControl;
 
 import java.util.List;
 
@@ -34,19 +35,37 @@ public class HashTableConfig  {
   private final List<NamedExpression> keyExprsBuild;
   private final List<NamedExpression> keyExprsProbe;
   private final List<Comparator> comparators;
+  private final int joinControl;
+
+  public HashTableConfig(
+      int initialCapacity,
+      float loadFactor,
+      List<NamedExpression> keyExprsBuild,
+      List<NamedExpression> keyExprsProbe,
+      List<Comparator> comparators
+  ) {
+    this(initialCapacity, false, loadFactor, keyExprsBuild, keyExprsProbe, comparators, JoinControl.DEFAULT);
+  }
 
   @JsonCreator
   public HashTableConfig(@JsonProperty("initialCapacity") int initialCapacity,
                          @JsonProperty("loadFactor") float loadFactor,
                          @JsonProperty("keyExprsBuild") List<NamedExpression> keyExprsBuild,
                          @JsonProperty("keyExprsProbe") List<NamedExpression> keyExprsProbe,
-                         @JsonProperty("comparators") List<Comparator> comparators) {
-    this.initialCapacity = initialCapacity;
-    this.initialSizeIsFinal = false;
-    this.loadFactor = loadFactor;
-    this.keyExprsBuild = keyExprsBuild;
-    this.keyExprsProbe = keyExprsProbe;
-    this.comparators = comparators;
+                         @JsonProperty("comparators") List<Comparator> comparators,
+                         @JsonProperty("joinControl") int joinControl) {
+    this(initialCapacity, false, loadFactor, keyExprsBuild, keyExprsProbe, comparators, joinControl);
+  }
+
+  public HashTableConfig(
+      int initialCapacity,
+      boolean initialSizeIsFinal,
+      float loadFactor,
+      List<NamedExpression> keyExprsBuild,
+      List<NamedExpression> keyExprsProbe,
+      List<Comparator> comparators
+  ) {
+    this(initialCapacity, initialSizeIsFinal, loadFactor, keyExprsBuild, keyExprsProbe, comparators, JoinControl.DEFAULT);
   }
 
   @JsonCreator
@@ -55,13 +74,16 @@ public class HashTableConfig  {
                          @JsonProperty("loadFactor") float loadFactor,
                          @JsonProperty("keyExprsBuild") List<NamedExpression> keyExprsBuild,
                          @JsonProperty("keyExprsProbe") List<NamedExpression> keyExprsProbe,
-                         @JsonProperty("comparators") List<Comparator> comparators) {
+                         @JsonProperty("comparators") List<Comparator> comparators,
+                         @JsonProperty("joinControl") int joinControl
+  ) {
     this.initialCapacity = initialCapacity;
     this.initialSizeIsFinal = initialSizeIsFinal;
     this.loadFactor = loadFactor;
     this.keyExprsBuild = keyExprsBuild;
     this.keyExprsProbe = keyExprsProbe;
     this.comparators = comparators;
+    this.joinControl = joinControl;
   }
 
   public HashTableConfig withInitialCapacity(int initialCapacity) {
@@ -70,7 +92,9 @@ public class HashTableConfig  {
       loadFactor,
       keyExprsBuild,
       keyExprsProbe,
-      comparators);
+      comparators,
+      JoinControl.DEFAULT
+    );
   }
 
   public int getInitialCapacity() {
@@ -97,4 +121,7 @@ public class HashTableConfig  {
     return comparators;
   }
 
+  public int getJoinControl() {
+    return joinControl;
+  }
 }
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/common/HashTableTemplate.java b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/common/HashTableTemplate.java
index fd42202..ae9a621 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/common/HashTableTemplate.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/common/HashTableTemplate.java
@@ -24,6 +24,7 @@ import java.util.Set;
 import javax.inject.Named;
 
 import org.apache.drill.shaded.guava.com.google.common.collect.Sets;
+import org.apache.commons.lang3.tuple.Pair;
 import org.apache.drill.common.types.TypeProtos.MinorType;
 import org.apache.drill.common.types.Types;
 import org.apache.drill.exec.compile.sig.RuntimeOverridden;
@@ -113,6 +114,8 @@ public abstract class HashTableTemplate implements HashTable {
 
   private int resizingTime = 0;
 
+  private Iterator<BatchHolder> htIter = null;
+
   // This class encapsulates the links, keys and values for up to BATCH_SIZE
   // *unique* records. Thus, suppose there are N incoming record batches, each
   // of size BATCH_SIZE..but they have M unique keys altogether, the number of
@@ -889,6 +892,26 @@ public abstract class HashTableTemplate implements HashTable {
     return vector;
   }
 
+  public Pair<VectorContainer, Integer> nextBatch() {
+    if (batchHolders == null || batchHolders.size() == 0) {
+      return null;
+    }
+    if (htIter == null) {
+      htIter = batchHolders.iterator();
+    }
+    if (htIter.hasNext()) {
+      BatchHolder bh = htIter.next();
+      // set the value count for the vectors in the batch
+      // TODO: investigate why the value count is not already set in the
+      // batch.. it seems even outputKeys() sets the value count explicitly
+      if (bh != null) {
+        bh.setValueCount();
+        return Pair.of(bh.htContainer, bh.maxOccupiedIdx);
+      }
+    }
+    return null;
+  }
+
   // These methods will be code-generated in the context of the outer class
   protected abstract void doSetup(@Named("incomingBuild") VectorContainer incomingBuild, @Named("incomingProbe") RecordBatch incomingProbe) throws SchemaChangeException;
 
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/join/HashJoinBatch.java b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/join/HashJoinBatch.java
index 929811c..a969ffd 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/join/HashJoinBatch.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/join/HashJoinBatch.java
@@ -25,8 +25,12 @@ import java.util.Set;
 
 import org.apache.drill.shaded.guava.com.google.common.base.Preconditions;
 import org.apache.drill.shaded.guava.com.google.common.collect.Lists;
+import org.apache.drill.shaded.guava.com.google.common.collect.Sets;
+import org.apache.drill.shaded.guava.com.google.common.collect.Iterables;
+
 import org.apache.commons.io.FileUtils;
 import org.apache.commons.lang3.mutable.MutableBoolean;
+import org.apache.commons.lang3.tuple.Pair;
 import org.apache.drill.common.exceptions.UserException;
 import org.apache.drill.common.expression.ErrorCollector;
 import org.apache.drill.common.expression.ErrorCollectorImpl;
@@ -57,19 +61,21 @@ import org.apache.drill.exec.physical.config.HashJoinPOP;
 import org.apache.drill.exec.physical.impl.aggregate.SpilledRecordbatch;
 import org.apache.drill.exec.physical.impl.common.AbstractSpilledPartitionMetadata;
 import org.apache.drill.exec.physical.impl.common.ChainedHashTable;
+import org.apache.drill.exec.physical.impl.common.Comparator;
 import org.apache.drill.exec.physical.impl.common.HashTable;
 import org.apache.drill.exec.physical.impl.common.HashTableConfig;
 import org.apache.drill.exec.physical.impl.common.HashTableStats;
-import org.apache.drill.exec.physical.impl.common.Comparator;
 import org.apache.drill.exec.physical.impl.common.HashPartition;
 import org.apache.drill.exec.physical.impl.common.SpilledState;
 import org.apache.drill.exec.physical.impl.spill.SpillSet;
+import org.apache.drill.exec.planner.common.JoinControl;
 import org.apache.drill.exec.record.AbstractBinaryRecordBatch;
 import org.apache.drill.exec.record.BatchSchema;
 import org.apache.drill.exec.record.JoinBatchMemoryManager;
 import org.apache.drill.exec.record.MaterializedField;
 import org.apache.drill.exec.record.RecordBatch;
 import org.apache.drill.exec.record.TypedFieldId;
+import org.apache.drill.exec.record.VectorContainer;
 import org.apache.drill.exec.record.VectorWrapper;
 import org.apache.drill.exec.util.record.RecordBatchStats;
 import org.apache.drill.exec.util.record.RecordBatchStats.RecordBatchIOType;
@@ -81,7 +87,6 @@ import org.apache.drill.exec.work.filter.BloomFilter;
 import org.apache.drill.exec.work.filter.BloomFilterDef;
 import org.apache.drill.exec.work.filter.RuntimeFilterDef;
 import org.apache.drill.exec.work.filter.RuntimeFilterReporter;
-import org.apache.drill.shaded.guava.com.google.common.collect.Sets;
 
 import static org.apache.drill.exec.record.RecordBatch.IterOutcome.EMIT;
 import static org.apache.drill.exec.record.RecordBatch.IterOutcome.NONE;
@@ -107,7 +112,7 @@ import static org.apache.drill.exec.record.RecordBatch.IterOutcome.OK_NEW_SCHEMA
  *   The code tracks these spilling "cycles". Normally any such "again" (i.e. cycle of 2 or greater) is a waste,
  *   indicating that the number of partitions chosen was too small.
  */
-public class HashJoinBatch extends AbstractBinaryRecordBatch<HashJoinPOP> {
+public class HashJoinBatch extends AbstractBinaryRecordBatch<HashJoinPOP> implements RowKeyJoin {
   protected static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(HashJoinBatch.class);
 
   /**
@@ -124,6 +129,8 @@ public class HashJoinBatch extends AbstractBinaryRecordBatch<HashJoinPOP> {
   // Join conditions
   private final List<JoinCondition> conditions;
 
+  private RowKeyJoin.RowKeyJoinState rkJoinState = RowKeyJoin.RowKeyJoinState.INITIAL;
+
   // Runtime generated class implementing HashJoinProbe interface
   private HashJoinProbe hashJoinProbe = null;
 
@@ -162,6 +169,16 @@ public class HashJoinBatch extends AbstractBinaryRecordBatch<HashJoinPOP> {
   // Schema of the probe side
   private BatchSchema probeSchema;
 
+  // Whether this HashJoin is used for a row-key based join
+  private boolean isRowKeyJoin = false;
+
+  private JoinControl joinControl;
+
+  // An iterator over the build side hash table (only applicable for row-key joins)
+  private boolean buildComplete = false;
+
+  // indicates if we have previously returned an output batch
+  private boolean firstOutputBatch = true;
 
   private int rightHVColPosition;
   private BufferAllocator allocator;
@@ -506,6 +523,24 @@ public class HashJoinBatch extends AbstractBinaryRecordBatch<HashJoinPOP> {
           return buildExecuteTermination;
         }
 
+        buildComplete = true;
+
+        if (isRowKeyJoin) {
+          // discard the first left batch which was fetched by buildSchema, and get the new
+          // one based on rowkey join
+          leftUpstream = next(left);
+
+          if (leftUpstream == IterOutcome.STOP || rightUpstream == IterOutcome.STOP) {
+            state = BatchState.STOP;
+            return leftUpstream;
+          }
+
+          if (leftUpstream == IterOutcome.OUT_OF_MEMORY || rightUpstream == IterOutcome.OUT_OF_MEMORY) {
+            state = BatchState.OUT_OF_MEMORY;
+            return leftUpstream;
+          }
+        }
+
         // Update the hash table related stats for the operator
         updateStats();
       }
@@ -614,7 +649,7 @@ public class HashJoinBatch extends AbstractBinaryRecordBatch<HashJoinPOP> {
       // No more output records, clean up and return
       state = BatchState.DONE;
 
-      this.cleanup();
+      cleanup();
 
       return IterOutcome.NONE;
     } catch (SchemaChangeException e) {
@@ -669,7 +704,7 @@ public class HashJoinBatch extends AbstractBinaryRecordBatch<HashJoinPOP> {
     }
 
     final HashTableConfig htConfig = new HashTableConfig((int) context.getOptions().getOption(ExecConstants.MIN_HASH_TABLE_SIZE),
-      true, HashTable.DEFAULT_LOAD_FACTOR, rightExpr, leftExpr, comparators);
+      true, HashTable.DEFAULT_LOAD_FACTOR, rightExpr, leftExpr, comparators, joinControl.asInt());
 
     // Create the chained hash table
     baseHashTable =
@@ -1129,6 +1164,9 @@ public class HashJoinBatch extends AbstractBinaryRecordBatch<HashJoinPOP> {
     joinIsRightOrFull = joinType == JoinRelType.RIGHT || joinType == JoinRelType.FULL;
     conditions = popConfig.getConditions();
     this.popConfig = popConfig;
+    this.isRowKeyJoin = popConfig.isRowKeyJoin();
+    this.joinControl = new JoinControl(popConfig.getJoinControl());
+
     rightExpr = new ArrayList<>(conditions.size());
     buildJoinColumns = Sets.newHashSet();
     List<SchemaPath> rightConditionPaths = new ArrayList<>();
@@ -1265,6 +1303,48 @@ public class HashJoinBatch extends AbstractBinaryRecordBatch<HashJoinPOP> {
     this.stats.setLongStat(Metric.SPILLED_PARTITIONS, numSpilled);
   }
 
+  /**
+   * Get the hash table iterator that is created for the build side of the hash join if
+   * this hash join was instantiated as a row-key join.
+   * @return hash table iterator or null if this hash join was not a row-key join or if it
+   * was a row-key join but the build has not yet completed.
+   */
+  @Override
+  public Pair<ValueVector, Integer> nextRowKeyBatch() {
+    if (buildComplete) {
+      // partition 0 because Row Key Join has only a single partition - no spilling
+      Pair<VectorContainer, Integer> pp = partitions[0].nextBatch();
+      if (pp != null) {
+        VectorWrapper<?> vw = Iterables.get(pp.getLeft(), 0);
+        ValueVector vv = vw.getValueVector();
+        return Pair.of(vv, pp.getRight());
+      }
+    } else if(partitions == null && firstOutputBatch) { //if there is data coming to right(build) side in build Schema stage, use it.
+      firstOutputBatch = false;
+      if ( right.getRecordCount() > 0 ) {
+        VectorWrapper<?> vw = Iterables.get(right, 0);
+        ValueVector vv = vw.getValueVector();
+        return Pair.of(vv, right.getRecordCount()-1);
+      }
+    }
+    return null;
+  }
+
+  @Override    // implement RowKeyJoin interface
+  public boolean hasRowKeyBatch() {
+    return buildComplete;
+  }
+
+  @Override   // implement RowKeyJoin interface
+  public BatchState getBatchState() {
+    return state;
+  }
+
+  @Override  // implement RowKeyJoin interface
+  public void setBatchState(BatchState newState) {
+    state = newState;
+  }
+
   @Override
   public void killIncoming(boolean sendUpstream) {
     wasKilled = true;
@@ -1290,6 +1370,16 @@ public class HashJoinBatch extends AbstractBinaryRecordBatch<HashJoinPOP> {
   }
 
   @Override
+  public void setRowKeyJoinState(RowKeyJoin.RowKeyJoinState newState) {
+    this.rkJoinState = newState;
+  }
+
+  @Override
+  public RowKeyJoin.RowKeyJoinState getRowKeyJoinState() {
+    return rkJoinState;
+  }
+
+  @Override
   public void close() {
     if (!spilledState.isFirstCycle()) { // spilling happened
       // In case closing due to cancellation, BaseRootExec.close() does not close the open
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/join/HashJoinBatchCreator.java b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/join/HashJoinBatchCreator.java
index d8cf1cb..183f971 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/join/HashJoinBatchCreator.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/join/HashJoinBatchCreator.java
@@ -21,6 +21,8 @@ import java.util.List;
 
 import org.apache.drill.common.exceptions.ExecutionSetupException;
 import org.apache.drill.exec.ops.ExecutorFragmentContext;
+import org.apache.drill.exec.physical.base.DbSubScan;
+import org.apache.drill.exec.physical.base.SubScan;
 import org.apache.drill.exec.physical.config.HashJoinPOP;
 import org.apache.drill.exec.physical.impl.BatchCreator;
 import org.apache.drill.exec.record.RecordBatch;
@@ -33,6 +35,13 @@ public class HashJoinBatchCreator implements BatchCreator<HashJoinPOP> {
   public HashJoinBatch getBatch(ExecutorFragmentContext context, HashJoinPOP config, List<RecordBatch> children)
       throws ExecutionSetupException {
     Preconditions.checkArgument(children.size() == 2);
-    return new HashJoinBatch(config, context, children.get(0), children.get(1));
+    HashJoinBatch hjBatch = new HashJoinBatch(config, context, children.get(0), children.get(1));
+    SubScan subScan = config.getSubScanForRowKeyJoin();
+    if (subScan != null
+        && subScan instanceof DbSubScan
+        && ((DbSubScan)subScan).isRestrictedSubScan()) {
+      ((DbSubScan)subScan).addJoinForRestrictedSubScan(hjBatch);
+    }
+    return hjBatch;
   }
 }
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/join/HashJoinHelper.java b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/join/HashJoinHelper.java
index b5b7183..37d8dee 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/join/HashJoinHelper.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/join/HashJoinHelper.java
@@ -178,7 +178,7 @@ public class HashJoinHelper {
     return compositeIndexes;
   }
 
-  public void setRecordMatched(int index) {
+  public boolean setRecordMatched(int index) {
     int batchIdx  = index >>> SHIFT_SIZE;
     int recordIdx = index & HashTable.BATCH_MASK;
 
@@ -186,7 +186,11 @@ public class HashJoinHelper {
     BuildInfo info = buildInfoList.get(batchIdx);
     BitSet bitVector = info.getKeyMatchBitVector();
 
+    if(bitVector.get(recordIdx)) {
+      return true;
+    }
     bitVector.set(recordIdx);
+    return false;
   }
 
   public void setCurrentIndex(int keyIndex, int batchIndex, int recordIndex) throws SchemaChangeException {
@@ -196,6 +200,11 @@ public class HashJoinHelper {
      * denotes the global index where the key for this record is
      * stored in the hash table
      */
+    if (keyIndex < 0) {
+      //receive a negative index, meaning we are not going to add this index (in distinct case when key already present)
+      return;
+    }
+
     int batchIdx  = keyIndex / HashTable.BATCH_SIZE;
     int offsetIdx = keyIndex % HashTable.BATCH_SIZE;
 
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/join/HashJoinProbeTemplate.java b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/join/HashJoinProbeTemplate.java
index a63f63d..c16812e 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/join/HashJoinProbeTemplate.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/join/HashJoinProbeTemplate.java
@@ -21,7 +21,9 @@ import java.util.ArrayList;
 import java.util.List;
 
 import org.apache.drill.exec.exception.SchemaChangeException;
+import org.apache.drill.exec.physical.config.HashJoinPOP;
 import org.apache.drill.exec.physical.impl.common.HashPartition;
+import org.apache.drill.exec.planner.common.JoinControl;
 import org.apache.drill.exec.record.BatchSchema;
 import org.apache.drill.exec.record.RecordBatch;
 import org.apache.drill.exec.record.RecordBatch.IterOutcome;
@@ -45,6 +47,10 @@ public abstract class HashJoinProbeTemplate implements HashJoinProbe {
   // Join type, INNER, LEFT, RIGHT or OUTER
   private JoinRelType joinType;
 
+  //joinControl object derived from the int type joinControl passed from outgoingBatch(HashJoinBatch)
+  //so we can do different things in hashtable for INTERSECT_DISTINCT and INTERSECT_ALL
+  private JoinControl joinControl;
+
   private HashJoinBatch outgoingJoinBatch = null;
 
   // Number of records to process on the probe side
@@ -125,6 +131,7 @@ public abstract class HashJoinProbeTemplate implements HashJoinProbe {
 
     partitionMask = numPartitions - 1; // e.g. 32 --> 0x1F
     bitsInMask = Integer.bitCount(partitionMask); // e.g. 0x1F -> 5
+    joinControl = new JoinControl(((HashJoinPOP)outgoingJoinBatch.getPopConfig()).getJoinControl());
 
     probeState = ProbeState.PROBE_PROJECT;
     this.recordsToProcess = 0;
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/join/RowKeyJoinBatch.java b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/join/RowKeyJoinBatch.java
new file mode 100644
index 0000000..7bca6bd
--- /dev/null
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/join/RowKeyJoinBatch.java
@@ -0,0 +1,284 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.drill.exec.physical.impl.join;
+
+
+import java.util.List;
+
+import org.apache.commons.lang3.tuple.Pair;
+import org.apache.drill.exec.exception.OutOfMemoryException;
+import org.apache.drill.exec.exception.SchemaChangeException;
+import org.apache.drill.exec.ops.FragmentContext;
+import org.apache.drill.exec.physical.config.RowKeyJoinPOP;
+import org.apache.drill.exec.record.AbstractRecordBatch;
+import org.apache.drill.exec.record.RecordBatch;
+import org.apache.drill.exec.record.TransferPair;
+import org.apache.drill.exec.record.VectorWrapper;
+import org.apache.drill.exec.record.selection.SelectionVector2;
+import org.apache.drill.exec.record.selection.SelectionVector4;
+import org.apache.drill.exec.vector.SchemaChangeCallBack;
+import org.apache.drill.exec.vector.ValueVector;
+
+import org.apache.drill.shaded.guava.com.google.common.collect.Iterables;
+import org.apache.drill.shaded.guava.com.google.common.collect.Lists;
+
+
+public class RowKeyJoinBatch extends AbstractRecordBatch<RowKeyJoinPOP> implements RowKeyJoin {
+  private static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(RowKeyJoinBatch.class);
+
+  // primary table side record batch
+  private final RecordBatch left;
+
+  // index table side record batch
+  private final RecordBatch right;
+
+  private boolean hasRowKeyBatch;
+  private IterOutcome leftUpstream = IterOutcome.NONE;
+  private IterOutcome rightUpstream = IterOutcome.NONE;
+  private final List<TransferPair> transfers = Lists.newArrayList();
+  private int recordCount = 0;
+  private SchemaChangeCallBack callBack = new SchemaChangeCallBack();
+  private RowKeyJoinState rkJoinState = RowKeyJoinState.INITIAL;
+
+  public RowKeyJoinBatch(RowKeyJoinPOP config, FragmentContext context, RecordBatch left, RecordBatch right)
+      throws OutOfMemoryException {
+    super(config, context, true /* need to build schema */);
+    this.left = left;
+    this.right = right;
+    this.hasRowKeyBatch = false;
+  }
+
+  @Override
+  public int getRecordCount() {
+    if (state == BatchState.DONE) {
+      return 0;
+    }
+    return recordCount;
+  }
+
+  @Override
+  public SelectionVector2 getSelectionVector2() {
+    throw new UnsupportedOperationException("RowKeyJoinBatch does not support selection vector");
+  }
+
+  @Override
+  public SelectionVector4 getSelectionVector4() {
+    throw new UnsupportedOperationException("RowKeyJoinBatch does not support selection vector");
+  }
+
+  @Override
+  protected void buildSchema() throws SchemaChangeException {
+    container.clear();
+
+    rightUpstream = next(right);
+
+    if (leftUpstream == IterOutcome.STOP || rightUpstream == IterOutcome.STOP) {
+      state = BatchState.STOP;
+      return;
+    }
+
+    if (right.getRecordCount() > 0) {
+      // set the hasRowKeyBatch flag such that calling next() on the left input
+      // would see the correct status
+      hasRowKeyBatch = true;
+    }
+
+    leftUpstream = next(left);
+
+    if (leftUpstream == IterOutcome.OUT_OF_MEMORY || rightUpstream == IterOutcome.OUT_OF_MEMORY) {
+      state = BatchState.OUT_OF_MEMORY;
+      return;
+    }
+
+    for(final VectorWrapper<?> v : left) {
+      final TransferPair pair = v.getValueVector().makeTransferPair(
+          container.addOrGet(v.getField(), callBack));
+      transfers.add(pair);
+    }
+
+    container.buildSchema(left.getSchema().getSelectionVectorMode());
+  }
+
+  @Override
+  public IterOutcome innerNext() {
+    if (state == BatchState.DONE) {
+      return IterOutcome.NONE;
+    }
+    try {
+      if (state == BatchState.FIRST && left.getRecordCount() > 0) {
+        logger.debug("First batch, outputting the batch with {} records.", left.getRecordCount());
+        // there is already a pending batch from left, output it
+        outputCurrentLeftBatch();
+        // Check if schema has changed (this is just to guard against potential changes to the
+        // output schema by outputCurrentLeftBatch.
+        if (callBack.getSchemaChangedAndReset()) {
+          return IterOutcome.OK_NEW_SCHEMA;
+        }
+        return IterOutcome.OK;
+      }
+
+      rightUpstream = next(right);
+
+      logger.debug("right input IterOutcome: {}", rightUpstream);
+
+      switch(rightUpstream) {
+      case NONE:
+      case OUT_OF_MEMORY:
+      case STOP:
+        rkJoinState = RowKeyJoinState.DONE;
+        state = BatchState.DONE;
+        return rightUpstream;
+      case OK_NEW_SCHEMA:
+      case OK:
+        // we got a new batch from the right input, set this flag
+        // such that subsequent check by a scan would indicate availability
+        // of the row keys.
+        while((rightUpstream == IterOutcome.OK || rightUpstream == IterOutcome.OK_NEW_SCHEMA) &&
+            right.getRecordCount() == 0) {
+          rightUpstream = next(right);
+          logger.trace("rowkeyjoin loop when recordCount == 0. rightUpstream {}", rightUpstream);
+        }
+
+        if (!hasRowKeyBatch && right.getRecordCount() > 0) {
+          hasRowKeyBatch = true;
+        }
+
+        logger.debug("right input num records = {}", right.getRecordCount());
+
+        if (hasRowKeyBatch) {
+          // get the next batch from left input
+          leftUpstream = next(left);
+
+          logger.debug("left input IterOutcome: {}", leftUpstream);
+
+          if (leftUpstream == IterOutcome.OK || leftUpstream == IterOutcome.OK_NEW_SCHEMA) {
+            logger.debug("left input num records = {}", left.getRecordCount());
+            if (left.getRecordCount() > 0) {
+              logger.debug("Outputting the left batch with {} records.", left.getRecordCount());
+              outputCurrentLeftBatch();
+              // Check if schema has changed (this is just to guard against potential changes to the
+              // output schema by outputCurrentLeftBatch, but in general the leftUpstream status should
+              // be sufficient)
+              if (callBack.getSchemaChangedAndReset()) {
+                return IterOutcome.OK_NEW_SCHEMA;
+              }
+            }
+          }
+        }
+
+        if (leftUpstream == IterOutcome.NONE) {
+          container.setRecordCount(0);
+          this.recordCount = 0;
+          return rightUpstream;
+        } else {
+          return leftUpstream;
+        }
+
+      default:
+        throw new IllegalStateException(String.format("Unknown state %s.", rightUpstream));
+      }
+    } finally {
+      if (state == BatchState.FIRST) {
+        state = BatchState.NOT_FIRST;
+      }
+      if (leftUpstream == IterOutcome.NONE && rkJoinState == RowKeyJoinState.PROCESSING) {
+        rkJoinState = RowKeyJoinState.INITIAL;
+      }
+    }
+  }
+
+  private void outputCurrentLeftBatch() {
+    //Schema change when state is FIRST shouldn't happen as buildSchema should
+    //take care of building the schema for the first batch. This check is introduced
+    //to guard against any schema change after buildSchema phase and reading
+    //the first batch of rows.
+    if (leftUpstream == IterOutcome.OK_NEW_SCHEMA && state == BatchState.FIRST ||
+        state == BatchState.NOT_FIRST) {
+      container.zeroVectors();
+      transfers.clear();
+
+      for(final VectorWrapper<?> v : left) {
+        final TransferPair pair = v.getValueVector().makeTransferPair(
+            container.addOrGet(v.getField(), callBack));
+        transfers.add(pair);
+      }
+
+      if (container.isSchemaChanged()) {
+        container.buildSchema(left.getSchema().getSelectionVectorMode());
+      }
+    }
+
+    for(TransferPair t : transfers) {
+      t.transfer();
+    }
+
+    container.setRecordCount(left.getRecordCount());
+    this.recordCount = left.getRecordCount();
+  }
+
+  @Override  // implement RowKeyJoin interface
+  public boolean hasRowKeyBatch() {
+    return hasRowKeyBatch;
+  }
+
+  @Override  // implement RowKeyJoin interface
+  public Pair<ValueVector, Integer> nextRowKeyBatch() {
+    if ( hasRowKeyBatch && right.getRecordCount() > 0 ) {
+      // since entire right row key batch will be returned to the caller, reset
+      // the hasRowKeyBatch to false
+      hasRowKeyBatch = false;
+      VectorWrapper<?> vw = Iterables.get(right, 0);
+      ValueVector vv = vw.getValueVector();
+      return Pair.of(vv, right.getRecordCount()-1);
+    }
+    return null;
+  }
+
+  @Override   // implement RowKeyJoin interface
+  public BatchState getBatchState() {
+    return state;
+  }
+
+  @Override  // implement RowKeyJoin interface
+  public void setBatchState(BatchState newState) {
+    state = newState;
+  }
+
+  @Override
+  public void setRowKeyJoinState(RowKeyJoinState newState) {
+    this.rkJoinState = newState;
+  }
+
+  @Override
+  public RowKeyJoinState getRowKeyJoinState() {
+    return rkJoinState;
+  }
+
+  @Override
+  public void killIncoming(boolean sendUpstream) {
+    left.kill(sendUpstream);
+    right.kill(sendUpstream);
+  }
+
+  @Override
+  public void close() {
+    rkJoinState = RowKeyJoinState.DONE;
+    super.close();
+  }
+
+}
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/join/HashJoinBatchCreator.java b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/join/RowKeyJoinBatchCreator.java
similarity index 63%
copy from exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/join/HashJoinBatchCreator.java
copy to exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/join/RowKeyJoinBatchCreator.java
index d8cf1cb..68aaec2 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/join/HashJoinBatchCreator.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/join/RowKeyJoinBatchCreator.java
@@ -21,18 +21,28 @@ import java.util.List;
 
 import org.apache.drill.common.exceptions.ExecutionSetupException;
 import org.apache.drill.exec.ops.ExecutorFragmentContext;
-import org.apache.drill.exec.physical.config.HashJoinPOP;
+import org.apache.drill.exec.physical.base.DbSubScan;
+import org.apache.drill.exec.physical.base.SubScan;
+import org.apache.drill.exec.physical.config.RowKeyJoinPOP;
 import org.apache.drill.exec.physical.impl.BatchCreator;
 import org.apache.drill.exec.record.RecordBatch;
 
 import org.apache.drill.shaded.guava.com.google.common.base.Preconditions;
 
-public class HashJoinBatchCreator implements BatchCreator<HashJoinPOP> {
+public class RowKeyJoinBatchCreator implements BatchCreator<RowKeyJoinPOP> {
 
   @Override
-  public HashJoinBatch getBatch(ExecutorFragmentContext context, HashJoinPOP config, List<RecordBatch> children)
+  public RowKeyJoinBatch getBatch(ExecutorFragmentContext context, RowKeyJoinPOP config, List<RecordBatch> children)
       throws ExecutionSetupException {
     Preconditions.checkArgument(children.size() == 2);
-    return new HashJoinBatch(config, context, children.get(0), children.get(1));
+    RowKeyJoinBatch rjBatch = new RowKeyJoinBatch(config, context, children.get(0), children.get(1));
+    SubScan subScan = config.getSubScanForRowKeyJoin();
+    if (subScan != null
+        && subScan instanceof DbSubScan
+        && ((DbSubScan)subScan).isRestrictedSubScan()) {
+      ((DbSubScan)subScan).addJoinForRestrictedSubScan(rjBatch);
+    }
+    return rjBatch;
   }
+
 }
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/mergereceiver/MergingRecordBatch.java b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/mergereceiver/MergingRecordBatch.java
index 12ee668..97f4b96 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/mergereceiver/MergingRecordBatch.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/mergereceiver/MergingRecordBatch.java
@@ -536,7 +536,10 @@ public class MergingRecordBatch extends AbstractRecordBatch<MergingReceiverPOP>
 
   @Override
   public BatchSchema getSchema() {
-    return outgoingContainer.getSchema();
+    if (outgoingContainer.hasSchema()) {
+      return outgoingContainer.getSchema();
+    }
+    return null;
   }
 
   @Override
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/partitionsender/Partitioner.java b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/partitionsender/Partitioner.java
index 316c262..2e2e760 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/partitionsender/Partitioner.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/partitionsender/Partitioner.java
@@ -29,6 +29,8 @@ import org.apache.drill.exec.physical.config.HashPartitionSender;
 import org.apache.drill.exec.record.RecordBatch;
 
 public interface Partitioner {
+  int DEFAULT_RECORD_BATCH_SIZE = (1 << 10) - 1;
+
   void setup(ExchangeFragmentContext context,
              RecordBatch incoming,
              HashPartitionSender popConfig,
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/rangepartitioner/RangePartitionRecordBatch.java b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/rangepartitioner/RangePartitionRecordBatch.java
new file mode 100644
index 0000000..d8fc94d
--- /dev/null
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/rangepartitioner/RangePartitionRecordBatch.java
@@ -0,0 +1,186 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.drill.exec.physical.impl.rangepartitioner;
+
+import java.util.List;
+
+import org.apache.drill.common.expression.FieldReference;
+import org.apache.drill.common.expression.SchemaPath;
+import org.apache.drill.common.types.TypeProtos;
+import org.apache.drill.common.types.Types;
+import org.apache.drill.exec.exception.OutOfMemoryException;
+import org.apache.drill.exec.exception.SchemaChangeException;
+import org.apache.drill.exec.expr.TypeHelper;
+import org.apache.drill.exec.ops.FragmentContext;
+import org.apache.drill.exec.physical.config.RangePartitionSender;
+import org.apache.drill.exec.record.AbstractSingleRecordBatch;
+import org.apache.drill.exec.record.MaterializedField;
+import org.apache.drill.exec.record.RecordBatch;
+import org.apache.drill.exec.record.TransferPair;
+import org.apache.drill.exec.record.VectorAccessible;
+import org.apache.drill.exec.record.VectorWrapper;
+import org.apache.drill.exec.vector.IntVector;
+import org.apache.drill.exec.vector.ValueVector;
+
+import org.apache.drill.shaded.guava.com.google.common.collect.Lists;
+
+/**
+ * A RangePartitionRecordBatch is a run-time operator that provides the ability to divide up the input rows
+ * into a fixed number of separate ranges or 'buckets' based on the values of a set of columns (the range
+ * partitioning columns).
+ */
+public class RangePartitionRecordBatch extends AbstractSingleRecordBatch<RangePartitionSender> {
+  static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(RangePartitionRecordBatch.class);
+
+  private int numPartitions;
+  private int recordCount;
+  private final IntVector partitionIdVector;
+  private final List<TransferPair> transfers;
+
+  public RangePartitionRecordBatch(RangePartitionSender popConfig, RecordBatch incoming, FragmentContext context) throws OutOfMemoryException {
+    super(popConfig, context, incoming);
+    this.numPartitions = popConfig.getDestinations().size();
+
+    SchemaPath outputPath = popConfig.getPartitionFunction().getPartitionFieldRef();
+    MaterializedField outputField = MaterializedField.create(outputPath.getAsNamePart().getName(), Types.required(TypeProtos.MinorType.INT));
+    this.partitionIdVector = (IntVector) TypeHelper.getNewVector(outputField, oContext.getAllocator());
+    this.transfers = Lists.newArrayList();
+  }
+
+
+  @Override
+  public void close() {
+    super.close();
+    partitionIdVector.clear();
+  }
+
+  @Override
+  protected void killIncoming(boolean sendUpstream) {
+    incoming.kill(sendUpstream);
+  }
+
+  @Override
+  public int getRecordCount() {
+    return recordCount;
+  }
+
+  @Override
+  protected IterOutcome doWork() {
+    int num = incoming.getRecordCount();
+    if (num > 0) {
+      // first setup the partition columns.  This needs to be done whenever we have
+      // a new batch (not just a new schema) because the partitioning function needs
+      // access to the correct value vector.
+      setupPartitionCols(incoming);
+
+      partitionIdVector.allocateNew(num);
+
+      recordCount = projectRecords(num, 0);
+      for (VectorWrapper<?> v : container) {
+        ValueVector.Mutator m = v.getValueVector().getMutator();
+        m.setValueCount(recordCount);
+      }
+    }
+    // returning OK here is fine because the base class AbstractSingleRecordBatch
+    // is handling the actual return status; thus if there was a new schema, the
+    // parent will get an OK_NEW_SCHEMA based on innerNext() in base class.
+    return IterOutcome.OK;
+  }
+
+  /**
+   * Sets up projection that will transfer all of the columns in batch, and also setup
+   * the partition column based on which partition a record falls into
+   *
+   * @param batch
+   * @throws SchemaChangeException
+   * @return True if the new schema differs from old schema, False otherwise
+   */
+  @Override
+  protected boolean setupNewSchema() throws SchemaChangeException {
+    container.clear();
+
+    for (VectorWrapper<?> vw : incoming) {
+      TransferPair tp = vw.getValueVector().getTransferPair(oContext.getAllocator());
+      transfers.add(tp);
+      container.add(tp.getTo());
+    }
+
+    container.add(this.partitionIdVector);
+    container.buildSchema(incoming.getSchema().getSelectionVectorMode());
+    /*
+     * Always return true because we transfer incoming. If upstream sent OK_NEW_SCHEMA,
+     * it would also generate new VVs. Since we get them via transfer, we should send
+     * OK_NEW_SCHEMA to downstream for it to re-init to the new VVs.
+     */
+    return true;
+  }
+
+  /**
+   *  Provide the partition function with the appropriate value vector(s) that
+   *  are involved in the range partitioning
+   *  @param batch
+   */
+  private void setupPartitionCols(VectorAccessible batch) {
+    List<VectorWrapper<?>> partitionCols = Lists.newArrayList();
+
+    for (VectorWrapper<?> vw : batch) {
+      if (isPartitioningColumn(vw.getField().getName())) {
+        partitionCols.add(vw);
+      }
+    }
+
+    popConfig.getPartitionFunction().setup(partitionCols);
+  }
+
+  private boolean isPartitioningColumn(String name) {
+    List<FieldReference> refList = popConfig.getPartitionFunction().getPartitionRefList();
+    for (FieldReference f : refList) {
+      if (f.getRootSegment().getPath().equals(name)) {
+        return true;
+      }
+    }
+    return false;
+  }
+
+  private int getPartition(int index) {
+    return popConfig.getPartitionFunction().eval(index, numPartitions);
+  }
+
+  /**
+   * For each incoming record, get the partition id it belongs to by invoking the
+   * partitioning function. Set this id in the output partitionIdVector.  For all other
+   * incoming value vectors, just do a transfer.
+   * @param recordCount
+   * @param firstOutputIndex
+   * @return the number of records projected
+   */
+  private final int projectRecords(final int recordCount, int firstOutputIndex) {
+    final int countN = recordCount;
+    int counter = 0;
+    for (int i = 0; i < countN; i++, firstOutputIndex++) {
+      int partition = getPartition(i);
+      partitionIdVector.getMutator().setSafe(i, partition);
+      counter++;
+    }
+    for(TransferPair t : transfers){
+        t.transfer();
+    }
+    return counter;
+  }
+
+}
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/rangepartitioner/RangePartitionSenderCreator.java b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/rangepartitioner/RangePartitionSenderCreator.java
new file mode 100644
index 0000000..082eaf4
--- /dev/null
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/rangepartitioner/RangePartitionSenderCreator.java
@@ -0,0 +1,46 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.drill.exec.physical.impl.rangepartitioner;
+
+import java.util.List;
+
+import org.apache.drill.common.exceptions.ExecutionSetupException;
+import org.apache.drill.exec.ops.ExecutorFragmentContext;
+import org.apache.drill.exec.physical.config.HashPartitionSender;
+import org.apache.drill.exec.physical.config.RangePartitionSender;
+import org.apache.drill.exec.physical.impl.RootCreator;
+import org.apache.drill.exec.physical.impl.RootExec;
+import org.apache.drill.exec.physical.impl.partitionsender.PartitionSenderRootExec;
+import org.apache.drill.exec.record.RecordBatch;
+
+import org.apache.drill.shaded.guava.com.google.common.base.Preconditions;
+
+public class RangePartitionSenderCreator implements RootCreator<RangePartitionSender> {
+
+  @Override
+  public RootExec getRoot(ExecutorFragmentContext context, RangePartitionSender config,
+                          List<RecordBatch> children) throws ExecutionSetupException {
+    Preconditions.checkArgument(children.size() == 1);
+    RangePartitionRecordBatch rangep = new RangePartitionRecordBatch(config, children.iterator().next(), context);
+    HashPartitionSender hpc = new HashPartitionSender(config.getOppositeMajorFragmentId(), config,
+        config.getPartitionFunction().getPartitionFieldRef(), config.getDestinations(),
+        RangePartitionSender.RANGE_PARTITION_OUTGOING_BATCH_SIZE);
+    return new PartitionSenderRootExec(context, rangep, hpc);
+  }
+
+}
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/PlannerPhase.java b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/PlannerPhase.java
index 613281a..ae55c9f 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/PlannerPhase.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/PlannerPhase.java
@@ -27,6 +27,8 @@ import org.apache.calcite.rel.rules.LoptOptimizeJoinRule;
 import org.apache.calcite.tools.RuleSet;
 import org.apache.calcite.tools.RuleSets;
 import org.apache.drill.exec.ops.OptimizerRulesContext;
+import org.apache.drill.exec.planner.index.rules.DbScanSortRemovalRule;
+import org.apache.drill.exec.planner.index.rules.DbScanToIndexScanPrule;
 import org.apache.drill.exec.planner.logical.DrillAggregateRule;
 import org.apache.drill.exec.planner.logical.DrillCorrelateRule;
 import org.apache.drill.exec.planner.logical.DrillFilterAggregateTransposeRule;
@@ -63,7 +65,7 @@ import org.apache.drill.exec.planner.physical.FilterPrule;
 import org.apache.drill.exec.planner.physical.HashAggPrule;
 import org.apache.drill.exec.planner.physical.HashJoinPrule;
 import org.apache.drill.exec.planner.physical.LimitPrule;
-import org.apache.drill.exec.planner.physical.LimitUnionExchangeTransposeRule;
+import org.apache.drill.exec.planner.physical.LimitExchangeTransposeRule;
 import org.apache.drill.exec.planner.physical.MergeJoinPrule;
 import org.apache.drill.exec.planner.physical.NestedLoopJoinPrule;
 import org.apache.drill.exec.planner.physical.PlannerSettings;
@@ -192,6 +194,7 @@ public enum PlannerPhase {
     public RuleSet getRules(OptimizerRulesContext context, Collection<StoragePlugin> plugins) {
       return PlannerPhase.mergedRuleSets(
           PlannerPhase.getPhysicalRules(context),
+          getIndexRules(context),
           getStorageRules(context, plugins, this));
     }
   },
@@ -310,6 +313,7 @@ public enum PlannerPhase {
       // RuleInstance.PROJECT_SET_OP_TRANSPOSE_RULE,
       RuleInstance.PROJECT_WINDOW_TRANSPOSE_RULE,
       DrillPushProjectIntoScanRule.INSTANCE,
+      DrillPushProjectIntoScanRule.DRILL_LOGICAL_INSTANCE,
 
       /*
        Convert from Calcite Logical to Drill Logical Rules.
@@ -384,6 +388,32 @@ public enum PlannerPhase {
 
     return RuleSets.ofList(pruneRules);
   }
+  /**
+   *
+   */
+  static RuleSet getIndexRules(OptimizerRulesContext optimizerRulesContext) {
+    final PlannerSettings ps = optimizerRulesContext.getPlannerSettings();
+    if (!ps.isIndexPlanningEnabled()) {
+      return RuleSets.ofList(ImmutableSet.<RelOptRule>builder().build());
+    }
+
+    final ImmutableSet<RelOptRule> indexRules = ImmutableSet.<RelOptRule>builder()
+        .add(
+            DbScanToIndexScanPrule.REL_FILTER_SCAN,
+            DbScanToIndexScanPrule.SORT_FILTER_PROJECT_SCAN,
+            DbScanToIndexScanPrule.SORT_PROJECT_FILTER_PROJECT_SCAN,
+            DbScanToIndexScanPrule.PROJECT_FILTER_PROJECT_SCAN,
+            DbScanToIndexScanPrule.SORT_PROJECT_FILTER_SCAN,
+            DbScanToIndexScanPrule.FILTER_PROJECT_SCAN,
+            DbScanToIndexScanPrule.FILTER_SCAN,
+            DbScanSortRemovalRule.INDEX_SORT_EXCHANGE_PROJ_SCAN,
+            DbScanSortRemovalRule.INDEX_SORT_EXCHANGE_SCAN,
+            DbScanSortRemovalRule.INDEX_SORT_SCAN,
+            DbScanSortRemovalRule.INDEX_SORT_PROJ_SCAN
+        )
+        .build();
+    return RuleSets.ofList(indexRules);
+  }
 
   /**
    *   Get an immutable list of pruning rules that will be used post physical planning.
@@ -458,7 +488,7 @@ public enum PlannerPhase {
     ruleList.add(WriterPrule.INSTANCE);
     ruleList.add(WindowPrule.INSTANCE);
     ruleList.add(PushLimitToTopN.INSTANCE);
-    ruleList.add(LimitUnionExchangeTransposeRule.INSTANCE);
+    ruleList.add(LimitExchangeTransposeRule.INSTANCE);
     ruleList.add(UnionAllPrule.INSTANCE);
     ruleList.add(ValuesPrule.INSTANCE);
     ruleList.add(DirectScanPrule.INSTANCE);
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/common/DrillJoinRelBase.java b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/common/DrillJoinRelBase.java
index fa7bda1..434016f 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/common/DrillJoinRelBase.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/common/DrillJoinRelBase.java
@@ -174,7 +174,11 @@ public abstract class DrillJoinRelBase extends Join implements DrillRelNode {
   private RelOptCost computeHashJoinCostWithKeySize(RelOptPlanner planner, int keySize, RelMetadataQuery mq) {
     double probeRowCount = mq.getRowCount(this.getLeft());
     double buildRowCount = mq.getRowCount(this.getRight());
+    return computeHashJoinCostWithRowCntKeySize(planner, probeRowCount, buildRowCount, keySize);
+  }
 
+  public static RelOptCost computeHashJoinCostWithRowCntKeySize(RelOptPlanner planner, double probeRowCount,
+                                                                double buildRowCount, int keySize) {
     // cpu cost of hashing the join keys for the build side
     double cpuCostBuild = DrillCostBase.HASH_CPU_COST * keySize * buildRowCount;
     // cpu cost of hashing the join keys for the probe side
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/common/DrillRelOptUtil.java b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/common/DrillRelOptUtil.java
index b39328e..32f6ac2 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/common/DrillRelOptUtil.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/common/DrillRelOptUtil.java
@@ -21,11 +21,15 @@ import java.util.AbstractList;
 import java.util.ArrayList;
 import java.util.Collection;
 import java.util.HashMap;
+import java.util.LinkedHashMap;
 import java.util.List;
 import java.util.Map;
+import java.util.Set;
 
 import org.apache.drill.shaded.guava.com.google.common.collect.ImmutableList;
+import org.apache.drill.shaded.guava.com.google.common.collect.ImmutableMap;
 import org.apache.drill.shaded.guava.com.google.common.collect.Lists;
+import org.apache.drill.shaded.guava.com.google.common.collect.Sets;
 import org.apache.calcite.plan.RelOptUtil;
 import org.apache.calcite.rel.RelNode;
 import org.apache.calcite.rel.core.Project;
@@ -41,14 +45,19 @@ import org.apache.calcite.rex.RexNode;
 import org.apache.calcite.rex.RexVisitor;
 import org.apache.calcite.rex.RexVisitorImpl;
 import org.apache.calcite.sql.SqlKind;
+import org.apache.calcite.sql.fun.SqlStdOperatorTable;
 import org.apache.calcite.sql.type.SqlTypeName;
 import org.apache.calcite.sql.validate.SqlValidatorUtil;
 import org.apache.calcite.util.Pair;
 import org.apache.calcite.util.Util;
+import org.apache.drill.common.expression.PathSegment;
+import org.apache.drill.common.expression.SchemaPath;
 import org.apache.drill.common.types.TypeProtos;
 import org.apache.drill.common.types.Types;
 import org.apache.drill.exec.planner.logical.DrillRelFactories;
+import org.apache.drill.exec.planner.logical.FieldsReWriterUtil;
 import org.apache.drill.exec.resolver.TypeCastRules;
+import org.apache.drill.exec.util.Utilities;
 
 /**
  * Utility class that is a subset of the RelOptUtil class and is a placeholder for Drill specific
@@ -413,4 +422,132 @@ public abstract class DrillRelOptUtil {
     }
     return false;
   }
+
+  /**
+   * Stores information about fields, their names and types.
+   * Is responsible for creating mapper which used in field re-writer visitor.
+   */
+  public static class ProjectPushInfo {
+    private final List<SchemaPath> fields;
+    private final FieldsReWriterUtil.FieldsReWriter reWriter;
+    private final List<String> fieldNames;
+    private final List<RelDataType> types;
+
+    public ProjectPushInfo(List<SchemaPath> fields, Map<String, FieldsReWriterUtil.DesiredField> desiredFields) {
+      this.fields = fields;
+      this.fieldNames = new ArrayList<>();
+      this.types = new ArrayList<>();
+
+      Map<RexNode, Integer> mapper = new HashMap<>();
+
+      int index = 0;
+      for (Map.Entry<String, FieldsReWriterUtil.DesiredField> entry : desiredFields.entrySet()) {
+        fieldNames.add(entry.getKey());
+        FieldsReWriterUtil.DesiredField desiredField = entry.getValue();
+        types.add(desiredField.getType());
+        for (RexNode node : desiredField.getNodes()) {
+          mapper.put(node, index);
+        }
+        index++;
+      }
+      this.reWriter = new FieldsReWriterUtil.FieldsReWriter(mapper);
+    }
+
+    public List<SchemaPath> getFields() {
+      return fields;
+    }
+
+    public FieldsReWriterUtil.FieldsReWriter getInputReWriter() {
+      return reWriter;
+    }
+
+    /**
+     * Creates new row type based on stores types and field names.
+     *
+     * @param factory factory for data type descriptors.
+     * @return new row type
+     */
+    public RelDataType createNewRowType(RelDataTypeFactory factory) {
+      return factory.createStructType(types, fieldNames);
+    }
+  }
+
+  public static ProjectPushInfo getFieldsInformation(RelDataType rowType, List<RexNode> projects) {
+    ProjectFieldsVisitor fieldsVisitor = new ProjectFieldsVisitor(rowType);
+    for (RexNode exp : projects) {
+      PathSegment segment = exp.accept(fieldsVisitor);
+      fieldsVisitor.addField(segment);
+    }
+
+    return fieldsVisitor.getInfo();
+  }
+
+  /**
+   * Visitor that finds the set of inputs that are used.
+   */
+  private static class ProjectFieldsVisitor extends RexVisitorImpl<PathSegment> {
+    private final List<String> fieldNames;
+    private final List<RelDataTypeField> fields;
+
+    private final Set<SchemaPath> newFields = Sets.newLinkedHashSet();
+    private final Map<String, FieldsReWriterUtil.DesiredField> desiredFields = new LinkedHashMap<>();
+
+    ProjectFieldsVisitor(RelDataType rowType) {
+      super(true);
+      this.fieldNames = rowType.getFieldNames();
+      this.fields = rowType.getFieldList();
+    }
+
+    void addField(PathSegment segment) {
+      if (segment != null && segment instanceof PathSegment.NameSegment) {
+        newFields.add(new SchemaPath((PathSegment.NameSegment) segment));
+      }
+    }
+
+    ProjectPushInfo getInfo() {
+      return new ProjectPushInfo(ImmutableList.copyOf(newFields), ImmutableMap.copyOf(desiredFields));
+    }
+
+    @Override
+    public PathSegment visitInputRef(RexInputRef inputRef) {
+      int index = inputRef.getIndex();
+      String name = fieldNames.get(index);
+      RelDataTypeField field = fields.get(index);
+      addDesiredField(name, field.getType(), inputRef);
+      return new PathSegment.NameSegment(name);
+    }
+
+    @Override
+    public PathSegment visitCall(RexCall call) {
+      String itemStarFieldName = FieldsReWriterUtil.getFieldNameFromItemStarField(call, fieldNames);
+      if (itemStarFieldName != null) {
+        addDesiredField(itemStarFieldName, call.getType(), call);
+        return new PathSegment.NameSegment(itemStarFieldName);
+      }
+
+      if (SqlStdOperatorTable.ITEM.equals(call.getOperator())) {
+        PathSegment mapOrArray = call.operands.get(0).accept(this);
+        if (mapOrArray != null) {
+          if (call.operands.get(1) instanceof RexLiteral) {
+            return mapOrArray.cloneWithNewChild(Utilities.convertLiteral((RexLiteral) call.operands.get(1)));
+          }
+          return mapOrArray;
+        }
+      } else {
+        for (RexNode operand : call.operands) {
+          addField(operand.accept(this));
+        }
+      }
+      return null;
+    }
+
+    private void addDesiredField(String name, RelDataType type, RexNode originalNode) {
+      FieldsReWriterUtil.DesiredField desiredField = desiredFields.get(name);
+      if (desiredField == null) {
+        desiredFields.put(name, new FieldsReWriterUtil.DesiredField(name, type, originalNode));
+      } else {
+        desiredField.addNode(originalNode);
+      }
+    }
+  }
 }
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/common/DrillScanRelBase.java b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/common/DrillScanRelBase.java
index 0331475..6f8ee0e 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/common/DrillScanRelBase.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/common/DrillScanRelBase.java
@@ -17,36 +17,59 @@
  */
 package org.apache.drill.exec.planner.common;
 
+import java.io.IOException;
+import java.util.List;
+import org.apache.drill.common.expression.SchemaPath;
+import org.apache.drill.exec.physical.base.GroupScan;
 import org.apache.drill.exec.planner.logical.DrillTable;
+import org.apache.drill.exec.planner.logical.DrillTranslatableTable;
+import org.apache.drill.common.exceptions.DrillRuntimeException;
 import org.apache.calcite.rel.core.TableScan;
-import org.apache.calcite.plan.Convention;
 import org.apache.calcite.plan.RelOptCluster;
 import org.apache.calcite.plan.RelOptTable;
 import org.apache.calcite.plan.RelTraitSet;
 import org.apache.drill.exec.util.Utilities;
 
 /**
- * Base class for logical scan rel implemented in Drill.
- * NOTE: we should eventually make this class independent of TableAccessRelBase and then
- * make it the base class for logical and physical scan rels.
+ * Base class for logical/physical scan rel implemented in Drill.
  */
 public abstract class DrillScanRelBase extends TableScan implements DrillRelNode {
+  private GroupScan groupScan;
   protected final DrillTable drillTable;
 
-  public DrillScanRelBase(Convention convention, RelOptCluster cluster, RelTraitSet traits, RelOptTable table) {
+  public DrillScanRelBase(RelOptCluster cluster,
+                          RelTraitSet traits,
+                          RelOptTable table,
+                          final List<SchemaPath> columns) {
     super(cluster, traits, table);
     this.drillTable = Utilities.getDrillTable(table);
     assert drillTable != null;
+    try {
+      this.groupScan = drillTable.getGroupScan().clone(columns);
+    } catch (final IOException e) {
+      throw new DrillRuntimeException("Failure creating scan.", e);
+    }
   }
 
-  public DrillScanRelBase(Convention convention, RelOptCluster cluster, RelTraitSet traits, RelOptTable table,
-      DrillTable drillTable) {
+  public DrillScanRelBase(RelOptCluster cluster,
+                          RelTraitSet traits,
+                          GroupScan grpScan,
+                          RelOptTable table) {
     super(cluster, traits, table);
-    this.drillTable = drillTable;
+    DrillTable unwrap = table.unwrap(DrillTable.class);
+    if (unwrap == null) {
+      unwrap = table.unwrap(DrillTranslatableTable.class).getDrillTable();
+    }
+    this.drillTable = unwrap;
+    assert drillTable != null;
+    this.groupScan = grpScan;
   }
 
   public DrillTable getDrillTable() {
     return drillTable;
   }
 
+  public GroupScan getGroupScan() {
+    return groupScan;
+  }
 }
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/common/JoinControl.java b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/common/JoinControl.java
new file mode 100644
index 0000000..b051ec5
--- /dev/null
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/common/JoinControl.java
@@ -0,0 +1,53 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.drill.exec.planner.common;
+
+/**
+ * For the int type control,
+ * the meaning of each bit start from lowest:
+ * bit 0: intersect or not, 0 -- default(no intersect), 1 -- INTERSECT (DISTINCT as default)
+ * bit 1: intersect type, 0 -- default (DISTINCT), 1 -- INTERSECT_ALL
+ */
+public class JoinControl {
+  public final static int DEFAULT = 0;
+  public final static int INTERSECT_DISTINCT = 0x01;//0001
+  public final static int INTERSECT_ALL = 0x03; //0011
+  public final static int INTERSECT_MASK = 0x03;
+  private final int joinControl;
+
+  public JoinControl(int intControl) {
+    joinControl = intControl;
+  }
+
+  public boolean isIntersect() {
+    return (joinControl & INTERSECT_MASK) != 0;
+  }
+
+  public boolean isIntersectDistinct() {
+    return (joinControl & INTERSECT_MASK) == INTERSECT_DISTINCT;
+  }
+
+  public boolean isIntersectAll() {
+    return (joinControl & INTERSECT_MASK) == INTERSECT_ALL;
+  }
+
+  public int asInt() {
+    return joinControl;
+  }
+
+}
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/cost/DrillDefaultRelMetadataProvider.java b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/cost/DrillDefaultRelMetadataProvider.java
index 5c02e3d..a7d925b 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/cost/DrillDefaultRelMetadataProvider.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/cost/DrillDefaultRelMetadataProvider.java
@@ -29,5 +29,6 @@ public class DrillDefaultRelMetadataProvider {
   public static final RelMetadataProvider INSTANCE = ChainedRelMetadataProvider.of(ImmutableList
       .of(DrillRelMdRowCount.SOURCE,
           DrillRelMdDistinctRowCount.SOURCE,
+          DrillRelMdSelectivity.SOURCE,
           DefaultRelMetadataProvider.INSTANCE));
 }
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/cost/DrillRelMdSelectivity.java b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/cost/DrillRelMdSelectivity.java
new file mode 100644
index 0000000..da42229
--- /dev/null
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/cost/DrillRelMdSelectivity.java
@@ -0,0 +1,90 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.drill.exec.planner.cost;
+
+import org.apache.calcite.plan.volcano.RelSubset;
+import org.apache.calcite.rel.RelNode;
+import org.apache.calcite.rel.metadata.ReflectiveRelMetadataProvider;
+import org.apache.calcite.rel.metadata.RelMdSelectivity;
+import org.apache.calcite.rel.metadata.RelMdUtil;
+import org.apache.calcite.rel.metadata.RelMetadataProvider;
+import org.apache.calcite.rel.metadata.RelMetadataQuery;
+import org.apache.calcite.rex.RexNode;
+import org.apache.calcite.util.BuiltInMethod;
+import org.apache.drill.exec.physical.base.DbGroupScan;
+import org.apache.drill.exec.physical.base.GroupScan;
+import org.apache.drill.exec.planner.logical.DrillScanRel;
+import org.apache.drill.exec.planner.physical.PlannerSettings;
+import org.apache.drill.exec.planner.physical.PrelUtil;
+import org.apache.drill.exec.planner.physical.ScanPrel;
+
+import java.util.List;
+
+public class DrillRelMdSelectivity extends RelMdSelectivity {
+  private static final DrillRelMdSelectivity INSTANCE = new DrillRelMdSelectivity();
+
+  public static final RelMetadataProvider SOURCE = ReflectiveRelMetadataProvider.reflectiveSource(BuiltInMethod.SELECTIVITY.method, INSTANCE);
+
+
+  public Double getSelectivity(RelNode rel, RexNode predicate) {
+    if (rel instanceof RelSubset) {
+      return getSelectivity((RelSubset) rel, predicate);
+    } else if (rel instanceof DrillScanRel) {
+      return getScanSelectivity(rel, predicate);
+    } else if (rel instanceof ScanPrel) {
+      return getScanSelectivity(rel, predicate);
+    }else {
+      return super.getSelectivity(rel, RelMetadataQuery.instance(), predicate);
+    }
+  }
+
+  private Double getSelectivity(RelSubset rel, RexNode predicate) {
+    if (rel.getBest() != null) {
+      return getSelectivity(rel.getBest(), predicate);
+    } else {
+      List<RelNode> list = rel.getRelList();
+      if (list != null && list.size() > 0) {
+        return getSelectivity(list.get(0), predicate);
+      }
+    }
+    return RelMdUtil.guessSelectivity(predicate);
+  }
+
+  private Double getScanSelectivity(RelNode rel, RexNode predicate) {
+    double ROWCOUNT_UNKNOWN = -1.0;
+    GroupScan scan = null;
+    PlannerSettings settings = PrelUtil.getPlannerSettings(rel.getCluster().getPlanner());
+    if (rel instanceof DrillScanRel) {
+      scan = ((DrillScanRel) rel).getGroupScan();
+    } else if (rel instanceof ScanPrel) {
+      scan = ((ScanPrel) rel).getGroupScan();
+    }
+    if (scan != null) {
+      if (settings.isStatisticsEnabled()
+          && scan instanceof DbGroupScan) {
+        double filterRows = ((DbGroupScan) scan).getRowCount(predicate, rel);
+        double totalRows = ((DbGroupScan) scan).getRowCount(null, rel);
+        if (filterRows != ROWCOUNT_UNKNOWN &&
+            totalRows != ROWCOUNT_UNKNOWN && totalRows > 0) {
+          return Math.min(1.0, filterRows / totalRows);
+        }
+      }
+    }
+    return super.getSelectivity(rel, RelMetadataQuery.instance(), predicate);
+  }
+}
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/fragment/Materializer.java b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/fragment/Materializer.java
index 1040bc2..e33a38a 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/fragment/Materializer.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/fragment/Materializer.java
@@ -34,6 +34,7 @@ import org.apache.drill.exec.physical.base.SubScan;
 import org.apache.drill.shaded.guava.com.google.common.collect.Lists;
 import org.apache.drill.exec.physical.config.LateralJoinPOP;
 import org.apache.drill.exec.physical.config.UnnestPOP;
+import org.apache.drill.exec.physical.config.RowKeyJoinPOP;
 
 public class Materializer extends AbstractPhysicalVisitor<PhysicalOperator, Materializer.IndexedFragmentNode, ExecutionSetupException>{
   static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(Materializer.class);
@@ -68,14 +69,18 @@ public class Materializer extends AbstractPhysicalVisitor<PhysicalOperator, Mate
 
   @Override
   public PhysicalOperator visitGroupScan(GroupScan groupScan, IndexedFragmentNode iNode) throws ExecutionSetupException {
-    PhysicalOperator child = groupScan.getSpecificScan(iNode.getMinorFragmentId());
+    SubScan child = groupScan.getSpecificScan(iNode.getMinorFragmentId());
     child.setOperatorId(Short.MAX_VALUE & groupScan.getOperatorId());
+    // remember the subscan for future use
+    iNode.addSubScan(child);
     return child;
   }
 
   @Override
   public PhysicalOperator visitSubScan(SubScan subScan, IndexedFragmentNode value) throws ExecutionSetupException {
     value.addAllocation(subScan);
+    // remember the subscan for future use
+    value.addSubScan(subScan);
     // TODO - implement this
     return super.visitOp(subScan, value);
   }
@@ -135,9 +140,32 @@ public class Materializer extends AbstractPhysicalVisitor<PhysicalOperator, Mate
     return newOp;
   }
 
+  @Override
+  public PhysicalOperator visitRowKeyJoin(RowKeyJoinPOP op, IndexedFragmentNode iNode) throws ExecutionSetupException {
+    iNode.addAllocation(op);
+    List<PhysicalOperator> children = Lists.newArrayList();
+
+    children.add(op.getLeft().accept(this, iNode));
+
+    // keep track of the subscan in left input before visiting the right input such that subsequently we can
+    // use it for the rowkey join
+    SubScan subScanInLeftInput = iNode.getSubScan();
+
+    children.add(op.getRight().accept(this, iNode));
+
+    PhysicalOperator newOp = op.getNewWithChildren(children);
+    newOp.setCost(op.getCost());
+    newOp.setOperatorId(Short.MAX_VALUE & op.getOperatorId());
+
+    ((RowKeyJoinPOP)newOp).setSubScanForRowKeyJoin(subScanInLeftInput);
+
+    return newOp;
+  }
+
   public static class IndexedFragmentNode{
     private final Wrapper info;
     private final int minorFragmentId;
+    SubScan subScan = null;
 
     private final Deque<UnnestPOP> unnest = new ArrayDeque<>();
 
@@ -171,6 +199,13 @@ public class Materializer extends AbstractPhysicalVisitor<PhysicalOperator, Mate
       return this.unnest.removeFirst();
     }
 
+    public void addSubScan(SubScan subScan) {
+      this.subScan = subScan;
+    }
+
+    public SubScan getSubScan() {
+      return this.subScan;
+    }
   }
 
 }
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/index/ExprToRex.java b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/index/ExprToRex.java
new file mode 100644
index 0000000..f0593b6
--- /dev/null
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/index/ExprToRex.java
@@ -0,0 +1,107 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.drill.exec.planner.index;
+
+import org.apache.drill.shaded.guava.com.google.common.collect.Lists;
+import org.apache.calcite.rel.RelNode;
+import org.apache.calcite.rel.type.RelDataType;
+import org.apache.calcite.rel.type.RelDataTypeField;
+import org.apache.calcite.rex.RexBuilder;
+import org.apache.calcite.rex.RexNode;
+import org.apache.calcite.sql.fun.SqlStdOperatorTable;
+import org.apache.calcite.sql.type.SqlTypeName;
+import org.apache.drill.common.expression.CastExpression;
+import org.apache.drill.common.expression.PathSegment;
+import org.apache.drill.common.expression.SchemaPath;
+import org.apache.drill.common.expression.visitors.AbstractExprVisitor;
+import org.apache.drill.exec.planner.sql.TypeInferenceUtils;
+
+import java.util.List;
+
+/**
+ * Convert a logicalExpression to RexNode, notice the inputRel could be in an old plan, but newRowType is the newly built rowType
+ * that the new RexNode will be applied upon, so when reference fields, use newRowType, when need cluster, plannerSetting, etc, use old inputRel
+ */
+public class ExprToRex extends AbstractExprVisitor<RexNode, Void, RuntimeException> {
+
+  static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(ExprToRex.class);
+
+  private final RexBuilder builder;
+  private final RelDataType newRowType;
+  private final RelNode inputRel;
+
+  public ExprToRex(RelNode inputRel, RelDataType newRowType, RexBuilder builder) {
+    this.inputRel = inputRel;
+    this.newRowType = newRowType;
+    this.builder = builder;
+  }
+
+  public static RelDataTypeField findField(String fieldName, RelDataType rowType) {
+    final String rootPart = SchemaPath.parseFromString(fieldName).getRootSegmentPath();
+
+    for (RelDataTypeField f : rowType.getFieldList()) {
+      if (rootPart.equalsIgnoreCase(f.getName())) {
+        return f;
+      }
+    }
+    return null;
+  }
+
+  private RexNode makeItemOperator(String[] paths, int index, RelDataType rowType) {
+    if (index == 0) { //last one, return ITEM([0]-inputRef, [1] Literal)
+      final RelDataTypeField field = findField(paths[0], rowType);
+      return field == null ? null : builder.makeInputRef(field.getType(), field.getIndex());
+    }
+    return builder.makeCall(SqlStdOperatorTable.ITEM,
+                            makeItemOperator(paths, index - 1, rowType),
+                            builder.makeLiteral(paths[index]));
+  }
+
+  @Override
+  public RexNode visitSchemaPath(SchemaPath path, Void value) throws RuntimeException {
+    PathSegment.NameSegment rootSegment = path.getRootSegment();
+    if (rootSegment.isLastPath()) {
+      final RelDataTypeField field = findField(rootSegment.getPath(), newRowType);
+      return field == null ? null : builder.makeInputRef(field.getType(), field.getIndex());
+    }
+    List<String> paths = Lists.newArrayList();
+    while (rootSegment != null) {
+      paths.add(rootSegment.getPath());
+      rootSegment = (PathSegment.NameSegment) rootSegment.getChild();
+    }
+    return makeItemOperator(paths.toArray(new String[0]), paths.size() - 1, newRowType);
+  }
+
+
+  @Override
+  public RexNode visitCastExpression(CastExpression e, Void value) throws RuntimeException {
+    RexNode convertedInput = e.getInput().accept(this, null);
+    String typeStr = e.getMajorType().getMinorType().toString();
+
+    if (SqlTypeName.get(typeStr) == null) {
+      logger.debug("SqlTypeName could not find {}", typeStr);
+    }
+
+    SqlTypeName typeName = TypeInferenceUtils.getCalciteTypeFromDrillType(e.getMajorType().getMinorType());
+
+    RelDataType targetType = TypeInferenceUtils.createCalciteTypeWithNullability(
+        inputRel.getCluster().getTypeFactory(), typeName, true);
+    return builder.makeCast(targetType, convertedInput);
+  }
+
+}
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/index/FindFiltersForCollation.java b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/index/FindFiltersForCollation.java
new file mode 100644
index 0000000..3328ee2
--- /dev/null
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/index/FindFiltersForCollation.java
@@ -0,0 +1,199 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+/**
+  * Licensed to the Apache Software Foundation (ASF) under one
+  * or more contributor license agreements.  See the NOTICE file
+  * distributed with this work for additional information
+  * regarding copyright ownership.  The ASF licenses this file
+  * to you under the Apache License, Version 2.0 (the
+  * "License"); you may not use this file except in compliance
+  * with the License.  You may obtain a copy of the License at
+  *
+  * http://www.apache.org/licenses/LICENSE-2.0
+  *
+  * Unless required by applicable law or agreed to in writing, software
+  * distributed under the License is distributed on an "AS IS" BASIS,
+  * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+  * See the License for the specific language governing permissions and
+  * limitations under the License.
+  */
+package org.apache.drill.exec.planner.index;
+
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+
+import org.apache.calcite.rel.RelNode;
+import org.apache.calcite.rel.type.RelDataType;
+import org.apache.calcite.rex.RexCall;
+import org.apache.calcite.rex.RexCorrelVariable;
+import org.apache.calcite.rex.RexDynamicParam;
+import org.apache.calcite.rex.RexFieldAccess;
+import org.apache.calcite.rex.RexInputRef;
+import org.apache.calcite.rex.RexLiteral;
+import org.apache.calcite.rex.RexLocalRef;
+import org.apache.calcite.rex.RexNode;
+import org.apache.calcite.rex.RexOver;
+import org.apache.calcite.rex.RexRangeRef;
+import org.apache.calcite.rex.RexVisitorImpl;
+import org.apache.calcite.sql.SqlKind;
+import org.apache.calcite.sql.SqlOperator;
+import org.apache.calcite.sql.fun.SqlStdOperatorTable;
+
+import org.apache.drill.shaded.guava.com.google.common.collect.Lists;
+import org.apache.drill.shaded.guava.com.google.common.collect.Maps;
+import org.apache.drill.shaded.guava.com.google.common.collect.Sets;
+
+
+/**
+ * A visitor class that analyzes a filter condition (typically an index condition)
+ * and a supplied input collation and determines what the output collation would be
+ * after applying the filter.
+ */
+public class FindFiltersForCollation extends RexVisitorImpl<Boolean> {
+
+  // map of field collation (e.g [0]) to the list of filter conditions
+  // involving the same field
+  private Map<Integer, List<RexNode> > collationFilterMap = Maps.newHashMap();
+
+  // set of comparison operators that allow collation
+  private Set<SqlKind> allowedComparisons = Sets.newHashSet();
+
+  // input rel's rowtype
+  private RelDataType inputRowType;
+
+  private int currentFieldIndex;
+
+  public FindFiltersForCollation(RelNode input) {
+    super(true);
+    inputRowType = input.getRowType();
+    init();
+  }
+
+  /**
+   * Initialize the set of comparison operators that allow creating collation property.
+   */
+  private void init() {
+    allowedComparisons.addAll(SqlKind.COMPARISON);
+    allowedComparisons.add(SqlKind.LIKE);
+  }
+
+  /**
+   * For each RelFieldCollation, gather the set of filter conditions corresponding to it
+   * e.g suppose input collation is [0][1] and there are filter conditions: $0 = 5 AND $1 > 10 AND $1 <20
+   * then the map will have 2 entries:
+   * [0] -> ($0 = 5)
+   * [1] -> {($1 > 10), ($1 < 20)}
+   *
+   * @param indexCondition index condition to analyze
+   * @return list of output RelFieldCollation
+   */
+  public Map<Integer, List<RexNode> > analyze(RexNode indexCondition) {
+    for (int idx = 0; idx < inputRowType.getFieldCount(); idx++) {
+      currentFieldIndex = idx;
+      indexCondition.accept(this);
+    }
+    return collationFilterMap;
+  }
+
+  @Override
+  public Boolean visitInputRef(RexInputRef inputRef) {
+    if (inputRef.getIndex() == currentFieldIndex) {
+      return true;
+    }
+    return false;
+  }
+
+  @Override
+  public Boolean visitLiteral(RexLiteral literal) {
+    return true;
+  }
+
+  @Override
+  public Boolean visitOver(RexOver over) {
+    return false;
+  }
+
+  @Override
+  public Boolean visitCorrelVariable(RexCorrelVariable correlVariable) {
+    return false;
+  }
+
+  @Override
+  public Boolean visitCall(RexCall call) {
+    final SqlOperator op = call.getOperator();
+    final SqlKind kind = op.getKind();
+
+    if (kind == SqlKind.AND) {
+      for (RexNode n : call.getOperands()) {
+        n.accept(this);
+      }
+    } else if (kind == SqlKind.CAST) {
+      // For the filter analyzer itself, if the Project has not been pushed
+      // down below the Filter, then CAST is present in the filter condition.
+      // Return True for such case since CAST exprs are valid for collation.
+      // Otherwise, filter is only referencing output of the Project and we won't
+      // hit this else condition (i.e filter will have $0, $1 etc which would be
+      // visited by visitInputRef()).
+      return true;
+    } else if (op == SqlStdOperatorTable.ITEM) {
+      List<RexNode> ops = call.getOperands();
+      boolean left = ops.get(0).accept(this);
+      boolean right = ops.get(1).accept(this);
+      return left && right;
+    } else if (allowedComparisons.contains(kind)) {
+      List<RexNode> ops = call.getOperands();
+      boolean left = ops.get(0).accept(this);
+      boolean right = ops.get(1).accept(this);
+
+      if (left && right) {
+        if (collationFilterMap.containsKey(currentFieldIndex)) {
+          List<RexNode> n = collationFilterMap.get(currentFieldIndex);
+          n.add(call);
+        } else {
+          List<RexNode> clist = Lists.newArrayList();
+          clist.add(call);
+          collationFilterMap.put(currentFieldIndex, clist);
+        }
+        return true;
+      }
+    }
+    return false;
+  }
+
+  @Override
+  public Boolean visitDynamicParam(RexDynamicParam dynamicParam) {
+    return false;
+  }
+
+  @Override
+  public Boolean visitRangeRef(RexRangeRef rangeRef) {
+    return false;
+  }
+
+  @Override
+  public Boolean visitFieldAccess(RexFieldAccess fieldAccess) {
+    return false;
+  }
+
+  @Override
+  public Boolean visitLocalRef(RexLocalRef localRef) {
+    return false;
+  }
+
+}
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/index/FunctionalIndexHelper.java b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/index/FunctionalIndexHelper.java
new file mode 100644
index 0000000..3ff81b4
--- /dev/null
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/index/FunctionalIndexHelper.java
@@ -0,0 +1,215 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.drill.exec.planner.index;
+
+import org.apache.drill.shaded.guava.com.google.common.collect.Lists;
+import org.apache.drill.shaded.guava.com.google.common.collect.Maps;
+import org.apache.drill.shaded.guava.com.google.common.collect.Sets;
+
+import org.apache.calcite.rel.type.RelDataType;
+import org.apache.calcite.rel.type.RelDataTypeFactory;
+import org.apache.calcite.rel.type.RelDataTypeField;
+import org.apache.calcite.rel.type.RelDataTypeFieldImpl;
+import org.apache.calcite.rel.type.RelRecordType;
+import org.apache.calcite.rex.RexBuilder;
+import org.apache.calcite.rex.RexNode;
+import org.apache.calcite.sql.type.SqlTypeName;
+import org.apache.drill.common.expression.CastExpression;
+import org.apache.drill.common.expression.LogicalExpression;
+import org.apache.drill.common.expression.SchemaPath;
+import org.apache.drill.exec.physical.base.DbGroupScan;
+import org.apache.drill.exec.physical.base.IndexGroupScan;
+import org.apache.drill.exec.planner.common.DrillScanRelBase;
+import org.apache.calcite.rel.RelNode;
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.LinkedHashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+
+public class FunctionalIndexHelper {
+
+  public static RelDataType rewriteFunctionalRowType(RelNode origScan, IndexCallContext indexContext,
+                                                     FunctionalIndexInfo functionInfo) {
+    return rewriteFunctionalRowType(origScan, indexContext, functionInfo, null);
+  }
+  /**
+   * if a field in rowType serves only the to-be-replaced column(s), we should replace it with new name "$1",
+   * otherwise we should keep this dataTypeField and add a new one for "$1"
+   * @param origScan  the original scan whose rowtype is to be rewritten
+   * @param indexContext the index plan context
+   * @param functionInfo functional index information that may impact rewrite
+   * @return
+   */
+  public static RelDataType rewriteFunctionalRowType(RelNode origScan, IndexCallContext indexContext,
+      FunctionalIndexInfo functionInfo, Collection<SchemaPath> addedPaths) {
+    RelDataType origRowType = origScan.getRowType();
+    if (!functionInfo.hasFunctional()) {
+      return origRowType;
+    }
+
+    List<RelDataTypeField> fields = Lists.newArrayList();
+
+    Set<String> leftOutFieldNames  = Sets.newHashSet();
+    if (indexContext.getLeftOutPathsInFunctions() != null) {
+      for (LogicalExpression expr : indexContext.getLeftOutPathsInFunctions()) {
+        leftOutFieldNames.add(((SchemaPath) expr).getRootSegmentPath());
+      }
+    }
+
+    Set<String> fieldInFunctions  = Sets.newHashSet();
+    for (SchemaPath path: functionInfo.allPathsInFunction()) {
+      fieldInFunctions.add(path.getRootSegmentPath());
+    }
+
+    RelDataTypeFactory typeFactory = origScan.getCluster().getTypeFactory();
+
+    for ( RelDataTypeField field: origRowType.getFieldList()) {
+      final String fieldName = field.getName();
+      if (fieldInFunctions.contains(fieldName)) {
+        if (!leftOutFieldNames.contains(fieldName)) {
+          continue;
+        }
+      }
+
+      fields.add(new RelDataTypeFieldImpl(
+          SchemaPath.parseFromString(fieldName).getRootSegmentPath(), fields.size(),
+          typeFactory.createSqlType(SqlTypeName.ANY)));
+    }
+
+    final Collection<SchemaPath> toAddToRowType = (addedPaths == null)? functionInfo.allNewSchemaPaths() : addedPaths;
+
+    for (SchemaPath dollarPath : toAddToRowType) {
+      fields.add(
+          new RelDataTypeFieldImpl(dollarPath.getRootSegmentPath(), fields.size(),
+              origScan.getCluster().getTypeFactory().createSqlType(SqlTypeName.ANY)));
+    }
+
+    return new RelRecordType(fields);
+  }
+
+  /**
+   * For IndexScan in non-covering case, rowType to return contains only row_key('_id') of primary table.
+   * so the rowType for IndexScan should be converted from [Primary_table.row_key, primary_table.indexed_col]
+   * to [indexTable.row_key(primary_table.indexed_col), indexTable.<primary_key.row_key> (Primary_table.row_key)]
+   * This will impact the columns of scan, the rowType of ScanRel
+   *
+   * @param origScan
+   * @param idxMarker  the IndexableExprMarker that has analyzed original index condition on top of index scan
+   * @param idxScan
+   * @return
+   */
+  public static RelDataType convertRowTypeForIndexScan(DrillScanRelBase origScan,
+                                                       IndexableExprMarker idxMarker,
+                                                       IndexGroupScan idxScan,
+                                                       FunctionalIndexInfo functionInfo) {
+    RelDataTypeFactory typeFactory = origScan.getCluster().getTypeFactory();
+    List<RelDataTypeField> fields = new ArrayList<>();
+
+    Set<SchemaPath> rowPaths = new LinkedHashSet<>();
+    //row_key in the rowType of scan on primary table
+    RelDataTypeField rowkey_primary;
+
+    RelRecordType newRowType = null;
+
+    DbGroupScan scan = (DbGroupScan) IndexPlanUtils.getGroupScan(origScan);
+
+    //first add row_key of primary table,
+    rowkey_primary = new RelDataTypeFieldImpl(
+        scan.getRowKeyName(), fields.size(),
+        typeFactory.createSqlType(SqlTypeName.ANY));
+    fields.add(rowkey_primary);
+
+    Map<RexNode, LogicalExpression> idxExprMap = idxMarker.getIndexableExpression();
+
+    for (LogicalExpression indexedExpr : idxExprMap.values()) {
+      if (indexedExpr instanceof SchemaPath) {
+        rowPaths.add((SchemaPath) indexedExpr);
+      }
+      else if(indexedExpr instanceof CastExpression) {
+        SchemaPath newPath = functionInfo.getNewPathFromExpr(indexedExpr);
+        if(newPath != null) {
+          rowPaths.add(newPath);
+        }
+      }
+    }
+    for (SchemaPath newPath : rowPaths) {
+      fields.add(new RelDataTypeFieldImpl(
+          newPath.getRootSegmentPath(), fields.size(),
+          typeFactory.createSqlType(SqlTypeName.ANY)));
+    }
+
+    //update columns of groupscan accordingly
+    Set<RelDataTypeField> rowfields = Sets.newLinkedHashSet();
+    final List<SchemaPath> columns = Lists.newArrayList();
+    for (RelDataTypeField f : fields) {
+      SchemaPath path = SchemaPath.parseFromString(f.getName());
+      rowfields.add(new RelDataTypeFieldImpl(
+          path.getRootSegmentPath(), rowfields.size(),
+          typeFactory.createMapType(typeFactory.createSqlType(SqlTypeName.VARCHAR),
+              typeFactory.createSqlType(SqlTypeName.ANY))
+      ));
+      columns.add(path);
+    }
+    idxScan.setColumns(columns);
+
+    //rowtype does not take the whole path, but only the rootSegment of the SchemaPath
+    newRowType = new RelRecordType(Lists.newArrayList(rowfields));
+    return newRowType;
+  }
+
+  public static RexNode convertConditionForIndexScan(RexNode idxCondition,
+      RelNode origScan, RelDataType idxRowType, RexBuilder builder, FunctionalIndexInfo functionInfo) {
+    IndexableExprMarker marker = new IndexableExprMarker(origScan);
+    idxCondition.accept(marker);
+    SimpleRexRemap remap = new SimpleRexRemap(origScan, idxRowType, builder);
+    remap.setExpressionMap(functionInfo.getExprMap());
+
+    if (functionInfo.supportEqualCharConvertToLike()) {
+      final Map<LogicalExpression, LogicalExpression> indexedExprs = functionInfo.getExprMap();
+
+      final Map<RexNode, LogicalExpression> equalCastMap = marker.getEqualOnCastChar();
+
+      Map<RexNode, LogicalExpression> toRewriteEqualCastMap = Maps.newHashMap();
+
+      // the marker collected all equal-cast-varchar, now check which one we should replace
+      for (Map.Entry<RexNode, LogicalExpression> entry : equalCastMap.entrySet()) {
+        CastExpression expr = (CastExpression) entry.getValue();
+        //whether this cast varchar/char expression is indexed even the length is not the same
+        for (LogicalExpression indexed : indexedExprs.keySet()) {
+          if (indexed instanceof CastExpression) {
+            final CastExpression indexedCast = (CastExpression) indexed;
+            if (expr.getInput().equals(indexedCast.getInput())
+                && expr.getMajorType().getMinorType().equals(indexedCast.getMajorType().getMinorType())
+                //if expr's length < indexedCast's length, we should convert equal to LIKE for this condition
+                && expr.getMajorType().getPrecision() < indexedCast.getMajorType().getPrecision()) {
+              toRewriteEqualCastMap.put(entry.getKey(), entry.getValue());
+            }
+          }
+        }
+      }
+      if (toRewriteEqualCastMap.size() > 0) {
+        idxCondition = remap.rewriteEqualOnCharToLike(idxCondition, toRewriteEqualCastMap);
+      }
+    }
+
+    return remap.rewriteWithMap(idxCondition, marker.getIndexableExpression());
+  }
+}
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/index/IndexConditionInfo.java b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/index/IndexConditionInfo.java
new file mode 100644
index 0000000..aa51f9b
--- /dev/null
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/index/IndexConditionInfo.java
@@ -0,0 +1,250 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.drill.exec.planner.index;
+
+
+import org.apache.drill.shaded.guava.com.google.common.collect.ImmutableList;
+import org.apache.drill.shaded.guava.com.google.common.collect.Lists;
+import org.apache.drill.shaded.guava.com.google.common.collect.Maps;
+import org.apache.drill.shaded.guava.com.google.common.collect.Sets;
+import org.apache.calcite.plan.RelOptUtil;
+import org.apache.calcite.rex.RexBuilder;
+import org.apache.calcite.rex.RexNode;
+import org.apache.calcite.rex.RexUtil;
+import org.apache.drill.common.expression.LogicalExpression;
+import org.apache.drill.exec.planner.logical.DrillScanRel;
+import org.apache.drill.exec.planner.logical.partition.RewriteCombineBinaryOperators;
+import org.apache.calcite.rel.RelNode;
+import java.util.ArrayList;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+
+public class IndexConditionInfo {
+  public final RexNode indexCondition;
+  public final RexNode remainderCondition;
+  public final boolean hasIndexCol;
+
+  public IndexConditionInfo(RexNode indexCondition, RexNode remainderCondition, boolean hasIndexCol) {
+    this.indexCondition = indexCondition;
+    this.remainderCondition = remainderCondition;
+    this.hasIndexCol = hasIndexCol;
+  }
+
+  public static Builder newBuilder(RexNode condition,
+                                   Iterable<IndexDescriptor> indexes,
+                                   RexBuilder builder,
+                                   RelNode scan) {
+    return new Builder(condition, indexes, builder, scan);
+  }
+
+  public static class Builder {
+    final RexBuilder builder;
+    final RelNode scan;
+    final Iterable<IndexDescriptor> indexes;
+    private RexNode condition;
+
+    public Builder(RexNode condition,
+                   Iterable<IndexDescriptor> indexes,
+                   RexBuilder builder,
+                   RelNode scan
+    ) {
+      this.condition = condition;
+      this.builder = builder;
+      this.scan = scan;
+      this.indexes = indexes;
+    }
+
+    public Builder(RexNode condition,
+                   IndexDescriptor index,
+                   RexBuilder builder,
+                   DrillScanRel scan
+    ) {
+      this.condition = condition;
+      this.builder = builder;
+      this.scan = scan;
+      this.indexes = Lists.newArrayList(index);
+    }
+
+    /**
+     * Get a single IndexConditionInfo in which indexCondition has field  on all indexes in this.indexes
+     * @return
+     */
+    public IndexConditionInfo getCollectiveInfo(IndexLogicalPlanCallContext indexContext) {
+      Set<LogicalExpression> paths = Sets.newLinkedHashSet();
+      for ( IndexDescriptor index : indexes ) {
+        paths.addAll(index.getIndexColumns());
+        //paths.addAll(index.getNonIndexColumns());
+      }
+      return indexConditionRelatedToFields(Lists.newArrayList(paths), condition);
+    }
+
+    /*
+     * A utility function to check whether the given index hint is valid.
+     */
+    public boolean isValidIndexHint(IndexLogicalPlanCallContext indexContext) {
+      if (indexContext.indexHint.equals("")) { return false; }
+
+      for ( IndexDescriptor index: indexes ) {
+        if ( indexContext.indexHint.equals(index.getIndexName())) {
+          return true;
+        }
+      }
+      return false;
+    }
+
+    /**
+     * Get a map of Index=>IndexConditionInfo, each IndexConditionInfo has the separated condition and remainder condition.
+     * The map is ordered, so the last IndexDescriptor will have the final remainderCondition after separating conditions
+     * that are relevant to this.indexes. The conditions are separated on LEADING index columns.
+     * @return Map containing index{@link IndexDescriptor} and condition {@link IndexConditionInfo} pairs
+     */
+    public Map<IndexDescriptor, IndexConditionInfo> getFirstKeyIndexConditionMap() {
+
+      Map<IndexDescriptor, IndexConditionInfo> indexInfoMap = Maps.newLinkedHashMap();
+
+      RexNode initCondition = condition;
+      for(IndexDescriptor index : indexes) {
+        List<LogicalExpression> leadingColumns = new ArrayList<>();
+        if(initCondition.isAlwaysTrue()) {
+          break;
+        }
+        //TODO: Ensure we dont get NULL pointer exceptions
+        leadingColumns.add(index.getIndexColumns().get(0));
+        IndexConditionInfo info = indexConditionRelatedToFields(leadingColumns, initCondition);
+        if(info == null || info.hasIndexCol == false) {
+          // No info found, based on remaining condition. Check if the leading columns are same as another index
+          IndexConditionInfo origInfo = indexConditionRelatedToFields(leadingColumns, condition);
+          if (origInfo == null || origInfo.hasIndexCol == false) {
+            // do nothing
+          } else {
+            indexInfoMap.put(index, origInfo);
+            // Leave the initCondition as-is, since this is a duplicate condition
+          }
+          continue;
+        }
+        indexInfoMap.put(index, info);
+        initCondition = info.remainderCondition;
+      }
+      return indexInfoMap;
+    }
+
+    public boolean isConditionPrefix(IndexDescriptor indexDesc, RexNode initCondition) {
+      List<LogicalExpression> indexCols = indexDesc.getIndexColumns();
+      boolean prefix = true;
+      int numPrefix = 0;
+      if (indexCols.size() > 0 && initCondition != null) {
+        int i=0;
+        while (prefix && i < indexCols.size()) {
+          LogicalExpression p = indexCols.get(i++);
+          List<LogicalExpression> prefixCol = ImmutableList.of(p);
+          IndexConditionInfo info = indexConditionRelatedToFields(prefixCol, initCondition);
+          if(info != null && info.hasIndexCol) {
+            numPrefix++;
+            initCondition = info.remainderCondition;
+            if (initCondition.isAlwaysTrue()) {
+              // all filter conditions are accounted for
+              break;
+            }
+          } else {
+            prefix = false;
+          }
+        }
+      }
+      return numPrefix > 0;
+    }
+
+    /**
+     * Get a map of Index=>IndexConditionInfo, each IndexConditionInfo has the separated condition and remainder condition.
+     * The map is ordered, so the last IndexDescriptor will have the final remainderCondition after separating conditions
+     * that are relevant to the indexList. The conditions are separated based on index columns.
+     * @return Map containing index{@link IndexDescriptor} and condition {@link IndexConditionInfo} pairs
+     */
+    public Map<IndexDescriptor, IndexConditionInfo> getIndexConditionMap(List<IndexDescriptor> indexList) {
+      return getIndexConditionMapInternal(indexList);
+    }
+
+    /**
+     * Get a map of Index=>IndexConditionInfo, each IndexConditionInfo has the separated condition and remainder condition.
+     * The map is ordered, so the last IndexDescriptor will have the final remainderCondition after separating conditions
+     * that are relevant to this.indexes. The conditions are separated based on index columns.
+     * @return Map containing index{@link IndexDescriptor} and condition {@link IndexConditionInfo} pairs
+     */
+    public Map<IndexDescriptor, IndexConditionInfo> getIndexConditionMap() {
+      return getIndexConditionMapInternal(Lists.newArrayList(indexes));
+    }
+
+    private Map<IndexDescriptor, IndexConditionInfo> getIndexConditionMapInternal(List<IndexDescriptor> indexes) {
+
+      Map<IndexDescriptor, IndexConditionInfo> indexInfoMap = Maps.newLinkedHashMap();
+      RexNode initCondition = condition;
+      for (IndexDescriptor index : indexes) {
+        if(initCondition.isAlwaysTrue()) {
+          break;
+        }
+        if(!isConditionPrefix(index, initCondition)) {
+          continue;
+        }
+        IndexConditionInfo info = indexConditionRelatedToFields(index.getIndexColumns(), initCondition);
+        if(info == null || info.hasIndexCol == false) {
+          continue;
+        }
+        initCondition = info.remainderCondition;
+        indexInfoMap.put(index, info);
+      }
+      return indexInfoMap;
+    }
+
+    /**
+     * Given a list of Index Expressions(usually indexed fields/functions from one or a set of indexes),
+     * separate a filter condition into
+     *     1), relevant subset of conditions (by relevant, it means at least one given index Expression was found) and,
+     *     2), the rest in remainderCondition
+     * @param relevantPaths
+     * @param condition
+     * @return
+     */
+    public IndexConditionInfo indexConditionRelatedToFields(List<LogicalExpression> relevantPaths, RexNode condition) {
+      // Use the same filter analyzer that is used for partitioning columns
+      RewriteCombineBinaryOperators reverseVisitor =
+          new RewriteCombineBinaryOperators(true, builder);
+
+      condition = condition.accept(reverseVisitor);
+
+      RexSeparator separator = new RexSeparator(relevantPaths, scan, builder);
+      RexNode indexCondition = separator.getSeparatedCondition(condition);
+
+      if (indexCondition == null) {
+        return new IndexConditionInfo(null, null, false);
+      }
+
+      List<RexNode> conjuncts = RelOptUtil.conjunctions(condition);
+      List<RexNode> indexConjuncts = RelOptUtil.conjunctions(indexCondition);
+      for(RexNode indexConjunction: indexConjuncts) {
+        RexUtil.removeAll(conjuncts, indexConjunction);
+      }
+
+      RexNode remainderCondition = RexUtil.composeConjunction(builder, conjuncts, false);
+
+      indexCondition = indexCondition.accept(reverseVisitor);
+
+      return new IndexConditionInfo(indexCondition, remainderCondition, true);
+    }
+
+  }
+}
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/index/IndexDiscoverFactory.java b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/index/IndexDiscoverFactory.java
new file mode 100644
index 0000000..74e284a
--- /dev/null
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/index/IndexDiscoverFactory.java
@@ -0,0 +1,75 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.drill.exec.planner.index;
+
+import org.apache.drill.common.logical.StoragePluginConfig;
+import org.apache.drill.exec.physical.base.GroupScan;
+import org.apache.drill.exec.planner.common.DrillScanRelBase;
+import org.apache.drill.exec.planner.physical.ScanPrel;
+import org.apache.calcite.rel.RelNode;
+
+import java.lang.reflect.Constructor;
+
+/**
+ * With this factory, we allow user to load a different indexDiscover class to obtain index information
+ */
+public class IndexDiscoverFactory {
+  static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(IndexDiscoverFactory.class);
+  static final String INDEX_DISCOVER_CLASS_KEY = "index.discoverClass";
+  static final String INDEX_DISCOVER_CONFIG_KEY = "index.meta";
+
+  public static IndexDiscover getIndexDiscover(StoragePluginConfig config,
+                                               GroupScan inScan, RelNode scanRel,
+                                               Class<? extends IndexDiscover> targetIndexDiscoverClass) {
+    Class discoverClass = targetIndexDiscoverClass;
+
+    try {
+      if (config != null ) {
+        String discoverClassName = config.getValue(INDEX_DISCOVER_CLASS_KEY);
+        if(discoverClassName!= null && discoverClassName != "") {
+          discoverClass = Class.forName(discoverClassName);
+        }
+      }
+    } catch(Exception e) {
+      logger.error("Could not find configured IndexDiscover class {}", e);
+    }
+    Constructor<? extends IndexDiscoverBase> constructor;
+    try {
+      constructor = getConstructor(discoverClass,
+          scanRel);
+      IndexDiscoverBase idxDiscover = constructor.newInstance(inScan, scanRel);
+      if((targetIndexDiscoverClass != null) && (discoverClass != targetIndexDiscoverClass)) {
+
+        //idxDiscover.setOriginalDiscover(targetIndexDiscoverClass);
+      }
+      return idxDiscover;
+    } catch(Exception e) {
+      logger.error("Could not construct {}", discoverClass.getName(), e);
+    }
+    return null;
+  }
+
+  private static Constructor<? extends IndexDiscoverBase> getConstructor(Class discoverClass,RelNode scanRel) throws Exception {
+    if (scanRel instanceof DrillScanRelBase) {
+      return discoverClass.getConstructor(GroupScan.class, DrillScanRelBase.class);
+    } else {
+      return discoverClass.getConstructor(GroupScan.class, ScanPrel.class);
+    }
+  }
+
+}
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/index/IndexLogicalPlanCallContext.java b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/index/IndexLogicalPlanCallContext.java
new file mode 100644
index 0000000..27198bb
--- /dev/null
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/index/IndexLogicalPlanCallContext.java
@@ -0,0 +1,178 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.drill.exec.planner.index;
+
+import org.apache.drill.shaded.guava.com.google.common.collect.Lists;
+import org.apache.calcite.plan.RelOptRuleCall;
+import org.apache.calcite.rel.RelCollation;
+import org.apache.calcite.rel.core.Sort;
+import org.apache.calcite.rex.RexNode;
+import org.apache.drill.common.expression.LogicalExpression;
+import org.apache.drill.common.expression.SchemaPath;
+import org.apache.drill.exec.physical.base.DbGroupScan;
+import org.apache.drill.exec.planner.common.DrillProjectRelBase;
+import org.apache.drill.exec.planner.logical.DrillFilterRel;
+import org.apache.drill.exec.planner.logical.DrillProjectRel;
+import org.apache.drill.exec.planner.logical.DrillScanRel;
+import org.apache.drill.exec.planner.logical.DrillSortRel;
+import org.apache.drill.exec.planner.common.DrillScanRelBase;
+import org.apache.drill.exec.planner.physical.DrillDistributionTrait.DistributionField;
+import org.apache.calcite.rel.RelNode;
+
+import java.util.Collections;
+import java.util.List;
+import java.util.Set;
+
+public class IndexLogicalPlanCallContext implements IndexCallContext {
+  final public RelOptRuleCall call;
+  final public DrillSortRel sort;
+  final public DrillProjectRel upperProject;
+  final public DrillFilterRel filter;
+  final public DrillProjectRel lowerProject;
+  final public DrillScanRel scan;
+  final public String indexHint;
+
+  public Set<LogicalExpression> leftOutPathsInFunctions;
+
+  public IndexableExprMarker origMarker;
+  public List<LogicalExpression> sortExprs;
+
+  public RexNode origPushedCondition;
+
+  public IndexLogicalPlanCallContext(RelOptRuleCall call,
+                       DrillProjectRel capProject,
+                       DrillFilterRel filter,
+                       DrillProjectRel project,
+                       DrillScanRel scan) {
+    this(call, null, capProject, filter, project, scan);
+  }
+
+  public IndexLogicalPlanCallContext(RelOptRuleCall call,
+      DrillSortRel sort,
+      DrillProjectRel capProject,
+      DrillFilterRel filter,
+      DrillProjectRel project,
+      DrillScanRel scan) {
+    this.call = call;
+    this.sort = sort;
+    this.upperProject = capProject;
+    this.filter = filter;
+    this.lowerProject = project;
+    this.scan = scan;
+    this.indexHint = ((DbGroupScan)this.scan.getGroupScan()).getIndexHint();
+  }
+
+  public IndexLogicalPlanCallContext(RelOptRuleCall call,
+                       DrillSortRel sort,
+                       DrillProjectRel project,
+                       DrillScanRel scan) {
+    this.call = call;
+    this.sort = sort;
+    this.upperProject = null;
+    this.filter = null;
+    this.lowerProject = project;
+    this.scan = scan;
+    this.indexHint = ((DbGroupScan)this.scan.getGroupScan()).getIndexHint();
+  }
+
+  public DbGroupScan getGroupScan() {
+    return (DbGroupScan) scan.getGroupScan();
+  }
+
+  public DrillScanRelBase getScan() {
+    return scan;
+  }
+
+  public List<RelCollation> getCollationList() {
+    if (sort != null) {
+      return sort.getCollationList();
+    }
+    return null;
+  }
+
+  public RelCollation getCollation() {
+    if (sort != null) {
+      return sort.getCollation();
+    }
+    return null;
+  }
+
+  public boolean hasLowerProject() {
+    return lowerProject != null;
+  }
+
+  public boolean hasUpperProject() {
+    return upperProject != null;
+  }
+
+  public RelOptRuleCall getCall() {
+    return call;
+  }
+
+  public Set<LogicalExpression> getLeftOutPathsInFunctions() {
+    return leftOutPathsInFunctions;
+  }
+
+  public RelNode getFilter() {
+    return filter;
+  }
+
+  public IndexableExprMarker getOrigMarker() {
+    return origMarker;
+  }
+
+  public List<LogicalExpression> getSortExprs() {
+    return sortExprs;
+  }
+
+  public DrillProjectRelBase getLowerProject() {
+    return lowerProject;
+  }
+
+  public DrillProjectRelBase getUpperProject() {
+    return upperProject;
+  }
+
+  public void setLeftOutPathsInFunctions(Set<LogicalExpression> exprs) {
+    leftOutPathsInFunctions = exprs;
+  }
+
+  public List<SchemaPath> getScanColumns() {
+    return scan.getColumns();
+  }
+
+  public RexNode getFilterCondition() {
+    return filter.getCondition();
+  }
+
+  public RexNode getOrigCondition() {
+    return origPushedCondition;
+  }
+
+  public Sort getSort() {
+    return sort;
+  }
+
+  public void createSortExprs() {
+    sortExprs = Lists.newArrayList();
+  }
+
+  public RelNode getExchange() { return null; }
+
+  public List<DistributionField> getDistributionFields() { return Collections.EMPTY_LIST; }
+}
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/index/IndexPhysicalPlanCallContext.java b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/index/IndexPhysicalPlanCallContext.java
new file mode 100644
index 0000000..9c7b651
--- /dev/null
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/index/IndexPhysicalPlanCallContext.java
@@ -0,0 +1,193 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.drill.exec.planner.index;
+
+import org.apache.drill.shaded.guava.com.google.common.collect.Lists;
+import org.apache.calcite.plan.RelOptRuleCall;
+import org.apache.calcite.rel.RelCollation;
+import org.apache.calcite.rel.RelNode;
+import org.apache.calcite.rel.core.Sort;
+import org.apache.calcite.rex.RexNode;
+import org.apache.drill.common.expression.LogicalExpression;
+import org.apache.drill.common.expression.SchemaPath;
+import org.apache.drill.exec.physical.base.AbstractDbGroupScan;
+import org.apache.drill.exec.physical.base.DbGroupScan;
+import org.apache.drill.exec.planner.common.DrillProjectRelBase;
+import org.apache.drill.exec.planner.common.DrillScanRelBase;
+import org.apache.drill.exec.planner.physical.SortPrel;
+import org.apache.drill.exec.planner.physical.ProjectPrel;
+import org.apache.drill.exec.planner.physical.FilterPrel;
+import org.apache.drill.exec.planner.physical.ScanPrel;
+import org.apache.drill.exec.planner.physical.ExchangePrel;
+import org.apache.drill.exec.planner.physical.HashToRandomExchangePrel;
+import org.apache.drill.exec.planner.physical.DrillDistributionTrait.DistributionField;
+
+import java.util.Collections;
+import java.util.List;
+import java.util.Set;
+
+public class IndexPhysicalPlanCallContext implements IndexCallContext {
+  final public RelOptRuleCall call;
+  final public SortPrel sort;
+  final public ProjectPrel upperProject;
+  final public FilterPrel filter;
+  final public ProjectPrel lowerProject;
+  final public ScanPrel scan;
+  final public ExchangePrel exch;
+  final public String indexHint;
+
+  public Set<LogicalExpression> leftOutPathsInFunctions;
+
+  public IndexableExprMarker origMarker;
+  public List<LogicalExpression> sortExprs;
+
+  public RexNode origPushedCondition;
+
+  public IndexPhysicalPlanCallContext(RelOptRuleCall call,
+                                      ProjectPrel capProject,
+                                      FilterPrel filter,
+                                      ProjectPrel project,
+                                      ScanPrel scan) {
+    this(call, null, capProject, filter, project, scan, null);
+  }
+
+  public IndexPhysicalPlanCallContext(RelOptRuleCall call,
+                                      SortPrel sort,
+                                      ProjectPrel capProject,
+                                      FilterPrel filter,
+                                      ProjectPrel project,
+                                      ScanPrel scan, ExchangePrel exch) {
+    this.call = call;
+    this.sort = sort;
+    this.upperProject = capProject;
+    this.filter = filter;
+    this.lowerProject = project;
+    this.scan = scan;
+    this.exch = exch;
+    this.indexHint = ((DbGroupScan)this.scan.getGroupScan()).getIndexHint();
+  }
+
+  public IndexPhysicalPlanCallContext(RelOptRuleCall call,
+                                      SortPrel sort,
+                                      ProjectPrel project,
+                                      ScanPrel scan, ExchangePrel exch) {
+    this.call = call;
+    this.sort = sort;
+    this.upperProject = null;
+    this.filter = null;
+    this.lowerProject = project;
+    this.scan = scan;
+    this.exch = exch;
+    this.indexHint = ((DbGroupScan)this.scan.getGroupScan()).getIndexHint();
+  }
+
+  public DbGroupScan getGroupScan() {
+    return (DbGroupScan) scan.getGroupScan();
+  }
+
+  public DrillScanRelBase getScan() {
+    return scan;
+  }
+
+
+  public List<RelCollation> getCollationList() {
+    if (sort != null) {
+      return sort.getCollationList();
+    }
+    return null;
+  }
+
+  public RelCollation getCollation() {
+    if (sort != null) {
+      return sort.getCollation();
+    }
+    return null;
+  }
+
+  public boolean hasLowerProject() {
+    return lowerProject != null;
+  }
+
+  public boolean hasUpperProject() {
+    return upperProject != null;
+  }
+
+  public RelOptRuleCall getCall() {
+    return call;
+  }
+
+  public Set<LogicalExpression> getLeftOutPathsInFunctions() {
+    return leftOutPathsInFunctions;
+  }
+
+  public RelNode getFilter() {
+    return filter;
+  }
+
+  public IndexableExprMarker getOrigMarker() {
+    return origMarker;
+  }
+
+  public List<LogicalExpression> getSortExprs() {
+    return sortExprs;
+  }
+
+  public DrillProjectRelBase getLowerProject() {
+    return lowerProject;
+  }
+
+  public DrillProjectRelBase getUpperProject() {
+    return upperProject;
+  }
+
+  public void setLeftOutPathsInFunctions(Set<LogicalExpression> exprs) {
+    leftOutPathsInFunctions = exprs;
+  }
+
+  public List<SchemaPath> getScanColumns() {
+    return ((AbstractDbGroupScan)scan.getGroupScan()).getColumns();
+  }
+
+  public RexNode getFilterCondition() {
+    return filter.getCondition();
+  }
+
+  public RexNode getOrigCondition() {
+    return origPushedCondition;
+  }
+
+  public Sort getSort() {
+    return sort;
+  }
+
+  public RelNode getExchange() {
+    return exch;
+  }
+
+  public void createSortExprs() {
+    sortExprs = Lists.newArrayList();
+  }
+
+  public List<DistributionField> getDistributionFields() {
+    if (exch != null) {
+      return ((HashToRandomExchangePrel) exch).getFields();
+    } else {
+      return Collections.EMPTY_LIST;
+    }
+  }
+}
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/index/IndexPlanUtils.java b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/index/IndexPlanUtils.java
new file mode 100644
index 0000000..666e282
--- /dev/null
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/index/IndexPlanUtils.java
@@ -0,0 +1,855 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.drill.exec.planner.index;
+
+import java.util.ArrayList;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+
+import org.apache.drill.shaded.guava.com.google.common.collect.ImmutableList;
+import org.apache.drill.shaded.guava.com.google.common.collect.Lists;
+import org.apache.drill.shaded.guava.com.google.common.collect.Maps;
+import org.apache.drill.shaded.guava.com.google.common.collect.Sets;
+
+import org.apache.calcite.plan.RelTraitSet;
+import org.apache.calcite.plan.volcano.RelSubset;
+import org.apache.calcite.rel.RelCollation;
+import org.apache.calcite.rel.RelCollationTraitDef;
+import org.apache.calcite.rel.RelCollations;
+import org.apache.calcite.rel.RelFieldCollation;
+import org.apache.calcite.rel.RelNode;
+import org.apache.calcite.rel.core.Sort;
+import org.apache.calcite.rex.RexBuilder;
+import org.apache.calcite.rex.RexUtil;
+import org.apache.calcite.sql.SqlKind;
+import org.apache.drill.common.expression.FieldReference;
+import org.apache.drill.common.expression.LogicalExpression;
+import org.apache.drill.common.expression.SchemaPath;
+import org.apache.drill.exec.physical.base.DbGroupScan;
+import org.apache.drill.exec.physical.base.GroupScan;
+import org.apache.drill.exec.physical.base.IndexGroupScan;
+import org.apache.drill.exec.planner.common.DrillProjectRelBase;
+import org.apache.drill.exec.planner.common.DrillScanRelBase;
+import org.apache.drill.exec.planner.fragment.DistributionAffinity;
+import org.apache.drill.exec.planner.logical.DrillOptiq;
+import org.apache.drill.exec.planner.logical.DrillParseContext;
+import org.apache.drill.exec.planner.logical.DrillScanRel;
+import org.apache.drill.exec.planner.physical.DrillDistributionTrait;
+import org.apache.drill.exec.planner.physical.Prel;
+import org.apache.drill.exec.planner.physical.PrelUtil;
+import org.apache.drill.exec.planner.physical.ScanPrel;
+import org.apache.drill.exec.planner.physical.ProjectPrel;
+import org.apache.calcite.rel.type.RelDataType;
+import org.apache.calcite.rel.type.RelDataTypeField;
+import org.apache.calcite.rex.RexInputRef;
+import org.apache.calcite.rex.RexNode;
+
+public class IndexPlanUtils {
+
+  public enum ConditionIndexed {
+    NONE,
+    PARTIAL,
+    FULL}
+
+  /**
+   * Check if any of the fields of the index are present in a list of LogicalExpressions supplied
+   * as part of IndexableExprMarker
+   * @param exprMarker, the marker that has analyzed original index condition on top of original scan
+   * @param indexDesc
+   * @return ConditionIndexed.FULL, PARTIAL or NONE depending on whether all, some or no columns
+   * of the indexDesc are present in the list of LogicalExpressions supplied as part of exprMarker
+   *
+   */
+  static public ConditionIndexed conditionIndexed(IndexableExprMarker exprMarker, IndexDescriptor indexDesc) {
+    Map<RexNode, LogicalExpression> mapRexExpr = exprMarker.getIndexableExpression();
+    List<LogicalExpression> infoCols = Lists.newArrayList();
+    infoCols.addAll(mapRexExpr.values());
+    if (indexDesc.allColumnsIndexed(infoCols)) {
+      return ConditionIndexed.FULL;
+    } else if (indexDesc.someColumnsIndexed(infoCols)) {
+      return ConditionIndexed.PARTIAL;
+    } else {
+      return ConditionIndexed.NONE;
+    }
+  }
+
+  /**
+   * check if we want to apply index rules on this scan,
+   * if group scan is not instance of DbGroupScan, or this DbGroupScan instance does not support secondary index, or
+   *    this scan is already an index scan or Restricted Scan, do not apply index plan rules on it.
+   * @param scanRel
+   * @return
+   */
+  static public boolean checkScan(DrillScanRel scanRel) {
+    GroupScan groupScan = scanRel.getGroupScan();
+    if (groupScan instanceof DbGroupScan) {
+      DbGroupScan dbscan = ((DbGroupScan) groupScan);
+      //if we already applied index convert rule, and this scan is indexScan or restricted scan already,
+      //no more trying index convert rule
+      return dbscan.supportsSecondaryIndex() && (!dbscan.isIndexScan()) && (!dbscan.isRestrictedScan());
+    }
+    return false;
+  }
+
+  /**
+   * For a particular table scan for table T1 and an index on that table, find out if it is a covering index
+   * @return
+   */
+  static public boolean isCoveringIndex(IndexCallContext indexContext, FunctionalIndexInfo functionInfo) {
+    if(functionInfo.hasFunctional()) {
+      //need info from full query
+      return queryCoveredByIndex(indexContext, functionInfo);
+    }
+    DbGroupScan groupScan = (DbGroupScan) getGroupScan(indexContext.getScan());
+    List<LogicalExpression> tableCols = Lists.newArrayList();
+    tableCols.addAll(groupScan.getColumns());
+    return functionInfo.getIndexDesc().isCoveringIndex(tableCols);
+  }
+
+
+  /**
+   * This method is called only when the index has at least one functional indexed field. If there is no function field,
+   * we don't need to worry whether there could be paths not found in Scan.
+   * In functional case, we have to check all available (if needed) operators to find out if the query is covered or not.
+   * E.g. cast(a.b as INT) in project, a.b in Scan's rowType or columns, and cast(a.b as INT)
+   * is an indexed field named '$0'. In this case, by looking at Scan, we see only 'a.b' which is not in index. We have to
+   * look into Project, and if we see 'a.b' is only used in functional index expression cast(a.b as INT), then we know
+   * this Project+Scan is covered.
+   * @param indexContext
+   * @param functionInfo
+   * @return false if the query could not be covered by the index (should not create covering index plan)
+   */
+  static private boolean queryCoveredByIndex(IndexCallContext indexContext,
+                              FunctionalIndexInfo functionInfo) {
+    //for indexed functions, if relevant schemapaths are included in index(in indexed fields or non-indexed fields),
+    // check covering based on the local information we have:
+    //   if references to schema paths in functional indexes disappear beyond capProject
+
+    if (indexContext.getFilter() != null && indexContext.getUpperProject() == null) {
+      if( !isFullQuery(indexContext)) {
+        return false;
+      }
+    }
+
+    DrillParseContext parserContext =
+        new DrillParseContext(PrelUtil.getPlannerSettings(indexContext.getCall().rel(0).getCluster()));
+
+    Set<LogicalExpression> exprs = Sets.newHashSet();
+    if (indexContext.getUpperProject() != null) {
+      if (indexContext.getLowerProject() == null) {
+        for (RexNode rex : indexContext.getUpperProject().getProjects()) {
+          LogicalExpression expr = RexToExpression.toDrill(parserContext, null, indexContext.getScan(), rex);
+          exprs.add(expr);
+        }
+        //now collect paths in filter since upperProject may drop some paths in filter
+        IndexableExprMarker filterMarker = new IndexableExprMarker(indexContext.getScan());
+        indexContext.getFilterCondition().accept(filterMarker);
+        for (RexNode rex : filterMarker.getIndexableExpression().keySet()) {
+          LogicalExpression expr = RexToExpression.toDrill(parserContext, null, indexContext.getScan(), rex);
+          exprs.add(expr);
+        }
+      } else {
+        //we have underneath project, so we have to do more to convert expressions
+        for (RexNode rex : indexContext.getUpperProject().getProjects()) {
+          LogicalExpression expr = RexToExpression.toDrill(parserContext, indexContext.getLowerProject(), indexContext.getScan(), rex);
+          exprs.add(expr);
+        }
+
+        // Now collect paths in filter since upperProject may drop some paths in filter.
+        // Since this is (upper)Proj+Filter+(lower)Proj+Scan case, and IndexableExprMarker works
+        // only with expressions that referencing directly to Scan, it has to use indexContext.origPushedCondition
+        IndexableExprMarker filterMarker = new IndexableExprMarker(indexContext.getScan());
+        indexContext.getOrigCondition().accept(filterMarker);
+
+        for (RexNode rex : filterMarker.getIndexableExpression().keySet()) {
+          // Since rex represents the filter expression directly referencing the scan row type,
+          // (the condition has been pushed down of lowerProject), set the lowerProject as null.
+          LogicalExpression expr = RexToExpression.toDrill(parserContext, null, indexContext.getScan(), rex);
+          exprs.add(expr);
+        }
+      }
+    }
+    else if (indexContext.getLowerProject() != null) {
+      for (RexNode rex : indexContext.getLowerProject().getProjects()) {
+        LogicalExpression expr = DrillOptiq.toDrill(parserContext, indexContext.getScan(), rex);
+        exprs.add(expr);
+      }
+    }
+    else {//upperProject and lowerProject both are null, the only place to find columns being used in query is scan
+      exprs.addAll(indexContext.getScanColumns());
+    }
+
+    Map<LogicalExpression, Set<SchemaPath>> exprPathMap = functionInfo.getPathsInFunctionExpr();
+    PathInExpr exprSearch = new PathInExpr(exprPathMap);
+
+    for(LogicalExpression expr: exprs) {
+      if(expr.accept(exprSearch, null) == false) {
+        return false;
+      }
+    }
+    //if we come to here, paths in indexed function expressions are covered in capProject.
+    //now we check other paths.
+
+    //check the leftout paths (appear in capProject other than functional index expression) are covered by other index fields or not
+    List<LogicalExpression> leftPaths = Lists.newArrayList(exprSearch.getRemainderPaths());
+
+    indexContext.setLeftOutPathsInFunctions(exprSearch.getRemainderPathsInFunctions());
+    return functionInfo.getIndexDesc().isCoveringIndex(leftPaths);
+  }
+
+  static private boolean isFullQuery(IndexCallContext indexContext) {
+    RelNode rootInCall = indexContext.getCall().rel(0);
+    //check if the tip of the operator stack we have is also the top of the whole query, if yes, return true
+    if (indexContext.getCall().getPlanner().getRoot() instanceof RelSubset) {
+      final RelSubset rootSet = (RelSubset) indexContext.getCall().getPlanner().getRoot();
+      if (rootSet.getRelList().contains(rootInCall)) {
+        return true;
+      }
+    } else {
+      if (indexContext.getCall().getPlanner().getRoot().equals(rootInCall)) {
+        return true;
+      }
+    }
+
+    return false;
+  }
+
+  /**
+   * Build collation property for the 'lower' project, the one closer to the Scan
+   * @param projectRexs
+   * @param input
+   * @param indexInfo
+   * @return the output RelCollation
+   */
+  public static RelCollation buildCollationLowerProject(List<RexNode> projectRexs, RelNode input, FunctionalIndexInfo indexInfo) {
+    //if leading fields of index are here, add them to RelCollation
+    List<RelFieldCollation> newFields = Lists.newArrayList();
+    if (!indexInfo.hasFunctional()) {
+      Map<LogicalExpression, Integer> projectExprs = Maps.newLinkedHashMap();
+      DrillParseContext parserContext = new DrillParseContext(PrelUtil.getPlannerSettings(input.getCluster()));
+      int idx=0;
+      for(RexNode rex : projectRexs) {
+        projectExprs.put(DrillOptiq.toDrill(parserContext, input, rex), idx);
+        idx++;
+      }
+      int idxFieldCount = 0;
+      for (LogicalExpression expr : indexInfo.getIndexDesc().getIndexColumns()) {
+        if (!projectExprs.containsKey(expr)) {
+          break;
+        }
+        RelFieldCollation.Direction dir = indexInfo.getIndexDesc().getCollation().getFieldCollations().get(idxFieldCount).direction;
+        if ( dir == null) {
+          break;
+        }
+        newFields.add(new RelFieldCollation(projectExprs.get(expr), dir,
+            RelFieldCollation.NullDirection.UNSPECIFIED));
+      }
+      idxFieldCount++;
+    } else {
+      // TODO: handle functional index
+    }
+
+    return RelCollations.of(newFields);
+  }
+
+  /**
+   * Build collation property for the 'upper' project, the one above the filter
+   * @param projectRexs
+   * @param inputCollation
+   * @param indexInfo
+   * @param collationFilterMap
+   * @return the output RelCollation
+   */
+  public static RelCollation buildCollationUpperProject(List<RexNode> projectRexs,
+                                                        RelCollation inputCollation, FunctionalIndexInfo indexInfo,
+                                                        Map<Integer, List<RexNode>> collationFilterMap) {
+    List<RelFieldCollation> outputFieldCollations = Lists.newArrayList();
+
+    if (inputCollation != null) {
+      List<RelFieldCollation> inputFieldCollations = inputCollation.getFieldCollations();
+      if (!indexInfo.hasFunctional()) {
+        for (int projectExprIdx = 0; projectExprIdx < projectRexs.size(); projectExprIdx++) {
+          RexNode n = projectRexs.get(projectExprIdx);
+          if (n instanceof RexInputRef) {
+            RexInputRef ref = (RexInputRef)n;
+            boolean eligibleForCollation = true;
+            int maxIndex = getIndexFromCollation(ref.getIndex(), inputFieldCollations);
+            if (maxIndex < 0) {
+              eligibleForCollation = false;
+              continue;
+            }
+            // check if the prefix has equality conditions
+            for (int i = 0; i < maxIndex; i++) {
+              int fieldIdx = inputFieldCollations.get(i).getFieldIndex();
+              List<RexNode> conditions = collationFilterMap != null ? collationFilterMap.get(fieldIdx) : null;
+              if ((conditions == null || conditions.size() == 0) &&
+                  i < maxIndex-1) {
+                // if an intermediate column has no filter condition, it would select all values
+                // of that column, so a subsequent column cannot be eligible for collation
+                eligibleForCollation = false;
+                break;
+              } else {
+                for (RexNode r : conditions) {
+                  if (!(r.getKind() == SqlKind.EQUALS)) {
+                    eligibleForCollation = false;
+                    break;
+                  }
+                }
+              }
+            }
+            // for every projected expr, if it is eligible for collation, get the
+            // corresponding field collation from the input
+            if (eligibleForCollation) {
+              for (RelFieldCollation c : inputFieldCollations) {
+                if (ref.getIndex() == c.getFieldIndex()) {
+                  RelFieldCollation outFieldCollation = new RelFieldCollation(projectExprIdx, c.getDirection(), c.nullDirection);
+                  outputFieldCollations.add(outFieldCollation);
+                }
+              }
+            }
+          }
+        }
+      } else {
+        // TODO: handle functional index
+      }
+    }
+    return RelCollations.of(outputFieldCollations);
+  }
+
+  public static int getIndexFromCollation(int refIndex, List<RelFieldCollation> inputFieldCollations) {
+    for (int i=0; i < inputFieldCollations.size(); i++) {
+      if (refIndex == inputFieldCollations.get(i).getFieldIndex()) {
+        return i;
+      }
+    }
+    return -1;
+  }
+
+  public static List<RexNode> getProjects(DrillProjectRelBase proj) {
+    return proj.getProjects();
+  }
+
+  /**
+   * generate logical expressions for sort rexNodes in SortRel, the result is store to IndexPlanCallContext
+   * @param indexContext
+   */
+  public static void updateSortExpression(IndexCallContext indexContext, List<RelFieldCollation> coll) {
+
+    if (coll == null) {
+      return;
+    }
+
+    DrillParseContext parserContext =
+        new DrillParseContext(PrelUtil.getPlannerSettings(indexContext.getCall().rel(0).getCluster()));
+
+    indexContext.createSortExprs();
+    for (RelFieldCollation collation : coll) {
+      int idx = collation.getFieldIndex();
+      DrillProjectRelBase oneProject;
+      if (indexContext.getUpperProject() != null && indexContext.getLowerProject() != null) {
+        LogicalExpression expr = RexToExpression.toDrill(parserContext, indexContext.getLowerProject(), indexContext.getScan(),
+            indexContext.getUpperProject().getProjects().get(idx));
+        indexContext.getSortExprs().add(expr);
+      }
+      else {//one project is null now
+        oneProject = (indexContext.getUpperProject() != null)? indexContext.getUpperProject() : indexContext.getLowerProject();
+        if(oneProject != null) {
+          LogicalExpression expr = RexToExpression.toDrill(parserContext, null, indexContext.getScan(),
+              getProjects(oneProject).get(idx));
+          indexContext.getSortExprs().add(expr);
+        }
+        else {//two projects are null
+          SchemaPath path;
+          RelDataTypeField f = indexContext.getScan().getRowType().getFieldList().get(idx);
+          String pathSeg = f.getName().replaceAll("`", "");
+          final String[] segs = pathSeg.split("\\.");
+          path = SchemaPath.getCompoundPath(segs);
+          indexContext.getSortExprs().add(path);
+        }
+      }
+    }
+  }
+
+  /**
+   * generate logical expressions for sort rexNodes in SortRel, the result is store to IndexPlanCallContext
+   * @param indexContext
+   */
+  public static void updateSortExpression(IndexPhysicalPlanCallContext indexContext, List<RelFieldCollation> coll) {
+
+    if (coll == null) {
+      return;
+    }
+
+    DrillParseContext parserContext =
+            new DrillParseContext(PrelUtil.getPlannerSettings(indexContext.call.rel(0).getCluster()));
+
+    indexContext.sortExprs = Lists.newArrayList();
+    for (RelFieldCollation collation : coll) {
+      int idx = collation.getFieldIndex();
+      ProjectPrel oneProject;
+      if (indexContext.upperProject != null && indexContext.lowerProject != null) {
+        LogicalExpression expr = RexToExpression.toDrill(parserContext, indexContext.lowerProject, indexContext.scan,
+                indexContext.upperProject.getProjects().get(idx));
+        indexContext.sortExprs.add(expr);
+      }
+      else {//one project is null now
+        oneProject = (indexContext.upperProject != null)? indexContext.upperProject : indexContext.lowerProject;
+        if(oneProject != null) {
+          LogicalExpression expr = RexToExpression.toDrill(parserContext, null, indexContext.scan,
+                  oneProject.getProjects().get(idx));
+          indexContext.sortExprs.add(expr);
+        }
+        else {//two projects are null
+          SchemaPath path;
+          RelDataTypeField f = indexContext.scan.getRowType().getFieldList().get(idx);
+          String pathSeg = f.getName().replaceAll("`", "");
+          final String[] segs = pathSeg.split("\\.");
+          path = SchemaPath.getCompoundPath(segs);
+          indexContext.sortExprs.add(path);
+        }
+      }
+    }
+  }
+
+  /**
+   *
+   * @param expr
+   * @param context
+   * @return if there is filter and expr is only in equality condition of the filter, return true
+   */
+  private static boolean exprOnlyInEquality(LogicalExpression expr, IndexCallContext context) {
+    //if there is no filter, expr wont be in equality
+    if(context.getFilter() == null) {
+      return false;
+    }
+    final Set<LogicalExpression> onlyInEquality = context.getOrigMarker().getExpressionsOnlyInEquality();
+    return onlyInEquality.contains(expr);
+
+  }
+  /**
+   * Build collation property for project, the one closer to the Scan
+   * @param projectRexs the expressions to project
+   * @param project the project between projectRexs and input, it could be null if no such intermediate project(lower project)
+   * @param input  the input RelNode to the project, usually it is the scan operator.
+   * @param indexInfo the index for which we are building index plan
+   * @param context the context of this index planning process
+   * @return the output RelCollation
+   */
+  public static RelCollation buildCollationProject(List<RexNode> projectRexs,
+                                                   DrillProjectRelBase project,
+                                                   RelNode input,
+                                                   FunctionalIndexInfo indexInfo,
+                                                   IndexCallContext context) {
+    Map<LogicalExpression, Integer> projectExprs = getProjectExprs(projectRexs, project, input);
+    return buildCollationForExpressions(projectExprs, indexInfo.getIndexDesc(), context);
+  }
+
+  /**
+   * Build the collation property for index scan
+   * @param indexDesc the index for which we are building index plan
+   * @param context the context of this index planning process
+   * @return the output RelCollation for the scan on index
+   */
+  public static RelCollation buildCollationCoveringIndexScan(IndexDescriptor indexDesc,
+      IndexCallContext context) {
+    Map<LogicalExpression, Integer> rowTypeExprs = getExprsFromRowType(context.getScan().getRowType());
+    return buildCollationForExpressions(rowTypeExprs, indexDesc, context);
+  }
+
+  public static Map<LogicalExpression, Integer> getProjectExprs(List<RexNode> projectRexs,
+                                                                DrillProjectRelBase project,
+                                                                RelNode input) {
+    Map<LogicalExpression, Integer> projectExprs = Maps.newLinkedHashMap();
+    DrillParseContext parserContext = new DrillParseContext(PrelUtil.getPlannerSettings(input.getCluster()));
+    int idx=0;
+    for(RexNode rex : projectRexs) {
+      LogicalExpression expr;
+      expr = RexToExpression.toDrill(parserContext, project, input, rex);
+      projectExprs.put(expr, idx);
+      idx++;
+    }
+    return projectExprs;
+  }
+
+  public static Map<LogicalExpression, Integer> getExprsFromRowType( RelDataType indexScanRowType) {
+
+    Map<LogicalExpression, Integer> rowTypeExprs = Maps.newLinkedHashMap();
+    int idx = 0;
+    for (RelDataTypeField field : indexScanRowType.getFieldList()) {
+      rowTypeExprs.put(FieldReference.getWithQuotedRef(field.getName()), idx++);
+    }
+    return rowTypeExprs;
+  }
+
+  /**
+   * Given index, compute the collations for a list of projected expressions(from Scan's rowType or Project's )
+   * in the context
+   * @param projectExprs the output expression list of a RelNode
+   * @param indexDesc  the index for which we are building index plan
+   * @param context  the context of this index planning process
+   * @return the collation provided by index that will be exposed by the expression list
+   */
+  public static RelCollation buildCollationForExpressions(Map<LogicalExpression, Integer> projectExprs,
+                                                        IndexDescriptor indexDesc,
+                                                        IndexCallContext context) {
+
+    assert projectExprs != null;
+
+    final List<LogicalExpression> sortExpressions = context.getSortExprs();
+    // if leading fields of index are here, add them to RelCollation
+    List<RelFieldCollation> newFields = Lists.newArrayList();
+    if (indexDesc.getCollation() == null) {
+      return RelCollations.of(newFields);
+    }
+
+    // go through indexed fields to build collation
+    // break out of the loop when found first indexed field [not projected && not _only_ in equality condition of filter]
+    // or the leading field is not projected
+    List<LogicalExpression> indexedCols = indexDesc.getIndexColumns();
+    for (int idxFieldCount=0; idxFieldCount<indexedCols.size(); ++idxFieldCount) {
+      LogicalExpression expr = indexedCols.get(idxFieldCount);
+
+      if (!projectExprs.containsKey(expr)) {
+        // leading indexed field is not projected
+        // but it is only-in-equality field, -- we continue to next indexed field, but we don't generate collation for this field
+        if(exprOnlyInEquality(expr, context)) {
+          continue;
+        }
+        // else no more collation is needed to be generated, since we now have one leading field which is not in equality condition
+        break;
+      }
+
+      // leading indexed field is projected,
+
+      // if this field is not in sort expression && only-in-equality, we don't need to generate collation for this field
+      // and we are okay to continue: generate collation for next indexed field.
+      if (sortExpressions != null &&
+          !sortExpressions.contains(expr) && exprOnlyInEquality(expr, context) ) {
+        continue;
+      }
+
+      RelCollation idxCollation = indexDesc.getCollation();
+      RelFieldCollation.NullDirection nullsDir = indexDesc.getNullsOrderingDirection();
+      RelFieldCollation.Direction dir = (idxCollation == null)?
+          null : idxCollation.getFieldCollations().get(idxFieldCount).direction;
+      if ( dir == null) {
+        break;
+      }
+      newFields.add(new RelFieldCollation(projectExprs.get(expr), dir, nullsDir));
+    }
+
+    return RelCollations.of(newFields);
+  }
+
+  // TODO: proper implementation
+  public static boolean pathOnlyInIndexedFunction(SchemaPath path) {
+    return true;
+  }
+
+  public static RelCollation buildCollationNonCoveringIndexScan(IndexDescriptor indexDesc,
+      RelDataType indexScanRowType,
+      RelDataType restrictedScanRowType, IndexCallContext context) {
+
+    if (context.getSortExprs() == null) {
+      return RelCollations.of(RelCollations.EMPTY.getFieldCollations());
+    }
+
+    final List<RelDataTypeField> indexFields = indexScanRowType.getFieldList();
+    final List<RelDataTypeField> rsFields = restrictedScanRowType.getFieldList();
+    final Map<LogicalExpression, RelFieldCollation> collationMap = indexDesc.getCollationMap();
+
+    assert collationMap != null : "Invalid collation map for index";
+
+    List<RelFieldCollation> fieldCollations = Lists.newArrayList();
+
+    Map<Integer, RelFieldCollation> rsScanCollationMap = Maps.newTreeMap();
+
+    // for each index field that is projected from the indexScan, find the corresponding
+    // field in the restricted scan's row type and keep track of the ordinal # in the
+    // restricted scan's row type.
+    for (int i = 0; i < indexScanRowType.getFieldCount(); i++) {
+      RelDataTypeField f1 = indexFields.get(i);
+      for (int j = 0; j < rsFields.size(); j++) {
+        RelDataTypeField f2 = rsFields.get(j);
+        if (f1.getName().equals(f2.getName())) {
+          FieldReference ref = FieldReference.getWithQuotedRef(f1.getName());
+          RelFieldCollation origCollation = collationMap.get(ref);
+          if (origCollation != null) {
+            RelFieldCollation fc = new RelFieldCollation(j,//origCollation.getFieldIndex(),
+                origCollation.direction, origCollation.nullDirection);
+            rsScanCollationMap.put(origCollation.getFieldIndex(), fc);
+          }
+        }
+      }
+    }
+
+    // should sort by the order of these fields in indexDesc
+    for (Map.Entry<Integer, RelFieldCollation> entry : rsScanCollationMap.entrySet()) {
+      RelFieldCollation fc = entry.getValue();
+      if (fc != null) {
+        fieldCollations.add(fc);
+      }
+    }
+
+    final RelCollation collation = RelCollations.of(fieldCollations);
+    return collation;
+  }
+
+  public static boolean scanIsPartition(GroupScan scan) {
+    return (scan.isDistributed() || scan.getDistributionAffinity() == DistributionAffinity.HARD);
+  }
+
+  public static GroupScan getGroupScan(DrillScanRelBase relNode) {
+    return relNode.getGroupScan();
+  }
+
+  public static RelCollation getCollation(Sort sort) {
+    return sort.getCollation();
+  }
+
+  public static List<DrillDistributionTrait.DistributionField> getDistributionField(Sort rel) {
+    List<DrillDistributionTrait.DistributionField> distFields = Lists.newArrayList();
+
+    for (RelFieldCollation relField : getCollation(rel).getFieldCollations()) {
+      DrillDistributionTrait.DistributionField field = new DrillDistributionTrait.DistributionField(relField.getFieldIndex());
+      distFields.add(field);
+    }
+    return distFields;
+  }
+
+  public static ScanPrel buildCoveringIndexScan(DrillScanRelBase origScan,
+      IndexGroupScan indexGroupScan,
+      IndexCallContext indexContext,
+      IndexDescriptor indexDesc) {
+
+    FunctionalIndexInfo functionInfo = indexDesc.getFunctionalInfo();
+    //to record the new (renamed)paths added
+    List<SchemaPath> rewrittenPaths = Lists.newArrayList();
+    DbGroupScan dbGroupScan = (DbGroupScan) getGroupScan(origScan);
+    indexGroupScan.setColumns(
+        rewriteFunctionColumn(dbGroupScan.getColumns(),
+            functionInfo, rewrittenPaths));
+
+    DrillDistributionTrait partition = scanIsPartition(getGroupScan(origScan))?
+        DrillDistributionTrait.RANDOM_DISTRIBUTED : DrillDistributionTrait.SINGLETON;
+    RelDataType newRowType = FunctionalIndexHelper.rewriteFunctionalRowType(origScan, indexContext, functionInfo, rewrittenPaths);
+
+    // add a default collation trait otherwise Calcite runs into a ClassCastException, which at first glance
+    // seems like a Calcite bug
+    RelTraitSet indexScanTraitSet = origScan.getTraitSet().plus(Prel.DRILL_PHYSICAL).
+        plus(RelCollationTraitDef.INSTANCE.getDefault()).plus(partition);
+
+    // Create the collation traits for index scan based on the index columns under the
+    // condition that the index actually has collation property (e.g hash indexes don't)
+    if (indexDesc.getCollation() != null) {
+      RelCollation collationTrait = buildCollationCoveringIndexScan(indexDesc, indexContext);
+      indexScanTraitSet = indexScanTraitSet.plus(collationTrait);
+    }
+
+    ScanPrel indexScanPrel = new ScanPrel(origScan.getCluster(),
+        indexScanTraitSet, indexGroupScan,
+        newRowType, origScan.getTable());
+
+    return indexScanPrel;
+  }
+
+  /**
+   * For IndexGroupScan, if a column is only appeared in the should-be-renamed function,
+   * this column is to-be-replaced column, we replace that column(schemaPath) from 'a.b'
+   * to '$1' in the list of SchemaPath.
+   * @param paths
+   * @param functionInfo functional index information that may impact rewrite
+   * @return
+   */
+  public static List<SchemaPath> rewriteFunctionColumn(List<SchemaPath> paths,
+                                                       FunctionalIndexInfo functionInfo,
+                                                       List<SchemaPath> addedPaths) {
+    if (!functionInfo.hasFunctional()) {
+      return paths;
+    }
+
+    List<SchemaPath> newPaths = Lists.newArrayList(paths);
+    for (int i=0; i<paths.size(); ++i) {
+      SchemaPath newPath = functionInfo.getNewPath(paths.get(i));
+      if(newPath == null) {
+        continue;
+      }
+
+      addedPaths.add(newPath);
+      // if this path only in indexed function, we are safe to replace it
+      if(pathOnlyInIndexedFunction(paths.get(i))) {
+        newPaths.set(i, newPath);
+      }
+      else {// we should not replace this column, instead we add a new "$N" field.
+        newPaths.add(newPath);
+      }
+    }
+    return newPaths;
+  }
+
+  /**
+   *A RexNode forest with three RexNodes for expressions "cast(a.q as int) * 2, b+c, concat(a.q, " world")"
+   * on Scan RowType('a', 'b', 'c') will be like this:
+   *
+   *          (0)Call:"*"                                       Call:"concat"
+   *           /         \                                    /           \
+   *    (1)Call:CAST     2            Call:"+"        (5)Call:ITEM     ' world'
+   *      /        \                   /     \          /   \
+   * (2)Call:ITEM  TYPE:INT       (3)$1    (4)$2       $0    'q'
+   *   /      \
+   *  $0     'q'
+   *
+   * So for above expressions, when visiting the RexNode trees using PathInExpr, we could mark indexed expressions in the trees,
+   * as shown in the diagram above are the node (1),
+   * then collect the schema paths in the indexed expression but found out of the indexed expression -- node (5),
+   * and other regular schema paths (3) (4)
+   *
+   * @param parseContext
+   * @param project
+   * @param scan
+   * @param toRewriteRex  the RexNode to be converted if it contain a functional index expression.
+   * @param newRowType
+   * @param functionInfo
+   * @return
+   */
+  public static RexNode rewriteFunctionalRex(IndexCallContext indexContext,
+                                       DrillParseContext parseContext,
+                                       DrillProjectRelBase project,
+                                       RelNode scan,
+                                       RexNode toRewriteRex,
+                                       RelDataType newRowType,
+                                       FunctionalIndexInfo functionInfo) {
+    if (!functionInfo.hasFunctional()) {
+      return toRewriteRex;
+    }
+    RexToExpression.RexToDrillExt rexToDrill = new RexToExpression.RexToDrillExt(parseContext, project, scan);
+    LogicalExpression expr = toRewriteRex.accept(rexToDrill);
+
+    final Map<LogicalExpression, Set<SchemaPath>> exprPathMap = functionInfo.getPathsInFunctionExpr();
+    PathInExpr exprSearch = new PathInExpr(exprPathMap);
+    expr.accept(exprSearch, null);
+    Set<LogicalExpression> remainderPaths = exprSearch.getRemainderPaths();
+
+    // now build the rex->logical expression map for SimpleRexRemap
+    // left out schema paths
+    Map<LogicalExpression, Set<RexNode>> exprToRex = rexToDrill.getMapExprToRex();
+    final Map<RexNode, LogicalExpression> mapRexExpr = Maps.newHashMap();
+    for (LogicalExpression leftExpr: remainderPaths) {
+      if (exprToRex.containsKey(leftExpr)) {
+        Set<RexNode> rexs = exprToRex.get(leftExpr);
+        for (RexNode rex: rexs) {
+          mapRexExpr.put(rex, leftExpr);
+        }
+      }
+    }
+
+    // functional expressions e.g. cast(a.b as int)
+    for (LogicalExpression functionExpr: functionInfo.getExprMap().keySet()) {
+      if (exprToRex.containsKey(functionExpr)) {
+        Set<RexNode> rexs = exprToRex.get(functionExpr);
+        for (RexNode rex: rexs) {
+          mapRexExpr.put(rex, functionExpr);
+        }
+      }
+
+    }
+
+    SimpleRexRemap remap = new SimpleRexRemap(indexContext.getScan(), newRowType, indexContext.getScan().getCluster().getRexBuilder());
+    remap.setExpressionMap(functionInfo.getExprMap());
+    return remap.rewriteWithMap(toRewriteRex, mapRexExpr);
+  }
+
+  public static RexNode getLeadingPrefixMap(Map<LogicalExpression, RexNode> leadingPrefixMap,
+                                  List<LogicalExpression> indexCols,
+                                  IndexConditionInfo.Builder builder, RexNode condition) {
+    boolean prefix = true;
+    int i=0;
+
+    RexNode initCondition = condition.isAlwaysTrue() ? null : condition;
+    while (prefix && i < indexCols.size()) {
+      LogicalExpression p = indexCols.get(i++);
+      List<LogicalExpression> prefixCol = ImmutableList.of(p);
+      IndexConditionInfo info = builder.indexConditionRelatedToFields(prefixCol, initCondition);
+      if(info != null && info.hasIndexCol) {
+        // the col had a match with one of the conditions; save the information about
+        // indexcol --> condition mapping
+        leadingPrefixMap.put(p, info.indexCondition);
+        initCondition = info.remainderCondition;
+        if (initCondition.isAlwaysTrue()) {
+          // all filter conditions are accounted for, so if the remainder is TRUE, set it to NULL because
+          // we don't need to keep track of it for rest of the index selection
+          initCondition = null;
+          break;
+        }
+      } else {
+        prefix = false;
+      }
+    }
+    return initCondition;
+  }
+
+  public static List<RexNode> getLeadingFilters (Map<LogicalExpression, RexNode> leadingPrefixMap, List<LogicalExpression> indexCols) {
+    List<RexNode> leadingFilters = Lists.newArrayList();
+    if (leadingPrefixMap.size() > 0) {
+      for (LogicalExpression p : indexCols) {
+        RexNode n;
+        if ((n = leadingPrefixMap.get(p)) != null) {
+          leadingFilters.add(n);
+        } else {
+          break; // break since the prefix property will not be preserved
+        }
+      }
+    }
+    return leadingFilters;
+  }
+
+  public static RexNode getLeadingColumnsFilter(List<RexNode> leadingFilters, RexBuilder rexBuilder) {
+    if (leadingFilters.size() > 0) {
+      RexNode leadingColumnsFilter = RexUtil.composeConjunction(rexBuilder, leadingFilters, false);
+      return leadingColumnsFilter;
+    }
+    return null;
+  }
+
+  public static RexNode getTotalRemainderFilter(RexNode indexColsRemFilter, RexNode incColsRemFilter, RexBuilder rexBuilder) {
+    if (indexColsRemFilter != null && incColsRemFilter != null) {
+      List<RexNode> operands = Lists.newArrayList();
+      operands.add(indexColsRemFilter);
+      operands.add(incColsRemFilter);
+      RexNode totalRemainder = RexUtil.composeConjunction(rexBuilder, operands, false);
+      return totalRemainder;
+    } else if (indexColsRemFilter != null) {
+      return indexColsRemFilter;
+    } else {
+      return incColsRemFilter;
+    }
+  }
+
+  public static RexNode getTotalFilter(RexNode leadColsFilter, RexNode totRemColsFilter, RexBuilder rexBuilder) {
+    RexNode condition = leadColsFilter;
+    if (leadColsFilter != null && totRemColsFilter != null && !totRemColsFilter.isAlwaysTrue()) {
+      List<RexNode> conditions = new ArrayList<RexNode>();
+      conditions.add(leadColsFilter);
+      conditions.add(totRemColsFilter);
+      return RexUtil.composeConjunction(rexBuilder, conditions, true);
+    }
+    return condition;
+  }
+}
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/index/IndexProperties.java b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/index/IndexProperties.java
index cfdd6d0..3059e27 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/index/IndexProperties.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/index/IndexProperties.java
@@ -37,6 +37,10 @@ public interface IndexProperties  {
                             RexNode indexColumnsRemainderFilter,
                             Statistics stats);
 
+  RexNode getLeadingColumnsFilter();
+
+  RexNode getTotalRemainderFilter();
+
   double getLeadingSelectivity();
 
   double getRemainderSelectivity();
@@ -49,7 +53,8 @@ public interface IndexProperties  {
 
   DrillScanRelBase getPrimaryTableScan();
 
-  RexNode getTotalRemainderFilter();
+  RelOptCost getIntersectCost(IndexGroup index, IndexConditionInfo.Builder builder,
+                                     RelOptPlanner planner);
 
   boolean satisfiesCollation();
 
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/index/IndexSelector.java b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/index/IndexSelector.java
new file mode 100644
index 0000000..b7f2b78
--- /dev/null
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/index/IndexSelector.java
@@ -0,0 +1,766 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.drill.exec.planner.index;
+
+import java.util.Collections;
+import java.util.Comparator;
+import java.util.List;
+import java.util.Map;
+
+import org.apache.calcite.plan.RelOptCost;
+import org.apache.calcite.plan.RelOptPlanner;
+import org.apache.calcite.rel.RelCollation;
+import org.apache.calcite.rel.RelCollationTraitDef;
+import org.apache.calcite.rel.metadata.RelMdUtil;
+import org.apache.calcite.rex.RexBuilder;
+import org.apache.calcite.rex.RexNode;
+import org.apache.calcite.rex.RexUtil;
+import org.apache.drill.common.expression.LogicalExpression;
+import org.apache.drill.exec.physical.base.DbGroupScan;
+import org.apache.drill.exec.planner.common.DrillJoinRelBase;
+import org.apache.drill.exec.planner.cost.DrillCostBase;
+import org.apache.drill.exec.planner.cost.PluginCost;
+import org.apache.drill.exec.planner.physical.PlannerSettings;
+import org.apache.drill.exec.planner.physical.PrelUtil;
+import org.apache.drill.exec.planner.physical.ScanPrel;
+import org.apache.drill.exec.planner.common.DrillScanRelBase;
+
+import org.apache.drill.shaded.guava.com.google.common.base.Preconditions;
+import org.apache.drill.shaded.guava.com.google.common.collect.ImmutableList;
+import org.apache.drill.shaded.guava.com.google.common.collect.Lists;
+import org.apache.drill.shaded.guava.com.google.common.collect.Maps;
+
+public class IndexSelector  {
+  static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(IndexSelector.class);
+  private static final double COVERING_TO_NONCOVERING_FACTOR = 100.0;
+  private RexNode indexCondition;   // filter condition on indexed columns
+  private RexNode otherRemainderCondition;  // remainder condition on all other columns
+  private double totalRows;
+  private Statistics stats;         // a Statistics instance that will be used to get estimated rowcount for filter conditions
+  private IndexConditionInfo.Builder builder;
+  private List<IndexProperties> indexPropList;
+  private DrillScanRelBase primaryTableScan;
+  private IndexCallContext indexContext;
+  private RexBuilder rexBuilder;
+
+  public IndexSelector(RexNode indexCondition,
+      RexNode otherRemainderCondition,
+      IndexCallContext indexContext,
+      IndexCollection collection,
+      RexBuilder rexBuilder,
+      double totalRows) {
+    this.indexCondition = indexCondition;
+    this.otherRemainderCondition = otherRemainderCondition;
+    this.indexContext = indexContext;
+    this.totalRows = totalRows;
+    this.stats = indexContext.getGroupScan().getStatistics();
+    this.rexBuilder = rexBuilder;
+    this.builder =
+        IndexConditionInfo.newBuilder(indexCondition, collection, rexBuilder, indexContext.getScan());
+    this.primaryTableScan = indexContext.getScan();
+    this.indexPropList = Lists.newArrayList();
+  }
+
+  /**
+   * This constructor is to build selector for no index condition case (no filter)
+   * @param indexContext
+   */
+  public IndexSelector(IndexCallContext indexContext) {
+    this.indexCondition = null;
+    this.otherRemainderCondition = null;
+    this.indexContext = indexContext;
+    this.totalRows = Statistics.ROWCOUNT_UNKNOWN;
+    this.stats = indexContext.getGroupScan().getStatistics();
+    this.rexBuilder = indexContext.getScan().getCluster().getRexBuilder();
+    this.builder = null;
+    this.primaryTableScan = indexContext.getScan();
+    this.indexPropList = Lists.newArrayList();
+  }
+
+  public void addIndex(IndexDescriptor indexDesc, boolean isCovering, int numProjectedFields) {
+    IndexProperties indexProps = new DrillIndexProperties(indexDesc, isCovering, otherRemainderCondition, rexBuilder,
+        numProjectedFields, totalRows, primaryTableScan);
+    indexPropList.add(indexProps);
+  }
+
+  /**
+   * This method analyzes an index's columns and starting from the first column, checks
+   * which part of the filter condition matches that column.  This process continues with
+   * subsequent columns.  The goal is to identify the portion of the filter condition that
+   * match the prefix columns.  If there are additional conditions that don't match prefix
+   * columns, that condition is set as a remainder condition.
+   * @param indexProps
+   */
+  public void analyzePrefixMatches(IndexProperties indexProps) {
+    RexNode initCondition = indexCondition.isAlwaysTrue() ? null : indexCondition;
+    Map<LogicalExpression, RexNode> leadingPrefixMap = Maps.newHashMap();
+    List<LogicalExpression> indexCols = indexProps.getIndexDesc().getIndexColumns();
+    boolean satisfiesCollation = false;
+
+    if (indexCols.size() > 0) {
+      if (initCondition != null) { // check filter condition
+        initCondition = IndexPlanUtils.getLeadingPrefixMap(leadingPrefixMap, indexCols, builder, indexCondition);
+      }
+      if (requiredCollation()) {
+        satisfiesCollation = buildAndCheckCollation(indexProps);
+      }
+    }
+
+    indexProps.setProperties(leadingPrefixMap, satisfiesCollation,
+        initCondition /* the remainder condition for indexed columns */, stats);
+  }
+
+  private boolean requiredCollation() {
+    if (indexContext.getCollationList() != null && indexContext.getCollationList().size() > 0) {
+      return true;
+    }
+    return false;
+  }
+
+  private boolean buildAndCheckCollation(IndexProperties indexProps) {
+    IndexDescriptor indexDesc = indexProps.getIndexDesc();
+    FunctionalIndexInfo functionInfo = indexDesc.getFunctionalInfo();
+
+    RelCollation inputCollation;
+    // for the purpose of collation we can assume that a covering index scan would provide
+    // the collation property that would be relevant for non-covering as well
+    ScanPrel indexScanPrel =
+        IndexPlanUtils.buildCoveringIndexScan(indexContext.getScan(), indexDesc.getIndexGroupScan(), indexContext, indexDesc);
+    inputCollation = indexScanPrel.getTraitSet().getTrait(RelCollationTraitDef.INSTANCE);
+
+    // we don't create collation for Filter because it will inherit the child's collation
+
+    if (indexContext.hasLowerProject()) {
+      inputCollation =
+          IndexPlanUtils.buildCollationProject(indexContext.getLowerProject().getProjects(), null,
+              indexContext.getScan(), functionInfo,indexContext);
+    }
+
+    if (indexContext.hasUpperProject()) {
+      inputCollation =
+          IndexPlanUtils.buildCollationProject(indexContext.getUpperProject().getProjects(), indexContext.getLowerProject(),
+              indexContext.getScan(), functionInfo, indexContext);
+    }
+
+    if ( (inputCollation != null) &&
+         (inputCollation.satisfies(indexContext.getCollation()))) {
+      return true;
+    }
+
+    return false;
+  }
+
+  private void addIndexIntersections(List<IndexGroup> candidateIndexes,
+      IndexConditionInfo.Builder infoBuilder, long maxIndexesToIntersect) {
+    // Sort the non-covering indexes for candidate intersect indexes. Generating all
+    // possible combinations of candidates is not feasible so we guide our decision
+    // based on selectivity/collation considerations
+    IndexGroup indexesInCandidate = new IndexGroup();
+    final double SELECTIVITY_UNKNOWN = -1.0;
+    // We iterate over indexes upto planner.index.max_indexes_to_intersect. An index which allows
+    // filter pushdown is added to the existing list of indexes provided it reduces the selectivity
+    // by COVERING_TO_NONCOVERING_FACTOR
+    double prevSel = SELECTIVITY_UNKNOWN;
+    for (int idx = 0; idx < candidateIndexes.size(); idx++) {
+      // Maximum allowed indexes is intersect plan reached!
+      if (indexesInCandidate.numIndexes() == maxIndexesToIntersect) {
+        break;
+      }
+      // Skip covering indexes
+      if (candidateIndexes.get(idx).getIndexProps().get(0).isCovering()) {
+        continue;
+      }
+      // Check if adding the current index to the already existing intersect indexes is redundant
+      List<IndexDescriptor> candidateDescs = Lists.newArrayList();
+      for (IndexProperties prop : indexesInCandidate.getIndexProps()) {
+        candidateDescs.add(prop.getIndexDesc());
+      }
+      candidateDescs.add(candidateIndexes.get(idx).getIndexProps().get(0).getIndexDesc());
+      Map<IndexDescriptor, IndexConditionInfo> intersectIdxInfoMap
+          = infoBuilder.getIndexConditionMap(candidateDescs);
+      // Current index redundant(no conditions pushed down) - skip!
+      if (intersectIdxInfoMap.keySet().size() < candidateDescs.size()) {
+        continue;
+      }
+      IndexProperties curProp = candidateIndexes.get(idx).getIndexProps().get(0);
+      indexesInCandidate.addIndexProp(curProp);
+      double currSel = 1.0;
+      if (indexesInCandidate.isIntersectIndex()) {
+        for (IndexProperties prop : indexesInCandidate.getIndexProps()) {
+          currSel *= prop.getLeadingSelectivity();
+        }
+        if (prevSel == SELECTIVITY_UNKNOWN ||
+            currSel/prevSel < COVERING_TO_NONCOVERING_FACTOR) {
+          prevSel = currSel;
+        } else {
+          indexesInCandidate.removeIndexProp(curProp);
+        }
+      }
+    }
+    // If intersect plan was generated, add it to the set of candidate indexes.
+    if (indexesInCandidate.isIntersectIndex()) {
+      candidateIndexes.add(indexesInCandidate);
+    }
+  }
+
+  /**
+   * Run the index selection algorithm and return the top N indexes
+   */
+  public void getCandidateIndexes(IndexConditionInfo.Builder infoBuilder, List<IndexGroup> coveringIndexes,
+      List<IndexGroup> nonCoveringIndexes, List<IndexGroup> intersectIndexes) {
+
+    RelOptPlanner planner = indexContext.getCall().getPlanner();
+    PlannerSettings settings = PrelUtil.getPlannerSettings(planner);
+    List<IndexGroup> candidateIndexes = Lists.newArrayList();
+
+    logger.info("index_plan_info: Analyzing {} indexes for prefix matches: {}",
+        indexPropList.size(), indexPropList);
+    // analysis phase
+    for (IndexProperties p : indexPropList) {
+      analyzePrefixMatches(p);
+
+      // only consider indexes that either have some leading prefix of the filter condition or
+      // can satisfy required collation
+      if (p.numLeadingFilters() > 0 || p.satisfiesCollation()) {
+        double selThreshold = p.isCovering() ? settings.getIndexCoveringSelThreshold() :
+          settings.getIndexNonCoveringSelThreshold();
+        // only consider indexes whose selectivity is <= the configured threshold OR consider
+        // all when full table scan is disable to avoid a CannotPlanException
+        if (settings.isDisableFullTableScan() || p.getLeadingSelectivity() <= selThreshold) {
+          IndexGroup index = new IndexGroup();
+          index.addIndexProp(p);
+          candidateIndexes.add(index);
+        }
+        else {
+          if (p.getLeadingSelectivity() > selThreshold) {
+            logger.debug("Skipping index {}. The leading selectivity {} is larger than threshold {}",
+                p.getIndexDesc().getIndexName(), p.getLeadingSelectivity(), selThreshold);
+          }
+        }
+      }
+    }
+
+    if (candidateIndexes.size() == 0) {
+      logger.info("index_plan_info: No suitable indexes found !");
+      return;
+    }
+
+    int max_candidate_indexes = (int)PrelUtil.getPlannerSettings(planner).getIndexMaxChosenIndexesPerTable();
+
+    // Ranking phase. Technically, we don't need to rank if there are fewer than max_candidate_indexes
+    // but we do it anyways for couple of reasons: the log output will show the indexes in a properly ranked
+    // order which helps diagnosing problems and secondly for internal unit/functional testing we want this code
+    // to be exercised even for few indexes
+    if (candidateIndexes.size() > 1) {
+      Collections.sort(candidateIndexes, new IndexComparator(planner, builder));
+    }
+
+    // Generate index intersections for ranking
+    addIndexIntersections(candidateIndexes, infoBuilder, settings.getMaxIndexesToIntersect());
+
+    // Sort again after intersect plan is added to the list
+    if (candidateIndexes.size() > 1) {
+      Collections.sort(candidateIndexes, new IndexComparator(planner, builder));
+    }
+
+    logger.info("index_plan_info: The top ranked indexes are: ");
+
+    int count = 0;
+    boolean foundCovering = false;
+    boolean foundCoveringCollation = false;
+    boolean foundNonCoveringCollation = false;
+
+    // pick the best N indexes
+    for (int i=0; i < candidateIndexes.size(); i++) {
+      IndexGroup index = candidateIndexes.get(i);
+      if (index.numIndexes() == 1
+          && index.getIndexProps().get(0).isCovering()) {
+        IndexProperties indexProps = index.getIndexProps().get(0);
+        if (foundCoveringCollation) {
+          // if previously we already found a higher ranked covering index that satisfies collation,
+          // then skip this one (note that selectivity and cost considerations were already handled
+          // by the ranking phase)
+          logger.debug("index_plan_info: Skipping covering index {} because a higher ranked covering index with collation already exists.", indexProps.getIndexDesc().getIndexName());
+          continue;
+        }
+        coveringIndexes.add(index);
+        logger.info("index_plan_info: name: {}, covering, collation: {}, leadingSelectivity: {}, cost: {}",
+            indexProps.getIndexDesc().getIndexName(),
+            indexProps.satisfiesCollation(),
+            indexProps.getLeadingSelectivity(),
+            indexProps.getSelfCost(planner));
+        count++;
+        foundCovering = true;
+        if (indexProps.satisfiesCollation()) {
+          foundCoveringCollation = true;
+        }
+      } else if (index.numIndexes() == 1) {  // non-covering
+        IndexProperties indexProps = index.getIndexProps().get(0);
+        // skip this non-covering index if (a) there was a higher ranked covering index
+        // with collation or (b) there was a higher ranked covering index and this
+        // non-covering index does not have collation
+        if (foundCoveringCollation ||
+            (foundCovering && !indexProps.satisfiesCollation())) {
+          logger.debug("index_plan_info: Skipping non-covering index {} because it does not have collation and a higher ranked covering index already exists.",
+              indexProps.getIndexDesc().getIndexName());
+          continue;
+        }
+        if (indexProps.satisfiesCollation()) {
+          foundNonCoveringCollation = true;
+        }
+        // all other non-covering indexes can be added to the list because 2 or more non-covering index could
+        // be considered for intersection later; currently the index selector is not costing the index intersection
+        nonCoveringIndexes.add(index);
+        logger.info("index_plan_info: name: {}, non-covering, collation: {}, leadingSelectivity: {}, cost: {}",
+            indexProps.getIndexDesc().getIndexName(),
+            indexProps.satisfiesCollation(),
+            indexProps.getLeadingSelectivity(),
+            indexProps.getSelfCost(planner));
+        count++;
+      } else {  // intersect indexes
+        if (foundCoveringCollation ||
+            (foundCovering && !index.getIndexProps().get(index.numIndexes()-1).satisfiesCollation()) ||
+            foundNonCoveringCollation) {
+          continue;
+        }
+        IndexGroup intersectIndex = new IndexGroup();
+        double isectLeadingSel = 1.0;
+        String isectName = "Intersect-"+count;
+        for (IndexProperties indexProps : index.getIndexProps()) {
+          intersectIndex.addIndexProp(indexProps);
+          isectLeadingSel *= indexProps.getLeadingSelectivity();
+          logger.info("name: {}, {}, collation: {}, leadingSelectivity: {}, cost: {}",
+              indexProps.getIndexDesc().getIndexName(),
+              isectName,
+              indexProps.satisfiesCollation(),
+              indexProps.getLeadingSelectivity(),
+              indexProps.getSelfCost(planner));
+        }
+        logger.info("name: {}, intersect-idx, collation: {}, leadingSelectivity: {}, cost: {}",
+            isectName,
+            index.getIndexProps().get(index.numIndexes()-1).satisfiesCollation(),
+            isectLeadingSel,
+            index.getIndexProps().get(0).getIntersectCost(index, builder, planner));
+        intersectIndexes.add(intersectIndex);
+      }
+      if (count == max_candidate_indexes) {
+        break;
+      }
+    }
+  }
+
+  /**
+   * we assume all the indexes added in indexPropList are all applicable (and covering).
+   * For now this function is used and tested only in IndexScanWithSortOnlyPrule
+   */
+  public IndexProperties getBestIndexNoFilter() {
+    if (indexPropList.size() == 0) {
+      return null;
+    }
+
+    RelOptPlanner planner = indexContext.getCall().getPlanner();
+    List<IndexGroup> candidateIndexes = Lists.newArrayList();
+
+    for (IndexProperties p : indexPropList) {
+      p.setSatisfiesCollation(buildAndCheckCollation(p));
+        IndexGroup index = new IndexGroup();
+        index.addIndexProp(p);
+        candidateIndexes.add(index);
+    }
+    Collections.sort(candidateIndexes, new IndexComparator(planner, builder));
+    return candidateIndexes.get(0).getIndexProps().get(0);
+  }
+
+  public static class IndexComparator implements Comparator<IndexGroup> {
+
+    private RelOptPlanner planner;
+    private IndexConditionInfo.Builder builder;
+    private PlannerSettings settings;
+
+    public IndexComparator(RelOptPlanner planner, IndexConditionInfo.Builder builder) {
+      this.planner = planner;
+      this.builder = builder;
+      this.settings = PrelUtil.getPlannerSettings(planner);
+    }
+
+    @Override
+    public int compare(IndexGroup index1, IndexGroup index2) {
+      // given a covering and a non-covering index, prefer covering index unless the
+      // difference in their selectivity is bigger than a configurable factor
+      List<IndexProperties> o1, o2;
+      boolean o1Covering, o2Covering, o1SatisfiesCollation, o2SatisfiesCollation;
+      int o1NumLeadingFilters, o2NumLeadingFilters;
+      double o1LeadingSelectivity, o2LeadingSelectivity;
+      DrillCostBase o1SelfCost, o2SelfCost;
+
+      o1 = index1.getIndexProps();
+      o2 = index2.getIndexProps();
+      Preconditions.checkArgument(o1.size() > 0 && o2.size() > 0);
+
+      if (o1.size() == 1) {
+        o1Covering = o1.get(0).isCovering();
+        o1NumLeadingFilters = o1.get(0).numLeadingFilters();
+        o1SatisfiesCollation = o1.get(0).satisfiesCollation();
+        o1LeadingSelectivity = o1.get(0).getLeadingSelectivity();
+      } else {
+        o1Covering = false;
+        // If the intersect plan is a left-deep join tree, the last index
+        // in the join would satisfy the collation. For now, assume no collation.
+        o1SatisfiesCollation = false;
+        o1NumLeadingFilters = o1.get(0).numLeadingFilters();
+        for (int idx=1; idx<o1.size(); idx++) {
+          o1NumLeadingFilters+=o1.get(idx).numLeadingFilters();
+        }
+        o1LeadingSelectivity = o1.get(0).getLeadingSelectivity();
+        for (int idx=1; idx<o1.size(); idx++) {
+          o1LeadingSelectivity*=o1.get(idx).getLeadingSelectivity();
+        }
+      }
+      if (o2.size() == 1) {
+        o2Covering = o2.get(0).isCovering();
+        o2NumLeadingFilters = o2.get(0).numLeadingFilters();
+        o2SatisfiesCollation = o2.get(0).satisfiesCollation();
+        o2LeadingSelectivity = o2.get(0).getLeadingSelectivity();
+      } else {
+        o2Covering = false;
+        // If the intersect plan is a left-deep join tree, the last index
+        // in the join would satisfy the collation. For now, assume no collation.
+        o2SatisfiesCollation = false;
+        o2NumLeadingFilters = o2.get(0).numLeadingFilters();
+        for (int idx=1; idx<o2.size(); idx++) {
+          o2NumLeadingFilters+=o2.get(idx).numLeadingFilters();
+        }
+        o2LeadingSelectivity = o2.get(0).getLeadingSelectivity();
+        for (int idx=1; idx<o2.size(); idx++) {
+          o2LeadingSelectivity*=o2.get(idx).getLeadingSelectivity();
+        }
+      }
+
+      if (o1Covering && !o2Covering) {
+        if (o1LeadingSelectivity/o2LeadingSelectivity < COVERING_TO_NONCOVERING_FACTOR) {
+          return -1;  // covering is ranked higher (better) than non-covering
+        }
+      }
+
+      if (o2Covering && !o1Covering) {
+        if (o2LeadingSelectivity/o1LeadingSelectivity < COVERING_TO_NONCOVERING_FACTOR) {
+          return 1;  // covering is ranked higher (better) than non-covering
+        }
+      }
+
+      if (!o1Covering && !o2Covering) {
+        if (o1.size() > 1) {
+          if (o1LeadingSelectivity/o2LeadingSelectivity < COVERING_TO_NONCOVERING_FACTOR) {
+            return -1; // Intersect is ranked higher than non-covering/intersect
+          }
+        } else if (o2.size() > 1) {
+          if (o2LeadingSelectivity/o1LeadingSelectivity < COVERING_TO_NONCOVERING_FACTOR) {
+            return -1; // Intersect is ranked higher than non-covering/intersect
+          }
+        }
+      }
+
+      if (o1SatisfiesCollation && !o2SatisfiesCollation) {
+        return -1;  // index with collation is ranked higher (better) than one without collation
+      } else if (o2SatisfiesCollation && !o1SatisfiesCollation) {
+        return 1;
+      }
+
+      if (o1.size() == 1) {
+        o1SelfCost = (DrillCostBase) o1.get(0).getSelfCost(planner);
+      } else {
+        o1SelfCost = (DrillCostBase) o1.get(0).getIntersectCost(index1, builder, planner);
+      }
+      if (o2.size() == 1) {
+        o2SelfCost = (DrillCostBase) o2.get(0).getSelfCost(planner);
+      } else {
+        o2SelfCost = (DrillCostBase) o2.get(0).getIntersectCost(index2, builder, planner);
+      }
+
+      DrillCostBase cost1 = o1SelfCost;
+      DrillCostBase cost2 = o2SelfCost;
+
+      if (cost1.isLt(cost2)) {
+        return -1;
+      } else if (cost1.isEqWithEpsilon(cost2)) {
+        if (o1NumLeadingFilters > o2NumLeadingFilters) {
+          return -1;
+        } else if (o1NumLeadingFilters < o2NumLeadingFilters) {
+          return 1;
+        }
+        return 0;
+      } else {
+        return 1;
+      }
+    }
+  }
+
+  /**
+   * IndexProperties encapsulates the various metrics of a single index that are related to
+   * the current query. These metrics are subsequently used to rank the index in comparison
+   * with other indexes.
+   */
+  public static class DrillIndexProperties  implements IndexProperties {
+    private IndexDescriptor indexDescriptor; // index descriptor
+
+    private double leadingSel = 1.0;    // selectivity of leading satisfiable conjunct
+    private double remainderSel = 1.0;  // selectivity of all remainder satisfiable conjuncts
+    private boolean satisfiesCollation = false; // whether index satisfies collation
+    private boolean isCovering = false;         // whether index is covering
+    private double avgRowSize;          // avg row size in bytes of the selected part of index
+
+    private int numProjectedFields;
+    private double totalRows;
+    private DrillScanRelBase primaryTableScan = null;
+    private RelOptCost selfCost = null;
+
+    private List<RexNode> leadingFilters = Lists.newArrayList();
+    private Map<LogicalExpression, RexNode> leadingPrefixMap;
+    private RexNode indexColumnsRemainderFilter = null;
+    private RexNode otherColumnsRemainderFilter = null;
+    private RexBuilder rexBuilder;
+
+    public DrillIndexProperties(IndexDescriptor indexDescriptor,
+        boolean isCovering,
+        RexNode otherColumnsRemainderFilter,
+        RexBuilder rexBuilder,
+        int numProjectedFields,
+        double totalRows,
+        DrillScanRelBase primaryTableScan) {
+      this.indexDescriptor = indexDescriptor;
+      this.isCovering = isCovering;
+      this.otherColumnsRemainderFilter = otherColumnsRemainderFilter;
+      this.rexBuilder = rexBuilder;
+      this.numProjectedFields = numProjectedFields;
+      this.totalRows = totalRows;
+      this.primaryTableScan = primaryTableScan;
+    }
+
+    public void setProperties(Map<LogicalExpression, RexNode> prefixMap,
+        boolean satisfiesCollation,
+        RexNode indexColumnsRemainderFilter,
+        Statistics stats) {
+      this.indexColumnsRemainderFilter = indexColumnsRemainderFilter;
+      this.satisfiesCollation = satisfiesCollation;
+      leadingPrefixMap = prefixMap;
+
+      logger.info("index_plan_info: Index {}: leading prefix map: {}, satisfies collation: {}, remainder condition: {}",
+          indexDescriptor.getIndexName(), leadingPrefixMap, satisfiesCollation, indexColumnsRemainderFilter);
+
+      // iterate over the columns in the index descriptor and lookup from the leadingPrefixMap
+      // the corresponding conditions
+      leadingFilters = IndexPlanUtils.getLeadingFilters(leadingPrefixMap, indexDescriptor.getIndexColumns());
+
+      // compute the estimated row count by calling the statistics APIs
+      // NOTE: the calls to stats.getRowCount() below supply the primary table scan
+      // which is ok because its main use is to convert the ordinal-based filter
+      // to a string representation for stats lookup.
+      String idxIdentifier = stats.buildUniqueIndexIdentifier(this.getIndexDesc());
+      for (RexNode filter : leadingFilters) {
+        double filterRows = stats.getRowCount(filter, idxIdentifier, primaryTableScan /* see comment above */);
+        double sel = 1.0;
+        if (filterRows != Statistics.ROWCOUNT_UNKNOWN) {
+          sel = filterRows/totalRows;
+          logger.info("index_plan_info: Filter: {}, filterRows = {}, totalRows = {}, selectivity = {}",
+              filter, filterRows, totalRows, sel);
+        } else {
+          sel = RelMdUtil.guessSelectivity(filter);
+          if (stats.isStatsAvailable()) {
+            logger.debug("index_plan_info: Filter row count is UNKNOWN for filter: {}, using guess {}", filter, sel);
+          }
+        }
+        leadingSel *= sel;
+      }
+
+      logger.debug("index_plan_info: Combined selectivity of all leading filters: {}", leadingSel);
+
+      if (indexColumnsRemainderFilter != null) {
+        // The remainder filter is evaluated against the primary table i.e. NULL index
+        double remFilterRows = stats.getRowCount(indexColumnsRemainderFilter, null, primaryTableScan);
+        if (remFilterRows != Statistics.ROWCOUNT_UNKNOWN) {
+          remainderSel = remFilterRows/totalRows;
+          logger.debug("index_plan_info: Selectivity of index columns remainder filters: {}", remainderSel);
+        } else {
+          remainderSel = RelMdUtil.guessSelectivity(indexColumnsRemainderFilter);
+          if (stats.isStatsAvailable()) {
+            logger.debug("index_plan_info: Filter row count is UNKNOWN for remainder filter : {}, using guess {}",
+                indexColumnsRemainderFilter, remainderSel);
+          }
+        }
+      }
+
+      // get the average row size based on the leading column filter
+      avgRowSize = stats.getAvgRowSize(idxIdentifier, false);
+      if (avgRowSize == Statistics.AVG_ROWSIZE_UNKNOWN) {
+        avgRowSize = numProjectedFields * Statistics.AVG_COLUMN_SIZE;
+        if (stats.isStatsAvailable()) {
+          logger.debug("index_plan_info: Average row size is UNKNOWN based on leading filter: {}, using guess {}, columns {}, columnSize {}",
+              leadingFilters.size() > 0 ? leadingFilters.get(0).toString() : "<NULL>",
+              avgRowSize, numProjectedFields, Statistics.AVG_COLUMN_SIZE);
+        }
+      } else {
+        logger.debug("index_plan_info: Filter: {}, Average row size: {}",
+            leadingFilters.size() > 0 ? leadingFilters.get(0).toString() : "<NULL>", avgRowSize);
+      }
+    }
+
+    public double getLeadingSelectivity() {
+      return leadingSel;
+    }
+
+    public double getRemainderSelectivity() {
+      return remainderSel;
+    }
+
+    public boolean isCovering() {
+      return isCovering;
+    }
+
+    public double getTotalRows() {
+      return totalRows;
+    }
+
+    public IndexDescriptor getIndexDesc() {
+      return indexDescriptor;
+    }
+
+    public RexNode getLeadingColumnsFilter() {
+      return IndexPlanUtils.getLeadingColumnsFilter(leadingFilters, rexBuilder);
+    }
+
+    public RexNode getTotalRemainderFilter() {
+      return IndexPlanUtils.getTotalRemainderFilter(indexColumnsRemainderFilter, otherColumnsRemainderFilter, rexBuilder);
+    }
+
+    public boolean satisfiesCollation() {
+      return satisfiesCollation;
+    }
+
+    public void setSatisfiesCollation(boolean satisfiesCollation) {
+      this.satisfiesCollation = satisfiesCollation;
+    }
+
+    public RelOptCost getSelfCost(RelOptPlanner planner) {
+      if (selfCost != null) {
+        return selfCost;
+      }
+      selfCost = indexDescriptor.getCost(this, planner, numProjectedFields, IndexPlanUtils.getGroupScan(primaryTableScan));
+      return selfCost;
+    }
+
+    public RelOptCost getIntersectCost(IndexGroup index, IndexConditionInfo.Builder builder,
+        RelOptPlanner planner) {
+      return getIntersectCost(index, builder, planner, indexDescriptor.getPluginCostModel(), primaryTableScan);
+    }
+
+    public int numLeadingFilters() {
+      return leadingFilters.size();
+    }
+
+    public double getAvgRowSize() {
+      return avgRowSize;
+    }
+
+    public DrillScanRelBase getPrimaryTableScan() {
+      return this.primaryTableScan;
+    }
+
+    public RelOptCost getIntersectCost(IndexGroup index, IndexConditionInfo.Builder builder,
+                                       RelOptPlanner planner, PluginCost costBase, DrillScanRelBase scanRel) {
+      DrillCostBase.DrillCostFactory costFactory = (DrillCostBase.DrillCostFactory)planner.getCostFactory();
+      double totLeadRowCount = 1.0;
+      double totalRows = 0.0, totCpuCost = 0.0, totDiskCost = 0.0, totNetworkCost = 0.0, totMemoryCost = 0.0;
+      double rightSideRows = Statistics.ROWCOUNT_UNKNOWN;
+      DbGroupScan primaryTableGroupScan = (DbGroupScan) IndexPlanUtils.getGroupScan(scanRel);
+      RexNode remFilters;
+      final List<RexNode> remFilterList = Lists.newArrayList();
+      for (IndexProperties indexProps : index.getIndexProps()) {
+        remFilterList.add(indexProps.getTotalRemainderFilter());
+      }
+      remFilters = RexUtil.composeConjunction(scanRel.getCluster().getRexBuilder(), remFilterList, false);
+
+      for (IndexProperties indexProps : index.getIndexProps()) {
+        totalRows = indexProps.getTotalRows();
+        double leadRowCount = indexProps.getLeadingSelectivity() * indexProps.getRemainderSelectivity() * totalRows;
+        totLeadRowCount *= indexProps.getLeadingSelectivity();
+        double avgRowSize = indexProps.getAvgRowSize();
+        Preconditions.checkArgument(primaryTableGroupScan instanceof DbGroupScan);
+        // IO Cost (Filter evaluation CPU cost for pushed down filters)
+        double numBlocksIndex = Math.ceil((leadRowCount * avgRowSize) / costBase.getBlockSize(primaryTableGroupScan));
+        double diskCostIndex = numBlocksIndex * costBase.getSequentialBlockReadCost(primaryTableGroupScan);
+        totDiskCost += diskCostIndex;
+        // Join Cost (include network cost?) borrow from Hash Join
+        if (rightSideRows != Statistics.ROWCOUNT_UNKNOWN) {
+          DrillCostBase joinCost = (DrillCostBase) DrillJoinRelBase.computeHashJoinCostWithRowCntKeySize(planner,
+              rightSideRows, leadRowCount, 1);
+          totDiskCost += joinCost.getIo();
+          totCpuCost += joinCost.getCpu();
+          totMemoryCost += joinCost.getMemory();
+          // No NDV statistics; compute join rowcount as max of build side and probe side rows
+          rightSideRows = PrelUtil.getPlannerSettings(planner).getRowCountEstimateFactor() *
+              Math.max(leadRowCount, rightSideRows);
+        } else {
+          rightSideRows = leadRowCount;
+        }
+        remFilters = remainderCondition(indexProps.getIndexDesc(), builder, remFilters);
+      }
+
+      totLeadRowCount *= totalRows;
+      // for the primary table join-back each row may belong to a different block, so in general num_blocks = num_rows;
+      // however, num_blocks cannot exceed the total number of blocks of the table
+      DbGroupScan dbGroupScan = (DbGroupScan) primaryTableGroupScan;
+      double totalBlocksPrimary = Math.ceil((dbGroupScan.getColumns().size() *
+          costBase.getAverageColumnSize(dbGroupScan) * totalRows) / costBase.getBlockSize(dbGroupScan));
+      double diskBlocksPrimary = Math.min(totalBlocksPrimary, totLeadRowCount);
+      double diskCostPrimary = diskBlocksPrimary * costBase.getRandomBlockReadCost(dbGroupScan);
+      totDiskCost += diskCostPrimary;
+      // CPU cost of remainder condition evaluation over the selected rows
+      if (remFilters != null) {
+        totCpuCost += totLeadRowCount * DrillCostBase.COMPARE_CPU_COST;
+      }
+      double networkCost = 0.0; // TODO: add network cost once full table scan also considers network cost
+      return costFactory.makeCost(totLeadRowCount, totCpuCost, totDiskCost, totNetworkCost, totMemoryCost);
+    }
+
+    public RexNode remainderCondition(IndexDescriptor indexDesc, IndexConditionInfo.Builder builder,
+                                      RexNode initCondition) {
+      List<LogicalExpression> indexCols = indexDesc.getIndexColumns();
+      boolean prefix = true;
+      if (indexCols.size() > 0 && initCondition != null) {
+        int i=0;
+        while (prefix && i < indexCols.size()) {
+          LogicalExpression p = indexCols.get(i++);
+          List<LogicalExpression> prefixCol = ImmutableList.of(p);
+          IndexConditionInfo info = builder.indexConditionRelatedToFields(prefixCol, initCondition);
+          if(info != null && info.hasIndexCol) {
+            initCondition = info.remainderCondition;
+            if (initCondition.isAlwaysTrue()) {
+              // all filter conditions are accounted for, so if the remainder is TRUE, set it to NULL because
+              // we don't need to keep track of it for rest of the index selection
+              initCondition = null;
+              break;
+            }
+          } else {
+            prefix = false;
+          }
+        }
+      }
+      return initCondition;
+    }
+  }
+}
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/index/IndexableExprMarker.java b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/index/IndexableExprMarker.java
index a1a6fc8..eb91833 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/index/IndexableExprMarker.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/index/IndexableExprMarker.java
@@ -173,7 +173,7 @@ public class IndexableExprMarker extends RexVisitorImpl<Boolean> {
 
   @Override
   public Boolean visitCall(RexCall call) {
-    if (call.getKind() == SqlKind.NOT) {
+    if (call.getKind() == SqlKind.NOT || call.getKind() == SqlKind.NOT_EQUALS || call.getKind() == SqlKind.NOT_IN) {
       // Conditions under NOT are not indexable
       return false;
     }
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/index/PathInExpr.java b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/index/PathInExpr.java
new file mode 100644
index 0000000..124fece
--- /dev/null
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/index/PathInExpr.java
@@ -0,0 +1,147 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.drill.exec.planner.index;
+
+import org.apache.drill.shaded.guava.com.google.common.collect.Sets;
+import org.apache.drill.common.expression.CastExpression;
+import org.apache.drill.common.expression.FunctionCall;
+import org.apache.drill.common.expression.FunctionHolderExpression;
+import org.apache.drill.common.expression.IfExpression;
+import org.apache.drill.common.expression.LogicalExpression;
+import org.apache.drill.common.expression.SchemaPath;
+
+import org.apache.drill.common.expression.visitors.AbstractExprVisitor;
+
+import java.util.Map;
+import java.util.Set;
+
+/**
+ * Class PathInExpr is to recursively analyze a expression trees with a map of indexed expression collected from indexDescriptor,
+ * e.g. Map 'cast(a.q as int)' -> '$0' means the expression 'cast(a.q as int)' is named as '$0' in index table.
+ *
+ * for project expressions: cast(a.q as int), a.q, b + c, PathInExpr will get remainderPath {a.q, b, c}, which is
+ * helpful to determine if q query is covering. remainderPathsInFunction will be {a.q}, which will be used to
+ * decide if the {a.q} in scan column and project rowtype should be removed or not.
+ *
+ * This class could be more generic to support any expression, for now it works for only 'cast(schemapath as type)'.
+ */
+public class PathInExpr extends AbstractExprVisitor<Boolean,Void,RuntimeException> {
+
+  //functional index fields and their involved SchemaPaths
+  final private Map<LogicalExpression, Set<SchemaPath>> pathsInExpr;
+
+  //collection of paths in all functional indexes
+  private Set<SchemaPath> allPaths;
+
+  //the paths were found out of functional index expressions in query.
+  private Set<LogicalExpression> remainderPaths;
+
+  //the paths in functional index fields but were found out of index functions expression in query
+  private Set<LogicalExpression> remainderPathsInFunctions;
+
+  //constructor is provided a map of all functional expressions to the paths involved in  the expressions.
+  public PathInExpr(Map<LogicalExpression, Set<SchemaPath>> pathsInExpr) {
+    this.pathsInExpr = pathsInExpr;
+    allPaths = Sets.newHashSet();
+    remainderPaths = Sets.newHashSet();
+    remainderPathsInFunctions = Sets.newHashSet();
+    for(Map.Entry<LogicalExpression, Set<SchemaPath>> entry: pathsInExpr.entrySet()) {
+      allPaths.addAll(entry.getValue());
+    }
+  }
+
+  public Set<LogicalExpression> getRemainderPaths() {
+    return remainderPaths;
+  }
+
+  public Set<LogicalExpression> getRemainderPathsInFunctions() {
+    return remainderPathsInFunctions;
+  }
+
+  private boolean preProcess(LogicalExpression inExpr) {
+    if (pathsInExpr.containsKey(inExpr)) {
+      return true;
+    }
+    return false;
+  }
+
+  @Override
+  public Boolean visitFunctionCall(FunctionCall call, Void value) throws RuntimeException {
+    if (preProcess(call)) {
+      //when it is true, we know this is exactly the indexed expression, no more deep search
+      return true;
+    }
+
+    boolean bret = true;
+    for (LogicalExpression arg : call.args) {
+      bret &= arg.accept(this, null);
+      if(bret == false) {
+        break;
+      }
+    }
+    return bret;
+  }
+
+  @Override
+  public Boolean visitFunctionHolderExpression(FunctionHolderExpression holder, Void value) throws RuntimeException {
+    if (preProcess(holder)) {
+      //when it is true, we know this is exactly the indexed expression, no more deep search
+      return true;
+    }
+    for (LogicalExpression arg : holder.args) {
+      arg.accept(this, null);
+    }
+    return null;
+  }
+
+  public Boolean visitCastExpression(CastExpression castExpr, Void value) throws RuntimeException {
+    if (preProcess(castExpr)) {
+      //when it is true, we know this is exactly the indexed expression, no more deep search
+      return true;
+    }
+    return castExpr.getInput().accept(this, null);
+  }
+
+  @Override
+  public Boolean visitIfExpression(IfExpression ifExpr, Void value) throws RuntimeException {
+    return (ifExpr.ifCondition.condition.accept(this, null)
+        && ifExpr.ifCondition.expression.accept(this, null)
+        && ifExpr.elseExpression.accept(this, null));
+  }
+
+  @Override
+  public Boolean visitSchemaPath(SchemaPath path, Void value) throws RuntimeException {
+    // we can come to here means this path was found out of indexed expressions,
+    // so there is a path from query is not covered in functions
+    remainderPaths.add(path);
+
+    if(allPaths.contains(path)) {
+      // 'path' is a path involved in a functional index field,
+      remainderPathsInFunctions.add(path);
+
+      return false;
+    }
+    //it is not in
+    return true;
+  }
+
+  @Override
+  public Boolean visitUnknown(LogicalExpression e, Void value) throws RuntimeException {
+    return true;
+  }
+}
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/index/RexSeparator.java b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/index/RexSeparator.java
new file mode 100644
index 0000000..8047724
--- /dev/null
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/index/RexSeparator.java
@@ -0,0 +1,116 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.drill.exec.planner.index;
+
+import org.apache.drill.shaded.guava.com.google.common.collect.Maps;
+import org.apache.calcite.rel.RelNode;
+import org.apache.calcite.rex.RexBuilder;
+import org.apache.calcite.rex.RexCall;
+import org.apache.calcite.rex.RexInputRef;
+import org.apache.calcite.rex.RexNode;
+import org.apache.drill.common.expression.CastExpression;
+import org.apache.drill.common.expression.LogicalExpression;
+import org.apache.drill.common.expression.SchemaPath;
+import org.apache.drill.common.types.TypeProtos;
+import org.apache.drill.exec.planner.logical.partition.FindPartitionConditions;
+
+import java.util.BitSet;
+import java.util.List;
+import java.util.Map;
+
+public class RexSeparator {
+
+  final private List<LogicalExpression> relatedPaths;
+  final private RelNode inputRel;
+  final private RexBuilder builder;
+
+  public RexSeparator(List<LogicalExpression> relatedPaths, RelNode inputRel, RexBuilder builder) {
+    this.relatedPaths = relatedPaths;
+    this.inputRel = inputRel;
+    this.builder = builder;
+  }
+
+  public RexNode getSeparatedCondition(RexNode expr) {
+    IndexableExprMarker marker = new IndexableExprMarker(inputRel);
+    expr.accept(marker);
+
+    final Map<RexNode, LogicalExpression> markMap = Maps.newHashMap();
+    final Map<RexNode, LogicalExpression> relevantRexMap = marker.getIndexableExpression();
+    for(Map.Entry<RexNode, LogicalExpression> entry : relevantRexMap.entrySet()) {
+      //for the expressions found in expr, only these in relatedPaths is related
+      LogicalExpression relevantExpr = entry.getValue();
+      int idxFound = relatedPaths.indexOf(relevantExpr);
+      if ( idxFound >= 0 ) {
+        if (relevantExpr instanceof SchemaPath) {
+          //case sensitive comparison
+          if (!((SchemaPath) relevantExpr).toExpr().equals(
+              ((SchemaPath) relatedPaths.get(idxFound)).toExpr())) {
+            continue;
+          }
+        }
+        else if (relevantExpr instanceof CastExpression) {
+          final CastExpression castExprInFilter = (CastExpression) relevantExpr;
+          if (castExprInFilter.getMajorType().getMinorType() == TypeProtos.MinorType.VARCHAR
+              && (castExprInFilter.getMajorType().getPrecision() > relatedPaths.get(idxFound).getMajorType().getPrecision())) {
+            continue;
+          }
+        }
+        markMap.put(entry.getKey(), entry.getValue());
+      }
+    }
+
+    ConditionSeparator separator = new ConditionSeparator(markMap, builder);
+    separator.analyze(expr);
+    return separator.getFinalCondition();
+  }
+
+  private static class ConditionSeparator extends  FindPartitionConditions {
+
+    final private Map<RexNode, LogicalExpression> markMap;
+    private boolean inAcceptedPath;
+
+    public ConditionSeparator(Map<RexNode, LogicalExpression> markMap, RexBuilder builder) {
+      super(new BitSet(), builder);
+      this.markMap = markMap;
+      inAcceptedPath = false;
+    }
+
+    @Override
+    protected boolean inputRefToPush(RexInputRef inputRef) {
+      //this class will based on the schemaPath to decide what to push
+      if (markMap.containsKey(inputRef) || inAcceptedPath) {
+        return true;
+      }
+      return false;
+    }
+
+    @Override
+    public Void visitCall(RexCall call) {
+      boolean oldValue = inAcceptedPath;
+      try {
+        if (markMap.containsKey(call)) {
+          inAcceptedPath = true;
+
+        }
+        return super.visitCall(call);
+      } finally {
+        inAcceptedPath = oldValue;
+      }
+    }
+  }
+}
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/index/RexToExpression.java b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/index/RexToExpression.java
new file mode 100644
index 0000000..6b143fb
--- /dev/null
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/index/RexToExpression.java
@@ -0,0 +1,165 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.drill.exec.planner.index;
+
+import org.apache.drill.shaded.guava.com.google.common.collect.Maps;
+import org.apache.drill.shaded.guava.com.google.common.collect.Sets;
+import org.apache.calcite.rel.RelNode;
+import org.apache.calcite.rel.type.RelDataType;
+import org.apache.calcite.rel.type.RelDataTypeField;
+import org.apache.calcite.rex.RexBuilder;
+import org.apache.calcite.rex.RexCall;
+import org.apache.calcite.rex.RexCorrelVariable;
+import org.apache.calcite.rex.RexDynamicParam;
+import org.apache.calcite.rex.RexFieldAccess;
+import org.apache.calcite.rex.RexInputRef;
+import org.apache.calcite.rex.RexLocalRef;
+import org.apache.calcite.rex.RexNode;
+import org.apache.calcite.rex.RexOver;
+import org.apache.calcite.rex.RexRangeRef;
+import org.apache.drill.exec.planner.common.DrillProjectRelBase;
+import org.apache.drill.common.expression.FieldReference;
+import org.apache.drill.common.expression.LogicalExpression;
+import org.apache.drill.exec.planner.logical.DrillOptiq;
+import org.apache.drill.exec.planner.logical.DrillParseContext;
+
+import java.util.Map;
+import java.util.Set;
+
+/**
+ * This class is an enhanced version for DrillOptiq,
+ * 1, it can convert expressions in one more layer(project that have expression) above project-scan,
+ * while DrillOptiq can only convert expressions directly reference scan's row type,
+ * 2, it can record the generated LogicalExpression of each rexNode in the rexNode tree for future reference
+ * this result can serve future rewrite that need to locate particular LogicalExpressions
+ */
+public class RexToExpression {
+
+  public static LogicalExpression toDrill(DrillParseContext context, DrillProjectRelBase project, RelNode input, RexNode expr) {
+    final RexToExpression.RexToDrillExt visitor = new RexToExpression.RexToDrillExt(context, project, input);
+    return expr.accept(visitor);
+  }
+
+  public static class RexToDrillExt extends DrillOptiq.RexToDrill {
+
+    /**
+     * projectToInput, this instance of RexToDrill can do convert for expressions that are directly using inputRef of 'input',
+     * e.g. rex nodes in 'project'.
+     *
+     */
+    private final DrillOptiq.RexToDrill projectToInput;
+
+    /**
+     * The project in between
+     */
+    private final DrillProjectRelBase project;
+
+    /**
+     * multiple rexNodes could convert to the same logical expressions. map the result expression to the equivalent rexNodes.
+     */
+    private final Map<LogicalExpression, Set<RexNode>> exprToRexs;
+
+
+    public RexToDrillExt(DrillParseContext context, DrillProjectRelBase project, RelNode input) {
+      super(context, input);
+      projectToInput = new DrillOptiq.RexToDrill(context, input);
+      this.project = project;
+      this.exprToRexs = Maps.newHashMap();
+    }
+
+    public Map<LogicalExpression, Set<RexNode>> getMapExprToRex() {
+      return exprToRexs;
+    }
+
+    private LogicalExpression addExprToRexs(LogicalExpression expr, RexNode rex) {
+      if (!exprToRexs.containsKey(expr)) {
+        exprToRexs.put(expr, Sets.newHashSet(rex));
+      }
+      else {
+        exprToRexs.get(expr).add(rex);
+      }
+      return expr;
+    }
+
+    protected RelDataType getRowType() {
+      if(project != null) {
+        return project.getRowType();
+      }
+      return super.getRowType();
+    }
+
+    protected RexBuilder getRexBuilder() {
+      if(project != null) {
+        return project.getCluster().getRexBuilder();
+      }
+      return super.getRexBuilder();
+    }
+
+    @Override
+    public LogicalExpression visitInputRef(RexInputRef inputRef) {
+      final int index = inputRef.getIndex();
+
+      //no extra project in between of input and this layer
+      if(project == null) {
+        final RelDataTypeField field = getRowType().getFieldList().get(index);
+        return addExprToRexs(FieldReference.getWithQuotedRef(field.getName()), inputRef);
+      }
+
+      //get the actual expression of this projected rex
+      RexNode rex = IndexPlanUtils.getProjects(project).get(index);
+      return addExprToRexs(rex.accept(projectToInput), inputRef);
+    }
+
+    @Override
+    public LogicalExpression visitCall(RexCall call) {
+
+      return addExprToRexs(super.visitCall(call), call);
+    }
+
+    @Override
+    public LogicalExpression visitLocalRef(RexLocalRef localRef) {
+      return addExprToRexs(super.visitLocalRef(localRef), localRef);
+    }
+
+    @Override
+    public LogicalExpression visitOver(RexOver over) {
+      return addExprToRexs(super.visitOver(over), over);
+    }
+
+    @Override
+    public LogicalExpression visitCorrelVariable(RexCorrelVariable correlVariable) {
+      return addExprToRexs(super.visitCorrelVariable(correlVariable), correlVariable);
+    }
+
+    @Override
+    public LogicalExpression visitDynamicParam(RexDynamicParam dynamicParam) {
+      return addExprToRexs(super.visitDynamicParam(dynamicParam), dynamicParam);
+    }
+
+    @Override
+    public LogicalExpression visitRangeRef(RexRangeRef rangeRef) {
+      return addExprToRexs(super.visitRangeRef(rangeRef), rangeRef);
+    }
+
+    @Override
+    public LogicalExpression visitFieldAccess(RexFieldAccess fieldAccess) {
+      return addExprToRexs(super.visitFieldAccess(fieldAccess), fieldAccess);
+    }
+  }
+
+}
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/index/SimpleRexRemap.java b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/index/SimpleRexRemap.java
new file mode 100644
index 0000000..eba44b1
--- /dev/null
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/index/SimpleRexRemap.java
@@ -0,0 +1,300 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.drill.exec.planner.index;
+
+import org.apache.drill.shaded.guava.com.google.common.collect.ImmutableMap;
+import org.apache.drill.shaded.guava.com.google.common.collect.Maps;
+import org.apache.calcite.rel.RelNode;
+import org.apache.calcite.rel.type.RelDataType;
+import org.apache.calcite.rel.type.RelDataTypeField;
+import org.apache.calcite.rex.RexBuilder;
+import org.apache.calcite.rex.RexCall;
+import org.apache.calcite.rex.RexCorrelVariable;
+import org.apache.calcite.rex.RexDynamicParam;
+import org.apache.calcite.rex.RexFieldAccess;
+
+import org.apache.calcite.rex.RexInputRef;
+import org.apache.calcite.rex.RexLiteral;
+import org.apache.calcite.rex.RexLocalRef;
+import org.apache.calcite.rex.RexNode;
+import org.apache.calcite.rex.RexOver;
+import org.apache.calcite.rex.RexRangeRef;
+import org.apache.calcite.rex.RexShuttle;
+import org.apache.calcite.rex.RexVisitorImpl;
+
+import org.apache.calcite.sql.fun.SqlStdOperatorTable;
+import org.apache.calcite.sql.type.SqlTypeName;
+import org.apache.calcite.util.NlsString;
+import org.apache.drill.common.expression.LogicalExpression;
+import org.apache.drill.common.expression.PathSegment;
+
+import java.util.List;
+import java.util.Map;
+
+/**
+ * Rewrite RexNode with these policies:
+ * 1) field renamed. The input field was named differently in index table,
+ * 2) field is in different position of underlying rowtype
+ *
+ * TODO: 3) certain operator needs rewriting. e.g. CAST function
+ * This class for now applies to only filter on scan, for filter-on-project-on-scan. A stack of
+ * rowType is required.
+ */
+public class SimpleRexRemap {
+  final RelNode origRel;
+  final RelDataType origRowType;
+  final RelDataType newRowType;
+
+  private RexBuilder builder;
+  private Map<LogicalExpression, LogicalExpression> destExprMap;
+
+  public SimpleRexRemap(RelNode origRel,
+                        RelDataType newRowType, RexBuilder builder) {
+    super();
+    this.origRel = origRel;
+    this.origRowType = origRel.getRowType();
+    this.newRowType = newRowType;
+    this.builder = builder;
+    this.destExprMap = Maps.newHashMap();
+  }
+
+  /**
+   * Set the map of src expression to target expression, expressions not in the map do not have assigned destinations
+   * @param exprMap
+   * @return
+   */
+  public SimpleRexRemap setExpressionMap(Map<LogicalExpression, LogicalExpression>  exprMap) {
+    destExprMap.putAll(exprMap);
+    return this;
+  }
+
+  public RexNode rewriteEqualOnCharToLike(RexNode expr,
+                                          Map<RexNode, LogicalExpression> equalOnCastCharExprs) {
+    Map<RexNode, RexNode> srcToReplace = Maps.newIdentityHashMap();
+    for(Map.Entry<RexNode, LogicalExpression> entry: equalOnCastCharExprs.entrySet()) {
+      RexNode equalOp = entry.getKey();
+      LogicalExpression opInput = entry.getValue();
+
+      final List<RexNode> operands = ((RexCall)equalOp).getOperands();
+      RexLiteral newLiteral = null;
+      RexNode input = null;
+      if(operands.size() == 2 ) {
+        RexLiteral oplit = null;
+        if (operands.get(0) instanceof RexLiteral) {
+          oplit = (RexLiteral) operands.get(0);
+          if(oplit.getTypeName() == SqlTypeName.CHAR) {
+            newLiteral = builder.makeLiteral(((NlsString) oplit.getValue()).getValue() + "%");
+            input = operands.get(1);
+          }
+        }
+        else if (operands.get(1) instanceof RexLiteral) {
+          oplit = (RexLiteral) operands.get(1);
+          if(oplit.getTypeName() == SqlTypeName.CHAR) {
+            newLiteral = builder.makeLiteral(((NlsString) oplit.getValue()).getValue() + "%");
+            input = operands.get(0);
+          }
+        }
+      }
+      if(newLiteral != null) {
+        srcToReplace.put(equalOp, builder.makeCall(SqlStdOperatorTable.LIKE, input, newLiteral));
+      }
+    }
+    if (srcToReplace.size() > 0) {
+      RexReplace replacer = new RexReplace(srcToReplace);
+      RexNode resultRex = expr.accept(replacer);
+      return resultRex;
+    }
+    return expr;
+  }
+
+  /**
+   *
+   * @param srcRex  the source RexNode to be rewritten
+   * @param mapRexToExpr a map of rex->logical expression to guide what rex to rewrite
+   * @return the RexNode after rewriting
+   */
+  public RexNode rewriteWithMap(RexNode srcRex, Map<RexNode, LogicalExpression> mapRexToExpr) {
+    Map<RexNode, RexNode> destNodeMap = Maps.newHashMap();
+    for(Map.Entry<RexNode, LogicalExpression> entry: mapRexToExpr.entrySet()) {
+      LogicalExpression entryExpr = entry.getValue();
+
+      LogicalExpression destExpr = destExprMap.get(entryExpr);
+      //then build rexNode from the path
+      RexNode destRex = buildRexForField(destExpr==null?entryExpr : destExpr, newRowType);
+      destNodeMap.put(entry.getKey(), destRex);
+    }
+
+    //Visit through the nodes, if destExprMap has an entry to provide substitute to replace a rexNode, replace the rexNode
+    RexReplace replacer = new RexReplace(destNodeMap);
+    RexNode resultRex = srcRex.accept(replacer);
+    return resultRex;
+  }
+
+  public RexNode rewrite(RexNode expr) {
+    IndexableExprMarker marker = new IndexableExprMarker(origRel);
+    expr.accept(marker);
+    return rewriteWithMap(expr, marker.getIndexableExpression());
+  }
+
+  private RexNode buildRexForField(LogicalExpression expr, RelDataType newRowType) {
+    ExprToRex toRex = new ExprToRex(origRel, newRowType, builder);
+    return expr.accept(toRex, null);
+  }
+
+  public static String getFullPath(PathSegment pathSeg) {
+    PathSegment.NameSegment nameSeg = (PathSegment.NameSegment)pathSeg;
+    if(nameSeg.isLastPath()) {
+      return nameSeg.getPath();
+    }
+    return String.format("%s.%s",
+        nameSeg.getPath(),
+        getFullPath(nameSeg.getChild()));
+  }
+
+  private static PathSegment convertLiteral(RexLiteral literal) {
+    switch (literal.getType().getSqlTypeName()) {
+      case CHAR:
+        return new PathSegment.NameSegment(RexLiteral.stringValue(literal));
+      case INTEGER:
+        return new PathSegment.ArraySegment(RexLiteral.intValue(literal));
+      default:
+        return null;
+    }
+  }
+
+  /**
+   * This class go through the RexNode, collect all the fieldNames, mark starting positions(RexNode) of fields
+   * so this information can be used later e,.g. replaced with a substitute node
+   */
+  public static class FieldsMarker extends RexVisitorImpl<PathSegment> {
+    final List<String> fieldNames;
+    final List<RelDataTypeField> fields;
+    final Map<RexNode, String> desiredFields = Maps.newHashMap();
+
+    int stackDepth;
+
+    public FieldsMarker(RelDataType rowType) {
+      super(true);
+      this.fieldNames = rowType.getFieldNames();
+      this.fields = rowType.getFieldList();
+      this.stackDepth = 0;
+    }
+
+    private PathSegment newPath(PathSegment segment, RexNode node) {
+      if (stackDepth == 0) {
+        desiredFields.put(node, getFullPath(segment));
+      }
+      return segment;
+    }
+
+    private PathSegment newPath(String path, RexNode node) {
+      PathSegment segment = new PathSegment.NameSegment(path);
+      if (stackDepth == 0) {
+        desiredFields.put(node, getFullPath(segment));
+      }
+      return segment;
+    }
+
+    public Map<RexNode, String> getFieldAndPos() {
+      return ImmutableMap.copyOf(desiredFields);
+    }
+
+    @Override
+    public PathSegment visitInputRef(RexInputRef inputRef) {
+      int index = inputRef.getIndex();
+      String name = fieldNames.get(index);
+      return newPath(name, inputRef);
+    }
+
+    @Override
+    public PathSegment visitCall(RexCall call) {
+      if ("ITEM".equals(call.getOperator().getName())) {
+        stackDepth++;
+        PathSegment mapOrArray = call.operands.get(0).accept(this);
+        stackDepth--;
+        if (mapOrArray != null) {
+          if (call.operands.get(1) instanceof RexLiteral) {
+            PathSegment newFieldPath = newPath(
+                mapOrArray.cloneWithNewChild(convertLiteral((RexLiteral) call.operands.get(1))),
+                call);
+            return newFieldPath;
+          }
+          return mapOrArray;
+        }
+      } else {
+        for (RexNode operand : call.operands) {
+          operand.accept(this);
+        }
+      }
+      return null;
+    }
+  }
+
+  public static class RexReplace extends RexShuttle {
+
+    final Map<RexNode, RexNode> rexMap;
+
+    public RexReplace( Map<RexNode, RexNode> rexMap) {
+      this.rexMap = rexMap;
+    }
+    boolean toReplace(RexNode node) {
+      return rexMap.containsKey(node);
+    }
+
+    RexNode replace(RexNode node) {
+      return rexMap.get(node);
+    }
+
+    public RexNode visitOver(RexOver over) {
+      return toReplace(over) ? replace(over) : super.visitOver(over);
+    }
+
+    public RexNode visitCall(final RexCall call) {
+      return toReplace(call) ? replace(call) : super.visitCall(call);
+    }
+
+    public RexNode visitCorrelVariable(RexCorrelVariable variable) {
+      return variable;
+    }
+
+    public RexNode visitFieldAccess(RexFieldAccess fieldAccess) {
+      return toReplace(fieldAccess) ? replace(fieldAccess) : super.visitFieldAccess(fieldAccess);
+    }
+
+    public RexNode visitInputRef(RexInputRef inputRef) {
+      return toReplace(inputRef) ? replace(inputRef) : super.visitInputRef(inputRef);
+    }
+
+    public RexNode visitLocalRef(RexLocalRef localRef) {
+      return toReplace(localRef) ? replace(localRef) : super.visitLocalRef(localRef);
+    }
+
+    public RexNode visitLiteral(RexLiteral literal) {
+      return literal;
+    }
+
+    public RexNode visitDynamicParam(RexDynamicParam dynamicParam) {
+      return dynamicParam;
+    }
+
+    public RexNode visitRangeRef(RexRangeRef rangeRef) {
+      return toReplace(rangeRef) ? replace(rangeRef) : super.visitRangeRef(rangeRef);
+    }
+  }
+
+}
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/index/Statistics.java b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/index/Statistics.java
index 2859102..06d008c 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/index/Statistics.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/index/Statistics.java
@@ -54,8 +54,7 @@ public interface Statistics {
    *  @param scanRel - The current scan rel
    *  @return the leading rowcount
    */
-  double getLeadingRowCount(RexNode condition, String tabIdxName, RelNode scanRel);
-
+  public double getLeadingRowCount(RexNode condition, String tabIdxName, DrillScanRelBase scanRel);
   /** Returns the average row size for the specified filter condition
    * @param tabIdxName - The index name generated using {@code buildUniqueIndexIdentifier}
    * @param isIndexScan - Whether the current rel is an index scan (false for primary table)
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/index/generators/AbstractIndexPlanGenerator.java b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/index/generators/AbstractIndexPlanGenerator.java
new file mode 100644
index 0000000..36ff61f
--- /dev/null
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/index/generators/AbstractIndexPlanGenerator.java
@@ -0,0 +1,259 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.drill.exec.planner.index.generators;
+
+import java.util.ArrayList;
+import java.util.List;
+
+import org.apache.drill.shaded.guava.com.google.common.collect.ImmutableList;
+import org.apache.drill.shaded.guava.com.google.common.collect.Lists;
+
+import org.apache.calcite.plan.RelTrait;
+import org.apache.calcite.plan.RelTraitSet;
+import org.apache.calcite.rel.RelNode;
+import org.apache.calcite.rel.InvalidRelException;
+import org.apache.calcite.rel.RelCollation;
+import org.apache.calcite.rel.RelCollationTraitDef;
+import org.apache.calcite.rel.core.Sort;
+import org.apache.calcite.rel.type.RelDataTypeFieldImpl;
+import org.apache.calcite.rel.type.RelRecordType;
+import org.apache.calcite.sql.type.SqlTypeName;
+import org.apache.drill.common.expression.FieldReference;
+import org.apache.drill.common.expression.SchemaPath;
+import org.apache.drill.exec.physical.base.DbGroupScan;
+import org.apache.drill.exec.planner.index.IndexCallContext;
+import org.apache.drill.exec.planner.index.IndexPlanUtils;
+import org.apache.drill.exec.planner.logical.DrillFilterRel;
+import org.apache.drill.exec.planner.logical.DrillProjectRel;
+import org.apache.drill.exec.planner.logical.DrillSortRel;
+import org.apache.drill.exec.planner.common.DrillProjectRelBase;
+import org.apache.drill.exec.planner.common.DrillScanRelBase;
+import org.apache.drill.exec.planner.physical.PlannerSettings;
+import org.apache.drill.exec.planner.physical.Prel;
+import org.apache.drill.exec.planner.physical.SortPrel;
+import org.apache.drill.exec.planner.physical.HashToMergeExchangePrel;
+import org.apache.drill.exec.planner.physical.SingleMergeExchangePrel;
+import org.apache.drill.exec.planner.physical.PrelUtil;
+import org.apache.drill.exec.planner.physical.Prule;
+import org.apache.drill.exec.planner.physical.SubsetTransformer;
+import org.apache.drill.exec.planner.physical.DrillDistributionTrait;
+import org.apache.calcite.plan.RelOptCluster;
+import org.apache.calcite.rel.type.RelDataType;
+import org.apache.calcite.rel.type.RelDataTypeFactory;
+import org.apache.calcite.rel.type.RelDataTypeField;
+import org.apache.calcite.rex.RexBuilder;
+import org.apache.calcite.rex.RexNode;
+
+public abstract class AbstractIndexPlanGenerator extends SubsetTransformer<RelNode, InvalidRelException>{
+
+  static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(AbstractIndexPlanGenerator.class);
+
+  final protected DrillProjectRelBase origProject;
+  final protected DrillScanRelBase origScan;
+  final protected DrillProjectRelBase upperProject;
+  final protected RelNode origSort;
+
+  final protected RexNode indexCondition;
+  final protected RexNode remainderCondition;
+  final protected RexBuilder builder;
+  final protected IndexCallContext indexContext;
+  final protected PlannerSettings settings;
+
+  public AbstractIndexPlanGenerator(IndexCallContext indexContext,
+      RexNode indexCondition,
+      RexNode remainderCondition,
+      RexBuilder builder,
+      PlannerSettings settings) {
+    super(indexContext.getCall());
+    this.origProject = indexContext.getLowerProject();
+    this.origScan = indexContext.getScan();
+    this.upperProject = indexContext.getUpperProject();
+    this.origSort = indexContext.getSort();
+    this.indexCondition = indexCondition;
+    this.remainderCondition = remainderCondition;
+    this.indexContext = indexContext;
+    this.builder = builder;
+    this.settings = settings;
+  }
+
+  //This class provides the utility functions that don't rely on index(one or multiple) or final plan (covering or not),
+  //but those helper functions that focus on serving building index plan (project-filter-indexscan)
+
+  public static int getRowKeyIndex(RelDataType rowType, DrillScanRelBase origScan) {
+    List<String> fieldNames = rowType.getFieldNames();
+    int idx = 0;
+    for (String field : fieldNames) {
+      if (field.equalsIgnoreCase(((DbGroupScan)IndexPlanUtils.getGroupScan(origScan)).getRowKeyName())) {
+        return idx;
+      }
+      idx++;
+    }
+    return -1;
+  }
+
+  protected RelDataType convertRowType(RelDataType origRowType, RelDataTypeFactory typeFactory) {
+    if ( getRowKeyIndex(origRowType, origScan)>=0 ) { // row key already present
+      return origRowType;
+    }
+    List<RelDataTypeField> fields = new ArrayList<>();
+
+    fields.addAll(origRowType.getFieldList());
+    fields.add(new RelDataTypeFieldImpl(
+        ((DbGroupScan)IndexPlanUtils.getGroupScan(origScan)).getRowKeyName(), fields.size(),
+            typeFactory.createSqlType(SqlTypeName.ANY)));
+    return new RelRecordType(fields);
+  }
+
+  protected boolean checkRowKey(List<SchemaPath> columns) {
+    for (SchemaPath s : columns) {
+      if (s.equals(((DbGroupScan)IndexPlanUtils.getGroupScan(origScan)).getRowKeyPath())) {
+        return true;
+      }
+    }
+    return false;
+  }
+
+  // Range distribute the right side of the join, on row keys using a range partitioning function
+  protected RelNode createRangeDistRight(final RelNode rightPrel,
+                                         final RelDataTypeField rightRowKeyField,
+                                         final DbGroupScan origDbGroupScan) {
+
+    List<DrillDistributionTrait.DistributionField> rangeDistFields =
+        Lists.newArrayList(new DrillDistributionTrait.DistributionField(0 /* rowkey ordinal on the right side */));
+
+    FieldReference rangeDistRef = FieldReference.getWithQuotedRef(rightRowKeyField.getName());
+    List<FieldReference> rangeDistRefList = Lists.newArrayList();
+    rangeDistRefList.add(rangeDistRef);
+
+    final DrillDistributionTrait distRight;
+    if (IndexPlanUtils.scanIsPartition(origDbGroupScan)) {
+      distRight = new DrillDistributionTrait(
+          DrillDistributionTrait.DistributionType.RANGE_DISTRIBUTED,
+          ImmutableList.copyOf(rangeDistFields),
+          origDbGroupScan.getRangePartitionFunction(rangeDistRefList));
+    }
+    else {
+      distRight = DrillDistributionTrait.SINGLETON;
+    }
+
+    RelTraitSet rightTraits = newTraitSet(distRight).plus(Prel.DRILL_PHYSICAL);
+    RelNode convertedRight = Prule.convert(rightPrel, rightTraits);
+
+    return convertedRight;
+  }
+
+  public RelTraitSet newTraitSet(RelTrait... traits) {
+    RelTraitSet set = indexContext.getCall().getPlanner().emptyTraitSet();
+    for (RelTrait t : traits) {
+      if(t != null) {
+        set = set.plus(t);
+      }
+    }
+    return set;
+  }
+
+  protected static boolean toRemoveSort(Sort sort, RelCollation inputCollation) {
+    if ( (inputCollation != null) && inputCollation.satisfies(IndexPlanUtils.getCollation(sort))) {
+      return true;
+    }
+    return false;
+  }
+
+  public static RelNode getExchange(RelOptCluster cluster, boolean isSingleton, boolean isExchangeRequired,
+                                    RelTraitSet traits, DrillDistributionTrait distributionTrait,
+                                    IndexCallContext indexContext, RelNode input) {
+    if (!isExchangeRequired) {
+      return input;
+    }
+
+    if (isSingleton) {
+      return new SingleMergeExchangePrel(cluster,
+              traits.replace(DrillDistributionTrait.SINGLETON),
+              input, indexContext.getCollation());
+    } else {
+      return new HashToMergeExchangePrel(cluster,
+              traits.replace(distributionTrait),
+              input, distributionTrait.getFields(), indexContext.getCollation(),
+              PrelUtil.getSettings(cluster).numEndPoints());
+    }
+  }
+
+  public static RelNode getSortNode(IndexCallContext indexContext, RelNode newRel, boolean donotGenerateSort,
+                                    boolean isSingleton, boolean isExchangeRequired) {
+    Sort rel = indexContext.getSort();
+    DrillDistributionTrait hashDistribution =
+        new DrillDistributionTrait(DrillDistributionTrait.DistributionType.HASH_DISTRIBUTED,
+            ImmutableList.copyOf(indexContext.getDistributionFields()));
+
+    if ( toRemoveSort(indexContext.getSort(), newRel.getTraitSet().getTrait(RelCollationTraitDef.INSTANCE))) {
+      //we are going to remove sort
+      logger.debug("Not generating SortPrel since we have the required collation");
+
+      RelTraitSet traits = newRel.getTraitSet().plus(IndexPlanUtils.getCollation(rel)).plus(Prel.DRILL_PHYSICAL);
+      newRel = Prule.convert(newRel, traits);
+      newRel = getExchange(newRel.getCluster(), isSingleton, isExchangeRequired,
+                                 traits, hashDistribution, indexContext, newRel);
+    }
+    else {
+      if (donotGenerateSort) {
+        logger.debug("Not generating SortPrel and index plan, since just picking index for full index scan is not beneficial.");
+        return null;
+      }
+      RelTraitSet traits = newRel.getTraitSet().plus(IndexPlanUtils.getCollation(rel)).plus(Prel.DRILL_PHYSICAL);
+      newRel = new SortPrel(rel.getCluster(),
+              newRel.getTraitSet().replace(Prel.DRILL_PHYSICAL).plus(IndexPlanUtils.getCollation(rel)),
+          Prule.convert(newRel, newRel.getTraitSet().replace(Prel.DRILL_PHYSICAL)), IndexPlanUtils.getCollation(rel));
+      newRel = getExchange(newRel.getCluster(), isSingleton, isExchangeRequired,
+                                 traits, hashDistribution, indexContext, newRel);
+    }
+    return newRel;
+  }
+
+  public abstract RelNode convertChild(RelNode current, RelNode child) throws InvalidRelException;
+
+  public boolean forceConvert(){
+    return true;
+  }
+
+  public void go() throws InvalidRelException {
+    RelNode top = indexContext.getCall().rel(0);
+    final RelNode input;
+    if (top instanceof DrillProjectRel) {
+      DrillProjectRel topProject = (DrillProjectRel) top;
+      input = topProject.getInput();
+    }
+    else if (top instanceof DrillFilterRel) {
+      DrillFilterRel topFilter = (DrillFilterRel)top;
+      input = topFilter.getInput();
+    } else if (top instanceof DrillSortRel) {
+      DrillSortRel topSort = (DrillSortRel)top;
+      input = topSort.getInput();
+    }
+    else if ( top instanceof DrillSortRel) {
+      DrillSortRel topSort = (DrillSortRel) top;
+      input = topSort.getInput();
+    }
+    else {
+      return;
+    }
+    RelTraitSet traits = input.getTraitSet().plus(Prel.DRILL_PHYSICAL);
+    RelNode convertedInput = Prule.convert(input, traits);
+    this.go(top, convertedInput);
+  }
+}
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/index/generators/CoveringIndexPlanGenerator.java b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/index/generators/CoveringIndexPlanGenerator.java
new file mode 100644
index 0000000..87dd77c
--- /dev/null
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/index/generators/CoveringIndexPlanGenerator.java
@@ -0,0 +1,208 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.drill.exec.planner.index.generators;
+
+import org.apache.drill.shaded.guava.com.google.common.base.Preconditions;
+import org.apache.drill.shaded.guava.com.google.common.collect.Lists;
+
+import org.apache.calcite.plan.RelTraitSet;
+import org.apache.calcite.rel.RelCollation;
+import org.apache.calcite.rel.type.RelDataType;
+import org.apache.calcite.rex.RexBuilder;
+import org.apache.calcite.rex.RexCall;
+import org.apache.calcite.rex.RexInputRef;
+import org.apache.calcite.rex.RexNode;
+import org.apache.drill.exec.physical.base.IndexGroupScan;
+import org.apache.drill.exec.planner.index.IndexLogicalPlanCallContext;
+import org.apache.drill.exec.planner.index.IndexDescriptor;
+import org.apache.drill.exec.planner.index.FunctionalIndexInfo;
+import org.apache.drill.exec.planner.index.FunctionalIndexHelper;
+import org.apache.drill.exec.planner.index.IndexPlanUtils;
+import org.apache.drill.exec.planner.index.SimpleRexRemap;
+import org.apache.drill.exec.planner.logical.DrillMergeProjectRule;
+import org.apache.drill.exec.planner.logical.DrillParseContext;
+import org.apache.drill.exec.planner.physical.FilterPrel;
+import org.apache.drill.exec.planner.physical.PlannerSettings;
+import org.apache.drill.exec.planner.physical.Prel;
+import org.apache.drill.exec.planner.physical.PrelUtil;
+import org.apache.drill.exec.planner.physical.ProjectPrel;
+import org.apache.drill.exec.planner.physical.Prule;
+import org.apache.drill.exec.planner.physical.ScanPrel;
+import org.apache.calcite.rel.InvalidRelException;
+
+import org.apache.calcite.rel.RelNode;
+
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+
+/**
+ * Generate a covering index plan that is equivalent to the original plan.
+ *
+ * This plan will be further optimized by the filter pushdown rule of the Index plugin which should
+ * push this filter into the index scan.
+ */
+public class CoveringIndexPlanGenerator extends AbstractIndexPlanGenerator {
+
+  static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(CoveringIndexPlanGenerator.class);
+  final protected IndexGroupScan indexGroupScan;
+  final protected IndexDescriptor indexDesc;
+
+  // Ideally This functionInfo should be cached along with indexDesc.
+  final protected FunctionalIndexInfo functionInfo;
+
+  public CoveringIndexPlanGenerator(IndexLogicalPlanCallContext indexContext,
+                                    FunctionalIndexInfo functionInfo,
+                                    IndexGroupScan indexGroupScan,
+                                    RexNode indexCondition,
+                                    RexNode remainderCondition,
+                                    RexBuilder builder,
+                                    PlannerSettings settings) {
+    super(indexContext, indexCondition, remainderCondition, builder, settings);
+    this.indexGroupScan = indexGroupScan;
+    this.functionInfo = functionInfo;
+    this.indexDesc = functionInfo.getIndexDesc();
+  }
+
+  /**
+   *
+   * @param inputIndex
+   * @param functionInfo functional index information that may impact rewrite
+   * @return
+   */
+  private RexNode rewriteFunctionalCondition(RexNode inputIndex, RelDataType newRowType,
+                                             FunctionalIndexInfo functionInfo) {
+    if (!functionInfo.hasFunctional()) {
+      return inputIndex;
+    }
+    return FunctionalIndexHelper.convertConditionForIndexScan(inputIndex,
+        origScan, newRowType, builder, functionInfo);
+  }
+
+  @Override
+  public RelNode convertChild(final RelNode filter, final RelNode input) throws InvalidRelException {
+
+    if (indexGroupScan == null) {
+      logger.error("Null indexgroupScan in CoveringIndexPlanGenerator.convertChild");
+      return null;
+    }
+
+    RexNode coveringCondition;
+    ScanPrel indexScanPrel =
+        IndexPlanUtils.buildCoveringIndexScan(origScan, indexGroupScan, indexContext, indexDesc);
+
+    // If remainder condition, then combine the index and remainder conditions. This is a covering plan so we can
+    // pushed the entire condition into the index.
+    coveringCondition = IndexPlanUtils.getTotalFilter(indexCondition, remainderCondition, indexScanPrel.getCluster().getRexBuilder());
+    RexNode newIndexCondition =
+        rewriteFunctionalCondition(coveringCondition, indexScanPrel.getRowType(), functionInfo);
+
+    // build collation for filter
+    RelTraitSet indexFilterTraitSet = indexScanPrel.getTraitSet();
+
+    FilterPrel indexFilterPrel = new FilterPrel(indexScanPrel.getCluster(), indexFilterTraitSet,
+        indexScanPrel, newIndexCondition);
+
+    ProjectPrel indexProjectPrel = null;
+    if (origProject != null) {
+      RelCollation collation = IndexPlanUtils.buildCollationProject(IndexPlanUtils.getProjects(origProject), null,
+          origScan, functionInfo, indexContext);
+      indexProjectPrel = new ProjectPrel(origScan.getCluster(), indexFilterTraitSet.plus(collation),
+          indexFilterPrel, IndexPlanUtils.getProjects(origProject), origProject.getRowType());
+    }
+
+    RelNode finalRel;
+    if (indexProjectPrel != null) {
+      finalRel = indexProjectPrel;
+    } else {
+      finalRel = indexFilterPrel;
+    }
+
+    if (upperProject != null) {
+      RelCollation newCollation =
+          IndexPlanUtils.buildCollationProject(IndexPlanUtils.getProjects(upperProject), origProject,
+              origScan, functionInfo, indexContext);
+
+      ProjectPrel cap = new ProjectPrel(upperProject.getCluster(),
+          newCollation==null?finalRel.getTraitSet() : finalRel.getTraitSet().plus(newCollation),
+          finalRel, IndexPlanUtils.getProjects(upperProject), upperProject.getRowType());
+
+      if (functionInfo.hasFunctional()) {
+        //if there is functional index field, then a rewrite may be needed in upperProject/indexProject
+        //merge upperProject with indexProjectPrel(from origProject) if both exist,
+        ProjectPrel newProject = cap;
+        if (indexProjectPrel != null) {
+          newProject = (ProjectPrel) DrillMergeProjectRule.replace(newProject, indexProjectPrel);
+        }
+        // then rewrite functional expressions in new project.
+        List<RexNode> newProjects = Lists.newArrayList();
+        DrillParseContext parseContxt = new DrillParseContext(PrelUtil.getPlannerSettings(newProject.getCluster()));
+        for(RexNode projectRex: newProject.getProjects()) {
+          RexNode newRex = IndexPlanUtils.rewriteFunctionalRex(indexContext, parseContxt, null, origScan, projectRex, indexScanPrel.getRowType(), functionInfo);
+          newProjects.add(newRex);
+        }
+
+        ProjectPrel rewrittenProject = new ProjectPrel(newProject.getCluster(),
+            newCollation==null? newProject.getTraitSet() : newProject.getTraitSet().plus(newCollation),
+            indexFilterPrel, newProjects, newProject.getRowType());
+
+        cap = rewrittenProject;
+      }
+
+      finalRel = cap;
+    }
+
+    if (indexContext.getSort() != null) {
+      finalRel = getSortNode(indexContext, finalRel, false,true, true);
+      Preconditions.checkArgument(finalRel != null);
+    }
+
+    finalRel = Prule.convert(finalRel, finalRel.getTraitSet().plus(Prel.DRILL_PHYSICAL));
+
+    logger.debug("CoveringIndexPlanGenerator got finalRel {} from origScan {}, original digest {}, new digest {}.",
+        finalRel.toString(), origScan.toString(),
+        upperProject==null?indexContext.getFilter().getDigest(): upperProject.getDigest(), finalRel.getDigest());
+    return finalRel;
+  }
+
+  private RexNode rewriteConditionForProject(RexNode condition, List<RexNode> projects) {
+    Map<RexNode, RexNode> mapping = new HashMap<>();
+    rewriteConditionForProjectInternal(condition, projects, mapping);
+    SimpleRexRemap.RexReplace replacer = new SimpleRexRemap.RexReplace(mapping);
+    return condition.accept(replacer);
+  }
+
+  private void rewriteConditionForProjectInternal(RexNode condition, List<RexNode> projects, Map<RexNode, RexNode> mapping) {
+    if (condition instanceof RexCall) {
+      if ("ITEM".equals(((RexCall) condition).getOperator().getName().toUpperCase())) {
+        int index = 0;
+        for (RexNode project : projects) {
+          if (project.toString().equals(condition.toString())) {
+            // Map it to the corresponding RexInputRef for the project
+            mapping.put(condition, new RexInputRef(index, project.getType()));
+          }
+          ++index;
+        }
+      } else {
+        for (RexNode child : ((RexCall) condition).getOperands()) {
+          rewriteConditionForProjectInternal(child, projects, mapping);
+        }
+      }
+    }
+  }
+}
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/index/generators/CoveringPlanNoFilterGenerator.java b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/index/generators/CoveringPlanNoFilterGenerator.java
new file mode 100644
index 0000000..163aef9
--- /dev/null
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/index/generators/CoveringPlanNoFilterGenerator.java
@@ -0,0 +1,125 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.drill.exec.planner.index.generators;
+
+import org.apache.drill.shaded.guava.com.google.common.collect.Lists;
+import org.apache.calcite.plan.RelTraitSet;
+import org.apache.calcite.rel.InvalidRelException;
+import org.apache.calcite.rel.RelCollation;
+import org.apache.calcite.rel.RelNode;
+import org.apache.calcite.rex.RexNode;
+import org.apache.drill.exec.physical.base.IndexGroupScan;
+import org.apache.drill.exec.planner.index.IndexCallContext;
+import org.apache.drill.exec.planner.index.IndexDescriptor;
+import org.apache.drill.exec.planner.index.FunctionalIndexInfo;
+import org.apache.drill.exec.planner.index.IndexPlanUtils;
+import org.apache.drill.exec.planner.logical.DrillParseContext;
+import org.apache.drill.exec.planner.physical.PlannerSettings;
+import org.apache.drill.exec.planner.physical.ScanPrel;
+import org.apache.drill.exec.planner.physical.ProjectPrel;
+import org.apache.drill.exec.planner.physical.Prel;
+import org.apache.drill.exec.planner.physical.PrelUtil;
+import org.apache.drill.exec.planner.physical.Prule;
+import org.apache.drill.exec.physical.base.DbGroupScan;
+import java.util.List;
+
+public class CoveringPlanNoFilterGenerator extends AbstractIndexPlanGenerator {
+  static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(CoveringIndexPlanGenerator.class);
+  final protected IndexGroupScan indexGroupScan;
+  final protected IndexDescriptor indexDesc;
+  final boolean isSingletonSortedStream;
+  // Ideally This functionInfo should be cached along with indexDesc.
+  final protected FunctionalIndexInfo functionInfo;
+
+  public CoveringPlanNoFilterGenerator(IndexCallContext indexContext,
+                                       FunctionalIndexInfo functionInfo,
+                                       boolean isSingleton,
+                                       PlannerSettings settings) {
+    super(indexContext, null, null, null, settings);
+    this.functionInfo = functionInfo;
+    this.indexDesc = functionInfo == null ? null : functionInfo.getIndexDesc();
+    this.indexGroupScan = functionInfo == null ? null : functionInfo.getIndexDesc().getIndexGroupScan();
+    this.isSingletonSortedStream = isSingleton;
+  }
+
+  public RelNode convertChild(final RelNode filter, final RelNode input) throws InvalidRelException {
+      return this.convertChild();
+  }
+
+  public RelNode convertChild() throws InvalidRelException {
+
+    if (indexGroupScan == null) {
+      logger.error("Null indexgroupScan in CoveringIndexPlanGenerator.convertChild");
+      return null;
+    }
+    //update sort expressions in context
+    IndexPlanUtils.updateSortExpression(indexContext, indexContext.getSort() != null ?
+            indexContext.getCollation().getFieldCollations() : null);
+
+    ScanPrel indexScanPrel =
+            IndexPlanUtils.buildCoveringIndexScan(origScan, indexGroupScan, indexContext, indexDesc);
+    ((IndexGroupScan)indexScanPrel.getGroupScan()).setStatistics(((DbGroupScan)IndexPlanUtils.getGroupScan(origScan)).getStatistics());
+    RelTraitSet indexScanTraitSet = indexScanPrel.getTraitSet();
+
+    RelNode finalRel = indexScanPrel;
+    if (indexContext.getLowerProject() != null) {
+
+      RelCollation collation = IndexPlanUtils.buildCollationProject(indexContext.getLowerProject().getProjects(), null,
+              indexContext.getScan(), functionInfo, indexContext);
+      finalRel = new ProjectPrel(indexContext.getScan().getCluster(), indexScanTraitSet.plus(collation),
+              indexScanPrel, indexContext.getLowerProject().getProjects(), indexContext.getLowerProject().getRowType());
+
+      if (functionInfo.hasFunctional()) {
+        //if there is functional index field, then a rewrite may be needed in upperProject/indexProject
+        //merge upperProject with indexProjectPrel(from origProject) if both exist,
+        ProjectPrel newProject = (ProjectPrel)finalRel;
+
+        // then rewrite functional expressions in new project.
+        List<RexNode> newProjects = Lists.newArrayList();
+        DrillParseContext parseContxt = new DrillParseContext(PrelUtil.getPlannerSettings(newProject.getCluster()));
+        for(RexNode projectRex: newProject.getProjects()) {
+          RexNode newRex = IndexPlanUtils.rewriteFunctionalRex(indexContext, parseContxt, null, origScan, projectRex,
+                  indexScanPrel.getRowType(), functionInfo);
+          newProjects.add(newRex);
+        }
+
+        ProjectPrel rewrittenProject = new ProjectPrel(newProject.getCluster(),
+                collation==null? newProject.getTraitSet() : newProject.getTraitSet().plus(collation),
+                indexScanPrel, newProjects, newProject.getRowType());
+
+        finalRel = rewrittenProject;
+      }
+    }
+
+    if (indexContext.getSort() != null) {
+      finalRel = getSortNode(indexContext, finalRel, true, isSingletonSortedStream, indexContext.getExchange() != null);
+      if (finalRel == null) {
+        return null;
+      }
+    }
+
+    finalRel = Prule.convert(finalRel, finalRel.getTraitSet().plus(Prel.DRILL_PHYSICAL));
+
+    logger.debug("CoveringPlanNoFilterGenerator got finalRel {} from origScan {}, original digest {}, new digest {}.",
+            finalRel.toString(), indexContext.getScan().toString(),
+            indexContext.getLowerProject()!=null?indexContext.getLowerProject().getDigest(): indexContext.getScan().getDigest(),
+            finalRel.getDigest());
+    return finalRel;
+  }
+}
+
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/index/generators/IndexIntersectPlanGenerator.java b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/index/generators/IndexIntersectPlanGenerator.java
new file mode 100644
index 0000000..f2091f6
--- /dev/null
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/index/generators/IndexIntersectPlanGenerator.java
@@ -0,0 +1,338 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.drill.exec.planner.index.generators;
+
+import org.apache.drill.shaded.guava.com.google.common.collect.ImmutableList;
+import org.apache.drill.shaded.guava.com.google.common.collect.Lists;
+import org.apache.drill.shaded.guava.com.google.common.collect.Maps;
+
+import org.apache.calcite.plan.RelOptUtil;
+import org.apache.calcite.plan.RelTraitSet;
+import org.apache.calcite.rel.InvalidRelException;
+import org.apache.calcite.rel.RelNode;
+import org.apache.calcite.rel.core.JoinRelType;
+import org.apache.calcite.rel.type.RelDataType;
+import org.apache.calcite.rel.type.RelDataTypeFactory;
+import org.apache.calcite.rel.type.RelDataTypeField;
+import org.apache.calcite.rex.RexBuilder;
+import org.apache.calcite.rex.RexInputRef;
+import org.apache.calcite.rex.RexNode;
+import org.apache.calcite.util.Pair;
+import org.apache.drill.common.expression.SchemaPath;
+import org.apache.drill.exec.physical.base.DbGroupScan;
+import org.apache.drill.exec.physical.base.IndexGroupScan;
+import org.apache.drill.exec.planner.common.JoinControl;
+import org.apache.drill.exec.planner.index.IndexLogicalPlanCallContext;
+import org.apache.drill.exec.planner.index.IndexDescriptor;
+import org.apache.drill.exec.planner.index.FunctionalIndexInfo;
+import org.apache.drill.exec.planner.index.FunctionalIndexHelper;
+import org.apache.drill.exec.planner.index.IndexPlanUtils;
+import org.apache.drill.exec.planner.index.IndexConditionInfo;
+import org.apache.drill.exec.planner.physical.DrillDistributionTrait;
+import org.apache.drill.exec.planner.physical.DrillDistributionTraitDef;
+import org.apache.drill.exec.planner.physical.DrillDistributionTrait.DistributionType;
+import org.apache.drill.exec.planner.physical.FilterPrel;
+import org.apache.drill.exec.planner.physical.HashJoinPrel;
+import org.apache.drill.exec.planner.physical.PlannerSettings;
+import org.apache.drill.exec.planner.physical.Prel;
+import org.apache.drill.exec.planner.physical.ProjectPrel;
+import org.apache.drill.exec.planner.physical.Prule;
+import org.apache.drill.exec.planner.physical.RowKeyJoinPrel;
+import org.apache.drill.exec.planner.physical.ScanPrel;
+
+import java.util.ArrayList;
+import java.util.List;
+import java.util.Map;
+
+/**
+ * IndexScanIntersectGenerator is to generate index plan against multiple index tables,
+ * the input indexes are assumed to be ranked by selectivity(low to high) already.
+ */
+public class IndexIntersectPlanGenerator extends AbstractIndexPlanGenerator {
+
+  static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(IndexIntersectPlanGenerator.class);
+
+  final Map<IndexDescriptor, IndexConditionInfo> indexInfoMap;
+
+  public IndexIntersectPlanGenerator(IndexLogicalPlanCallContext indexContext,
+                                     Map<IndexDescriptor, IndexConditionInfo> indexInfoMap,
+                                     RexBuilder builder,
+                                     PlannerSettings settings) {
+    super(indexContext, null, null, builder, settings);
+    this.indexInfoMap = indexInfoMap;
+  }
+
+  public RelNode buildRowKeyJoin(RelNode left, RelNode right, boolean isRowKeyJoin, int htControl)
+      throws InvalidRelException {
+    final int leftRowKeyIdx = getRowKeyIndex(left.getRowType(), origScan);
+    final int rightRowKeyIdx = 0; // only rowkey field is being projected from right side
+
+    assert leftRowKeyIdx >= 0;
+
+    List<Integer> leftJoinKeys = ImmutableList.of(leftRowKeyIdx);
+    List<Integer> rightJoinKeys = ImmutableList.of(rightRowKeyIdx);
+
+    logger.trace(String.format(
+        "buildRowKeyJoin: leftIdx: %d, rightIdx: %d",
+        leftRowKeyIdx, rightRowKeyIdx));
+    RexNode joinCondition =
+        RelOptUtil.createEquiJoinCondition(left, leftJoinKeys,
+            right, rightJoinKeys, builder);
+
+    if (isRowKeyJoin == true) {
+      RelNode newRel;
+      if (settings.isIndexUseHashJoinNonCovering()) {
+        HashJoinPrel hjPrel = new HashJoinPrel(left.getCluster(), left.getTraitSet(), left,
+            right, joinCondition, JoinRelType.INNER, false /* no swap */, null /* no runtime filter */,
+            isRowKeyJoin, htControl);
+        newRel = hjPrel;
+      } else {
+        RowKeyJoinPrel rjPrel = new RowKeyJoinPrel(left.getCluster(), left.getTraitSet(),
+            left, right, joinCondition, JoinRelType.INNER);
+        newRel = rjPrel;
+      }
+      // since there is a restricted Scan on left side, assume original project
+      return buildOriginalProject(newRel);
+    } else {
+      //there is no restricted scan on left, do a regular rowkey join
+      HashJoinPrel hjPrel = new HashJoinPrel(left.getCluster(), left.getTraitSet(), left,
+          right, joinCondition, JoinRelType.INNER, false /* no swap */, null /* no runtime filter */,
+          isRowKeyJoin, htControl);
+      return buildRowKeyProject(hjPrel, leftRowKeyIdx);
+    }
+  }
+
+  public RelNode buildRowKeyProject(RelNode inputRel, int fieldIndex) {
+    List<RelDataTypeField> inputFields = inputRel.getRowType().getFieldList();
+    final RelDataTypeField rowKeyField = inputFields.get(fieldIndex);
+      RexNode expr = builder.makeInputRef(rowKeyField.getType(), rowKeyField.getIndex());
+    List<RexNode> exprs = Lists.newArrayList();
+    exprs.add(expr);
+
+    final RelDataTypeFactory.FieldInfoBuilder rightFieldTypeBuilder =
+        inputRel.getCluster().getTypeFactory().builder();
+
+    rightFieldTypeBuilder.add(rowKeyField);
+    final RelDataType projectRowType = rightFieldTypeBuilder.build();
+
+    ProjectPrel proj = new ProjectPrel(inputRel.getCluster(), inputRel.getTraitSet(), inputRel, exprs, projectRowType);
+
+    return proj;
+  }
+
+  public RelNode buildOriginalProject (RelNode newRel) {
+    RelDataType origRowType = origProject == null ? origScan.getRowType() : origProject.getRowType();
+
+    final RelDataTypeFactory.FieldInfoBuilder finalFieldTypeBuilder =
+        origScan.getCluster().getTypeFactory().builder();
+
+    List<RelDataTypeField> hjRowFields = newRel.getRowType().getFieldList();
+    int toRemoveRowKeyCount = 1;
+    if (getRowKeyIndex(origRowType, origScan)  < 0 ) {
+      toRemoveRowKeyCount = 2;
+    }
+    finalFieldTypeBuilder.addAll(hjRowFields.subList(0, hjRowFields.size()-toRemoveRowKeyCount));
+    final RelDataType finalProjectRowType = finalFieldTypeBuilder.build();
+
+    List<RexNode> resetExprs = Lists.newArrayList();
+    for (int idx=0; idx<hjRowFields.size()-toRemoveRowKeyCount; ++idx) {
+      resetExprs.add(RexInputRef.of(idx, newRel.getRowType()));
+    }
+
+    final ProjectPrel resetProjectPrel = new ProjectPrel(newRel.getCluster(), newRel.getTraitSet(),
+        newRel, resetExprs, finalProjectRowType);
+    newRel = resetProjectPrel;
+
+    RelNode finalRel = Prule.convert(newRel, newRel.getTraitSet());
+    return finalRel;
+  }
+
+  private FunctionalIndexInfo getFunctionalIndexInfo(IndexDescriptor index) {
+    return index.getFunctionalInfo();
+  }
+
+  public RelNode buildIntersectPlan(Map.Entry<IndexDescriptor, RexNode> pair, RelNode right,
+      boolean generateDistribution) throws InvalidRelException {
+    IndexDescriptor index = pair.getKey();
+    RexNode condition = pair.getValue();
+
+    FunctionalIndexInfo functionInfo = getFunctionalIndexInfo(index);
+    IndexGroupScan indexScan = index.getIndexGroupScan();
+    RelDataType indexScanRowType = FunctionalIndexHelper.convertRowTypeForIndexScan(
+        origScan, indexContext.getOrigMarker(), indexScan, functionInfo);
+    DrillDistributionTrait partition = IndexPlanUtils.scanIsPartition(IndexPlanUtils.getGroupScan(origScan))?
+        DrillDistributionTrait.RANDOM_DISTRIBUTED : DrillDistributionTrait.SINGLETON;
+
+    ScanPrel indexScanPrel = new ScanPrel(origScan.getCluster(),
+        origScan.getTraitSet().plus(Prel.DRILL_PHYSICAL).plus(partition), indexScan, indexScanRowType, origScan.getTable());
+    FilterPrel  indexFilterPrel = new FilterPrel(indexScanPrel.getCluster(), indexScanPrel.getTraitSet(),
+        indexScanPrel, FunctionalIndexHelper.convertConditionForIndexScan(condition, origScan,
+        indexScanRowType, builder, functionInfo));
+    // project the rowkey column from the index scan
+    List<RexNode> indexProjectExprs = Lists.newArrayList();
+    int rowKeyIndex = getRowKeyIndex(indexScanPrel.getRowType(), origScan);//indexGroupScan.getRowKeyOrdinal();
+    assert rowKeyIndex >= 0;
+
+    indexProjectExprs.add(RexInputRef.of(rowKeyIndex, indexScanPrel.getRowType()));
+
+    final RelDataTypeFactory.FieldInfoBuilder rightFieldTypeBuilder =
+        indexScanPrel.getCluster().getTypeFactory().builder();
+
+    // build the row type for the right Project
+    final List<RelDataTypeField> indexScanFields = indexScanPrel.getRowType().getFieldList();
+
+    final RelDataTypeField rightRowKeyField = indexScanFields.get(rowKeyIndex);
+    rightFieldTypeBuilder.add(rightRowKeyField);
+    final RelDataType indexProjectRowType = rightFieldTypeBuilder.build();
+
+    final ProjectPrel indexProjectPrel = new ProjectPrel(indexScanPrel.getCluster(), indexScanPrel.getTraitSet(),
+        indexFilterPrel, indexProjectExprs, indexProjectRowType);
+
+    RelTraitSet rightSideTraits = newTraitSet().plus(Prel.DRILL_PHYSICAL);
+    //if build(right) side does not exist, this index scan is the right most.
+    if (right == null) {
+      if (partition == DrillDistributionTrait.RANDOM_DISTRIBUTED &&
+          settings.getSliceTarget() < indexProjectPrel.getRows()) {
+        final DrillDistributionTrait distRight =
+            new DrillDistributionTrait(DistributionType.BROADCAST_DISTRIBUTED);
+        rightSideTraits = newTraitSet(distRight).plus(Prel.DRILL_PHYSICAL);
+      }
+    }
+
+    RelNode converted = Prule.convert(indexProjectPrel, rightSideTraits);
+
+    if (right == null) {
+      return converted;
+    }
+
+    //if build(right) side exist, the plan we got in 'converted' is left (probe). Intersect with right(build) side
+    RelNode finalRel = buildRowKeyJoin(converted, right, false, JoinControl.INTERSECT_DISTINCT);
+
+    if (generateDistribution &&
+        right.getTraitSet().getTrait(DrillDistributionTraitDef.INSTANCE) != DrillDistributionTrait.SINGLETON) {
+      final DrillDistributionTrait distRight =
+          new DrillDistributionTrait(DistributionType.BROADCAST_DISTRIBUTED);
+      rightSideTraits = newTraitSet(distRight).plus(Prel.DRILL_PHYSICAL);
+      // This join will serve as the right side for the next intersection join, if applicable
+      finalRel = Prule.convert(finalRel, rightSideTraits);
+    }
+
+    logger.trace("IndexIntersectPlanGenerator got finalRel {} from origScan {}",
+        finalRel.toString(), origScan.toString());
+    return finalRel;
+  }
+
+  private Pair<RelNode, DbGroupScan> buildRestrictedDBScan(RexNode remnant) {
+
+    DbGroupScan origDbGroupScan = (DbGroupScan)IndexPlanUtils.getGroupScan(origScan);
+    List<SchemaPath> cols = new ArrayList<SchemaPath>(origDbGroupScan.getColumns());
+    if (!checkRowKey(cols)) {
+      cols.add(origDbGroupScan.getRowKeyPath());
+    }
+
+    // Create a restricted groupscan from the primary table's groupscan
+    DbGroupScan restrictedGroupScan  = origDbGroupScan.getRestrictedScan(cols);
+    if (restrictedGroupScan == null) {
+      logger.error("Null restricted groupscan in IndexIntersectPlanGenerator.convertChild");
+      return null;
+    }
+    DrillDistributionTrait partition = IndexPlanUtils.scanIsPartition(IndexPlanUtils.getGroupScan(origScan))?
+        DrillDistributionTrait.RANDOM_DISTRIBUTED : DrillDistributionTrait.SINGLETON;
+
+    RelNode lastRelNode;
+    RelDataType dbscanRowType = convertRowType(origScan.getRowType(), origScan.getCluster().getTypeFactory());
+    ScanPrel dbScan = new ScanPrel(origScan.getCluster(),
+        origScan.getTraitSet().plus(Prel.DRILL_PHYSICAL).plus(partition), restrictedGroupScan, dbscanRowType, origScan.getTable());
+    lastRelNode = dbScan;
+    // build the row type for the left Project
+    List<RexNode> leftProjectExprs = Lists.newArrayList();
+    int leftRowKeyIndex = getRowKeyIndex(dbScan.getRowType(), origScan);
+    final RelDataTypeField leftRowKeyField = dbScan.getRowType().getFieldList().get(leftRowKeyIndex);
+    final RelDataTypeFactory.FieldInfoBuilder leftFieldTypeBuilder =
+        dbScan.getCluster().getTypeFactory().builder();
+
+    FilterPrel leftIndexFilterPrel = new FilterPrel(dbScan.getCluster(), dbScan.getTraitSet(),
+          dbScan, indexContext.getOrigCondition());
+    lastRelNode = leftIndexFilterPrel;
+
+    // new Project's rowtype is original Project's rowtype [plus rowkey if rowkey is not in original rowtype]
+    ProjectPrel leftIndexProjectPrel = null;
+    if (origProject != null) {
+      RelDataType origRowType = origProject.getRowType();
+      List<RelDataTypeField> origProjFields = origRowType.getFieldList();
+      leftFieldTypeBuilder.addAll(origProjFields);
+      // get the exprs from the original Project IFF there is a project
+
+      leftProjectExprs.addAll(IndexPlanUtils.getProjects(origProject));
+      // add the rowkey IFF rowkey is not in orig scan
+      if (getRowKeyIndex(origRowType, origScan) < 0) {
+        leftFieldTypeBuilder.add(leftRowKeyField);
+        leftProjectExprs.add(RexInputRef.of(leftRowKeyIndex, dbScan.getRowType()));
+      }
+
+      final RelDataType leftProjectRowType = leftFieldTypeBuilder.build();
+      leftIndexProjectPrel = new ProjectPrel(dbScan.getCluster(), dbScan.getTraitSet(),
+          leftIndexFilterPrel == null ? dbScan : leftIndexFilterPrel, leftProjectExprs, leftProjectRowType);
+      lastRelNode = leftIndexProjectPrel;
+    }
+
+    final RelTraitSet leftTraits = dbScan.getTraitSet().plus(Prel.DRILL_PHYSICAL);
+    // final RelNode convertedLeft = convert(leftIndexProjectPrel, leftTraits);
+    final RelNode convertedLeft = Prule.convert(lastRelNode, leftTraits);
+
+    return new Pair<>(convertedLeft, restrictedGroupScan);
+  }
+
+  @Override
+  public RelNode convertChild(final RelNode filter, final RelNode input) throws InvalidRelException {
+    Map<IndexDescriptor, RexNode> idxConditionMap = Maps.newLinkedHashMap();
+    for(IndexDescriptor idx : indexInfoMap.keySet()) {
+      idxConditionMap.put(idx, indexInfoMap.get(idx).indexCondition);
+    }
+
+    RelNode indexPlan = null;
+    boolean generateDistribution;
+    int curIdx = 0;
+    RexNode remnant = indexContext.getFilterCondition();
+    for (Map.Entry<IndexDescriptor, RexNode> pair : idxConditionMap.entrySet()) {
+      //For the last index, the generated join is distributed using createRangeDistRight instead!
+      generateDistribution = (idxConditionMap.entrySet().size()-1-curIdx) > 0;
+      indexPlan = buildIntersectPlan(pair, indexPlan, generateDistribution);
+      remnant = indexInfoMap.get(pair.getKey()).remainderCondition;
+      ++curIdx;
+    }
+
+    final RelDataTypeField rightRowKeyField = indexPlan.getRowType().getFieldList().get(0);
+    final RelNode rangeDistRight = createRangeDistRight(indexPlan, rightRowKeyField,
+        (DbGroupScan)IndexPlanUtils.getGroupScan(origScan));
+
+    //now with index plan constructed, build plan of left(probe) side to use restricted db scan
+
+    Pair<RelNode, DbGroupScan> leftRelAndScan = buildRestrictedDBScan(remnant);
+
+    RelNode finalRel = buildRowKeyJoin(leftRelAndScan.left, rangeDistRight, true, JoinControl.DEFAULT);
+    if ( upperProject != null) {
+      ProjectPrel cap = new ProjectPrel(finalRel.getCluster(), finalRel.getTraitSet(),
+          finalRel, IndexPlanUtils.getProjects(upperProject), upperProject.getRowType());
+      finalRel = cap;
+    }
+
+    return finalRel;
+  }
+
+}
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/index/generators/NonCoveringIndexPlanGenerator.java b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/index/generators/NonCoveringIndexPlanGenerator.java
new file mode 100644
index 0000000..db220fa
--- /dev/null
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/index/generators/NonCoveringIndexPlanGenerator.java
@@ -0,0 +1,335 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.drill.exec.planner.index.generators;
+
+import java.util.ArrayList;
+import java.util.List;
+
+import org.apache.calcite.plan.RelOptUtil;
+import org.apache.calcite.plan.RelTraitSet;
+import org.apache.calcite.rel.RelCollations;
+import org.apache.drill.common.expression.SchemaPath;
+import org.apache.drill.exec.physical.base.DbGroupScan;
+import org.apache.drill.exec.physical.base.IndexGroupScan;
+import org.apache.drill.exec.planner.common.JoinControl;
+import org.apache.drill.exec.planner.index.IndexLogicalPlanCallContext;
+import org.apache.drill.exec.planner.index.IndexDescriptor;
+import org.apache.drill.exec.planner.index.FunctionalIndexInfo;
+import org.apache.drill.exec.planner.index.FunctionalIndexHelper;
+import org.apache.drill.exec.planner.index.IndexPlanUtils;
+import org.apache.drill.exec.planner.logical.DrillScanRel;
+import org.apache.drill.exec.planner.physical.DrillDistributionTrait;
+import org.apache.drill.exec.planner.physical.DrillDistributionTraitDef;
+import org.apache.drill.exec.planner.physical.FilterPrel;
+import org.apache.drill.exec.planner.physical.HashJoinPrel;
+import org.apache.drill.exec.planner.physical.PlannerSettings;
+import org.apache.drill.exec.planner.physical.Prel;
+import org.apache.drill.exec.planner.physical.ProjectPrel;
+import org.apache.drill.exec.planner.physical.Prule;
+import org.apache.drill.exec.planner.physical.RowKeyJoinPrel;
+import org.apache.drill.exec.planner.physical.ScanPrel;
+import org.apache.calcite.rel.InvalidRelException;
+import org.apache.calcite.rel.RelCollation;
+import org.apache.calcite.rel.RelCollationTraitDef;
+import org.apache.calcite.rel.RelNode;
+import org.apache.calcite.rel.core.JoinRelType;
+import org.apache.calcite.rel.type.RelDataType;
+import org.apache.calcite.rel.type.RelDataTypeFactory;
+import org.apache.calcite.rel.type.RelDataTypeField;
+import org.apache.calcite.rex.RexBuilder;
+import org.apache.calcite.rex.RexInputRef;
+import org.apache.calcite.rex.RexNode;
+import org.apache.drill.shaded.guava.com.google.common.base.Preconditions;
+import org.apache.drill.shaded.guava.com.google.common.collect.ImmutableList;
+import org.apache.drill.shaded.guava.com.google.common.collect.Lists;
+
+/**
+ * Generate a non-covering index plan that is equivalent to the original plan. The non-covering plan consists
+ * of a join-back between an index lookup and the primary table. This join-back is performed using a rowkey join.
+ * For the primary table, we use a restricted scan that allows doing skip-scan instead of sequential scan.
+ *
+ * Original Plan:
+ *               Filter
+ *                 |
+ *            DBGroupScan
+ *
+ * New Plan:
+ *
+ *            RowKeyJoin
+ *          /         \
+ * Remainder Filter  Exchange
+ *         |            |
+ *   Restricted    Filter (with index columns only)
+ *   DBGroupScan        |
+ *                  IndexGroupScan
+ *
+ * This plan will be further optimized by the filter pushdown rule of the Index plugin which should
+ * push the index column filters into the index scan.
+ */
+public class NonCoveringIndexPlanGenerator extends AbstractIndexPlanGenerator {
+
+  static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(NonCoveringIndexPlanGenerator.class);
+  final protected IndexGroupScan indexGroupScan;
+  final private IndexDescriptor indexDesc;
+  // Ideally This functionInfo should be cached along with indexDesc.
+  final protected FunctionalIndexInfo functionInfo;
+
+  public NonCoveringIndexPlanGenerator(IndexLogicalPlanCallContext indexContext,
+                                       IndexDescriptor indexDesc,
+                                       IndexGroupScan indexGroupScan,
+                                       RexNode indexCondition,
+                                       RexNode remainderCondition,
+                                       RexBuilder builder,
+                                       PlannerSettings settings) {
+    super(indexContext, indexCondition, remainderCondition, builder, settings);
+    this.indexGroupScan = indexGroupScan;
+    this.indexDesc = indexDesc;
+    this.functionInfo = indexDesc.getFunctionalInfo();
+  }
+
+  @Override
+  public RelNode convertChild(final RelNode topRel, final RelNode input) throws InvalidRelException {
+
+    if (indexGroupScan == null) {
+      logger.error("Null indexgroupScan in NonCoveringIndexPlanGenerator.convertChild");
+      return null;
+    }
+
+    RelDataType dbscanRowType = convertRowType(origScan.getRowType(), origScan.getCluster().getTypeFactory());
+    RelDataType indexScanRowType = FunctionalIndexHelper.convertRowTypeForIndexScan(
+        origScan, indexContext.getOrigMarker(), indexGroupScan, functionInfo);
+
+    DrillDistributionTrait partition = IndexPlanUtils.scanIsPartition(IndexPlanUtils.getGroupScan(origScan))?
+        DrillDistributionTrait.RANDOM_DISTRIBUTED : DrillDistributionTrait.SINGLETON;
+
+    ScanPrel indexScanPrel = new ScanPrel(origScan.getCluster(),
+        origScan.getTraitSet().plus(Prel.DRILL_PHYSICAL).plus(partition), indexGroupScan, indexScanRowType, origScan.getTable());
+    DbGroupScan origDbGroupScan = (DbGroupScan)IndexPlanUtils.getGroupScan(origScan);
+
+    // right (build) side of the rowkey join: do a distribution of project-filter-indexscan subplan
+    RexNode convertedIndexCondition = FunctionalIndexHelper.convertConditionForIndexScan(indexCondition,
+        origScan, indexScanRowType, builder, functionInfo);
+    FilterPrel  rightIndexFilterPrel = new FilterPrel(indexScanPrel.getCluster(), indexScanPrel.getTraitSet(),
+          indexScanPrel, convertedIndexCondition);
+
+    double finalRowCount = indexGroupScan.getRowCount(indexContext.getOrigCondition(), origScan);
+
+    // project the rowkey column from the index scan
+    List<RexNode> rightProjectExprs = Lists.newArrayList();
+    int rightRowKeyIndex = getRowKeyIndex(indexScanPrel.getRowType(), origScan);//indexGroupScan.getRowKeyOrdinal();
+    assert rightRowKeyIndex >= 0;
+
+    rightProjectExprs.add(RexInputRef.of(rightRowKeyIndex, indexScanPrel.getRowType()));
+
+    final List<RelDataTypeField> indexScanFields = indexScanPrel.getRowType().getFieldList();
+
+    final RelDataTypeFactory.FieldInfoBuilder rightFieldTypeBuilder =
+        indexScanPrel.getCluster().getTypeFactory().builder();
+
+    // build the row type for the right Project
+    final RelDataTypeField rightRowKeyField = indexScanFields.get(rightRowKeyIndex);
+    rightFieldTypeBuilder.add(rightRowKeyField);
+    final RelDataType rightProjectRowType = rightFieldTypeBuilder.build();
+
+    final ProjectPrel rightIndexProjectPrel = new ProjectPrel(indexScanPrel.getCluster(), indexScanPrel.getTraitSet(),
+        rightIndexFilterPrel, rightProjectExprs, rightProjectRowType);
+
+    // create a RANGE PARTITION on the right side (this could be removed later during ExcessiveExchangeIdentifier phase
+    // if the estimated row count is smaller than slice_target
+    final RelNode rangeDistRight = createRangeDistRight(rightIndexProjectPrel, rightRowKeyField, origDbGroupScan);
+
+    // the range partitioning adds an extra column for the partition id but in the final plan we already have a
+    // renaming Project for the _id field inserted as part of the JoinPrelRenameVisitor. Thus, we are not inserting
+    // a separate Project here.
+    final RelNode convertedRight = rangeDistRight;
+
+    // left (probe) side of the rowkey join
+
+    List<SchemaPath> cols = new ArrayList<SchemaPath>(origDbGroupScan.getColumns());
+    if (!checkRowKey(cols)) {
+      cols.add(origDbGroupScan.getRowKeyPath());
+    }
+
+    // Create a restricted groupscan from the primary table's groupscan
+    DbGroupScan restrictedGroupScan  = (DbGroupScan)origDbGroupScan.getRestrictedScan(cols);
+    if (restrictedGroupScan == null) {
+      logger.error("Null restricted groupscan in NonCoveringIndexPlanGenerator.convertChild");
+      return null;
+    }
+    // Set left side (restricted scan) row count as rows returned from right side (index scan)
+    DrillScanRel rightIdxRel = new DrillScanRel(origScan.getCluster(), origScan.getTraitSet(),
+        origScan.getTable(), origScan.getRowType(), indexContext.getScanColumns());
+    double rightIdxRowCount = indexGroupScan.getRowCount(indexCondition, rightIdxRel);
+    restrictedGroupScan.setRowCount(null, rightIdxRowCount, rightIdxRowCount);
+
+    RelTraitSet origScanTraitSet = origScan.getTraitSet();
+    RelTraitSet restrictedScanTraitSet = origScanTraitSet.plus(Prel.DRILL_PHYSICAL);
+
+    // Create the collation traits for restricted scan based on the index columns under the
+    // conditions that (a) the index actually has collation property (e.g hash indexes don't)
+    // and (b) if an explicit sort operation is not enforced
+    RelCollation collation = null;
+    if (indexDesc.getCollation() != null &&
+         !settings.isIndexForceSortNonCovering()) {
+      collation = IndexPlanUtils.buildCollationNonCoveringIndexScan(indexDesc, indexScanRowType, dbscanRowType, indexContext);
+      if (restrictedScanTraitSet.contains(RelCollationTraitDef.INSTANCE)) { // replace existing trait
+        restrictedScanTraitSet = restrictedScanTraitSet.plus(partition).replace(collation);
+      } else {  // add new one
+        restrictedScanTraitSet = restrictedScanTraitSet.plus(partition).plus(collation);
+      }
+    }
+
+    ScanPrel dbScan = new ScanPrel(origScan.getCluster(),
+        restrictedScanTraitSet, restrictedGroupScan, dbscanRowType, origScan.getTable());
+    RelNode lastLeft = dbScan;
+    // build the row type for the left Project
+    List<RexNode> leftProjectExprs = Lists.newArrayList();
+    int leftRowKeyIndex = getRowKeyIndex(dbScan.getRowType(), origScan);
+    final RelDataTypeField leftRowKeyField = dbScan.getRowType().getFieldList().get(leftRowKeyIndex);
+    final RelDataTypeFactory.FieldInfoBuilder leftFieldTypeBuilder =
+        dbScan.getCluster().getTypeFactory().builder();
+
+    //we are applying the same index condition to primary table's restricted scan, the reason
+    // for this is, the scans on index table and primary table are not a transaction, meaning that _after_ index scan,
+    // primary table might already have data get updated, thus some rows picked by index were modified and no more satisfy the
+    // index condition. By applying the same index condition again here, we will avoid the possibility to have some
+    //not-wanted records get into downstream operators in such scenarios.
+    //the remainder condition will be applied on top of RowKeyJoin.
+    FilterPrel leftIndexFilterPrel = new FilterPrel(dbScan.getCluster(), dbScan.getTraitSet(),
+          dbScan, indexContext.getOrigCondition());
+
+    lastLeft = leftIndexFilterPrel;
+
+    RelDataType origRowType = origProject == null ? origScan.getRowType() : origProject.getRowType();
+
+    if (origProject != null) {// then we also  don't need a project
+      // new Project's rowtype is original Project's rowtype [plus rowkey if rowkey is not in original rowtype]
+      List<RelDataTypeField> origProjFields = origRowType.getFieldList();
+      leftFieldTypeBuilder.addAll(origProjFields);
+      // get the exprs from the original Project
+
+      leftProjectExprs.addAll(IndexPlanUtils.getProjects(origProject));
+      // add the rowkey IFF rowkey is not in orig scan
+      if (getRowKeyIndex(origRowType, origScan) < 0) {
+        leftFieldTypeBuilder.add(leftRowKeyField);
+        leftProjectExprs.add(RexInputRef.of(leftRowKeyIndex, dbScan.getRowType()));
+      }
+
+      final RelDataType leftProjectRowType = leftFieldTypeBuilder.build();
+
+      //build collation in project
+      if (!settings.isIndexForceSortNonCovering()){
+        collation = IndexPlanUtils.buildCollationProject(leftProjectExprs, null, dbScan, functionInfo, indexContext);
+      }
+
+      final ProjectPrel leftIndexProjectPrel = new ProjectPrel(dbScan.getCluster(),
+          collation != null ? dbScan.getTraitSet().plus(collation) : dbScan.getTraitSet(),
+          leftIndexFilterPrel == null ? dbScan : leftIndexFilterPrel, leftProjectExprs, leftProjectRowType);
+      lastLeft = leftIndexProjectPrel;
+    }
+    final RelTraitSet leftTraits = dbScan.getTraitSet().plus(Prel.DRILL_PHYSICAL);
+    // final RelNode convertedLeft = convert(leftIndexProjectPrel, leftTraits);
+    final RelNode convertedLeft = Prule.convert(lastLeft, leftTraits);
+
+    // find the rowkey column on the left side of join
+    final int leftRowKeyIdx = getRowKeyIndex(convertedLeft.getRowType(), origScan);
+    final int rightRowKeyIdx = 0; // only rowkey field is being projected from right side
+
+    assert leftRowKeyIdx >= 0;
+
+    List<Integer> leftJoinKeys = ImmutableList.of(leftRowKeyIdx);
+    List<Integer> rightJoinKeys = ImmutableList.of(rightRowKeyIdx);
+
+    RexNode joinCondition =
+        RelOptUtil.createEquiJoinCondition(convertedLeft, leftJoinKeys,
+            convertedRight, rightJoinKeys, builder);
+
+    RelNode newRel;
+    if (settings.isIndexUseHashJoinNonCovering()) {
+      //for hash join, collation will be cleared
+      HashJoinPrel hjPrel = new HashJoinPrel(topRel.getCluster(), leftTraits, convertedLeft,
+          convertedRight, joinCondition, JoinRelType.INNER, false /* no swap */,
+          null /* no runtime filter */,
+          true /* useful for join-restricted scans */, JoinControl.DEFAULT);
+      newRel = hjPrel;
+    } else {
+      //if there is collation, add to rowkey join
+      RowKeyJoinPrel rjPrel = new RowKeyJoinPrel(topRel.getCluster(),
+          collation != null ? leftTraits.plus(collation) : leftTraits,
+          convertedLeft, convertedRight, joinCondition, JoinRelType.INNER);
+
+      rjPrel.setEstimatedRowCount(finalRowCount);
+      newRel = rjPrel;
+    }
+
+    final RelDataTypeFactory.FieldInfoBuilder finalFieldTypeBuilder =
+        origScan.getCluster().getTypeFactory().builder();
+
+    List<RelDataTypeField> rjRowFields = newRel.getRowType().getFieldList();
+    int toRemoveRowKeyCount = 1;
+    if (getRowKeyIndex(origRowType, origScan)  < 0 ) {
+      toRemoveRowKeyCount = 2;
+    }
+    finalFieldTypeBuilder.addAll(rjRowFields.subList(0, rjRowFields.size()-toRemoveRowKeyCount));
+    final RelDataType finalProjectRowType = finalFieldTypeBuilder.build();
+
+    List<RexNode> resetExprs = Lists.newArrayList();
+    for (int idx=0; idx<rjRowFields.size()-toRemoveRowKeyCount; ++idx) {
+      resetExprs.add(RexInputRef.of(idx, newRel.getRowType()));
+    }
+
+    //rewrite the collation for this projectPrel
+    final ProjectPrel resetProjectPrel = new ProjectPrel(newRel.getCluster(), newRel.getTraitSet(),
+        newRel, resetExprs, finalProjectRowType);
+    newRel = resetProjectPrel;
+
+    if ( upperProject != null) {
+      RelCollation newCollation = RelCollations.of(RelCollations.EMPTY.getFieldCollations());
+      DrillDistributionTrait newDist = null;
+
+      newDist = upperProject.getInput().getTraitSet().getTrait(DrillDistributionTraitDef.INSTANCE);
+      if (!settings.isIndexForceSortNonCovering()) {
+        newCollation = IndexPlanUtils.buildCollationProject(IndexPlanUtils.getProjects(upperProject), origProject, origScan,
+            functionInfo, indexContext);
+      }
+      RelTraitSet newProjectTraits = newTraitSet(Prel.DRILL_PHYSICAL, newDist, newCollation);
+      ProjectPrel cap = new ProjectPrel(upperProject.getCluster(),
+          newProjectTraits,
+          newRel, IndexPlanUtils.getProjects(upperProject), upperProject.getRowType());
+      newRel = cap;
+    }
+
+    //whether to remove sort
+    if (indexContext.getSort() != null) {
+      // When ordering is required, serialize the index scan side. With parallel index scans, the rowkey join may receive
+      // unsorted input because ordering is not guaranteed across different parallel inputs.
+      if (toRemoveSort(indexContext.getSort(), newRel.getTraitSet().getTrait(RelCollationTraitDef.INSTANCE))) {
+        ((IndexGroupScan)indexScanPrel.getGroupScan()).setParallelizationWidth(1);
+      }
+      newRel = getSortNode(indexContext, newRel, false,true, true);
+      Preconditions.checkArgument(newRel != null);
+    }
+
+    RelNode finalRel = Prule.convert(newRel, newRel.getTraitSet());
+    logger.debug("NonCoveringIndexPlanGenerator got finalRel {} from origScan {}",
+        finalRel.toString(), origScan.toString());
+    return finalRel;
+  }
+}
+
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/index/rules/AbstractMatchFunction.java b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/index/rules/AbstractMatchFunction.java
new file mode 100644
index 0000000..0449c10
--- /dev/null
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/index/rules/AbstractMatchFunction.java
@@ -0,0 +1,58 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.drill.exec.planner.index.rules;
+
+import org.apache.drill.common.expression.SchemaPath;
+import org.apache.drill.exec.physical.base.DbGroupScan;
+import org.apache.drill.exec.physical.base.GroupScan;
+import org.apache.drill.exec.planner.logical.DrillScanRel;
+
+public abstract class AbstractMatchFunction<T> implements MatchFunction<T> {
+  public boolean checkScan(DrillScanRel scanRel) {
+    GroupScan groupScan = scanRel.getGroupScan();
+    if (groupScan instanceof DbGroupScan) {
+      DbGroupScan dbscan = ((DbGroupScan) groupScan);
+      //if we already applied index convert rule, and this scan is indexScan or restricted scan already,
+      //no more trying index convert rule
+      return dbscan.supportsSecondaryIndex() && (!dbscan.isIndexScan()) && (!dbscan.isRestrictedScan());
+    }
+    return false;
+  }
+
+  public boolean checkScan(GroupScan groupScan) {
+    if (groupScan instanceof DbGroupScan) {
+      DbGroupScan dbscan = ((DbGroupScan) groupScan);
+      //if we already applied index convert rule, and this scan is indexScan or restricted scan already,
+      //no more trying index convert rule
+      return dbscan.supportsSecondaryIndex() &&
+             !dbscan.isRestrictedScan() &&
+              (!dbscan.isFilterPushedDown() || dbscan.isIndexScan()) &&
+             !containsStar(dbscan);
+    }
+    return false;
+  }
+
+  public static boolean containsStar(DbGroupScan dbscan) {
+    for (SchemaPath column : dbscan.getColumns()) {
+      if (column.getRootSegment().getPath().startsWith("*")) {
+        return true;
+      }
+    }
+    return false;
+  }
+}
\ No newline at end of file
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/index/rules/DbScanSortRemovalRule.java b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/index/rules/DbScanSortRemovalRule.java
new file mode 100644
index 0000000..db09504
--- /dev/null
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/index/rules/DbScanSortRemovalRule.java
@@ -0,0 +1,226 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.drill.exec.planner.index.rules;
+
+import org.apache.drill.shaded.guava.com.google.common.base.Stopwatch;
+import org.apache.calcite.plan.RelOptRule;
+import org.apache.calcite.plan.RelOptRuleCall;
+import org.apache.calcite.plan.RelOptRuleOperand;
+import org.apache.calcite.rel.RelCollationTraitDef;
+import org.apache.drill.exec.physical.base.DbGroupScan;
+import org.apache.drill.exec.planner.index.IndexPhysicalPlanCallContext;
+import org.apache.drill.exec.planner.index.IndexProperties;
+import org.apache.drill.exec.planner.index.IndexSelector;
+import org.apache.drill.exec.planner.index.IndexPlanUtils;
+import org.apache.drill.exec.planner.index.IndexCollection;
+import org.apache.drill.exec.planner.index.IndexDescriptor;
+import org.apache.drill.exec.planner.index.FunctionalIndexInfo;
+import org.apache.drill.exec.planner.index.generators.AbstractIndexPlanGenerator;
+import org.apache.drill.exec.planner.index.generators.CoveringPlanNoFilterGenerator;
+import org.apache.drill.exec.planner.logical.RelOptHelper;
+import org.apache.drill.exec.planner.physical.Prule;
+import org.apache.drill.exec.planner.physical.PlannerSettings;
+import org.apache.drill.exec.planner.physical.PrelUtil;
+import org.apache.drill.exec.planner.physical.ExchangePrel;
+import org.apache.drill.exec.planner.physical.ProjectPrel;
+import org.apache.drill.exec.planner.physical.ScanPrel;
+import org.apache.drill.exec.planner.physical.SortPrel;
+import org.apache.drill.exec.planner.physical.Prel;
+import org.apache.drill.exec.planner.physical.HashToRandomExchangePrel;
+import org.apache.calcite.rel.RelNode;
+import org.apache.drill.shaded.guava.com.google.common.collect.Lists;
+
+import java.util.List;
+import java.util.concurrent.TimeUnit;
+
+public class DbScanSortRemovalRule extends Prule {
+  static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(DbScanSortRemovalRule.class);
+
+  public static final RelOptRule INDEX_SORT_EXCHANGE_SCAN =
+      new DbScanSortRemovalRule(RelOptHelper.some(SortPrel.class,
+          RelOptHelper.some(HashToRandomExchangePrel.class,
+              RelOptHelper.any(ScanPrel.class))), "DbScanSortRemovalRule:sort_exchange_Scan", new MatchSES());
+
+  public static final RelOptRule INDEX_SORT_SCAN =
+          new DbScanSortRemovalRule(RelOptHelper.some(SortPrel.class,
+                          RelOptHelper.any(ScanPrel.class)), "DbScanSortRemovalRule:Sort_Scan", new MatchSS());
+
+  public static final RelOptRule INDEX_SORT_PROJ_SCAN =
+          new DbScanSortRemovalRule(RelOptHelper.some(SortPrel.class,
+                  RelOptHelper.some(ProjectPrel.class,
+                    RelOptHelper.any(ScanPrel.class))), "DbScanSortRemovalRule:Sort_Proj_Scan", new MatchSPS());
+
+  public static final RelOptRule INDEX_SORT_EXCHANGE_PROJ_SCAN =
+      new DbScanSortRemovalRule(RelOptHelper.some(SortPrel.class,
+          RelOptHelper.some(HashToRandomExchangePrel.class,
+              RelOptHelper.some(ProjectPrel.class,
+                  RelOptHelper.any(ScanPrel.class)))), "DbScanSortRemovalRule:sort_exchange_proj_Scan", new MatchSEPS());
+
+  final private MatchFunction match;
+
+  private DbScanSortRemovalRule(RelOptRuleOperand operand,
+                                   String description,
+                                   MatchFunction match) {
+    super(operand, description);
+    this.match = match;
+  }
+
+  private static class MatchSES extends AbstractMatchFunction<IndexPhysicalPlanCallContext> {
+
+    public boolean match(RelOptRuleCall call) {
+      final ScanPrel scan = (ScanPrel)call.rel(2);
+      return checkScan(scan.getGroupScan());
+    }
+
+    public IndexPhysicalPlanCallContext onMatch(RelOptRuleCall call) {
+      final ScanPrel scan = call.rel(2);
+      final SortPrel sort = call.rel(0);
+      final ExchangePrel exch = call.rel(1);
+      return new IndexPhysicalPlanCallContext(call, sort, null, scan, exch);
+    }
+  }
+
+  private static class MatchSS extends AbstractMatchFunction<IndexPhysicalPlanCallContext> {
+
+    public boolean match(RelOptRuleCall call) {
+      final ScanPrel scan = (ScanPrel)call.rel(1);
+      return checkScan(scan.getGroupScan());
+    }
+
+    public IndexPhysicalPlanCallContext onMatch(RelOptRuleCall call) {
+      final ScanPrel scan = call.rel(1);
+      final SortPrel sort = call.rel(0);
+      return new IndexPhysicalPlanCallContext(call, sort, null, scan, null);
+    }
+  }
+
+  private static class MatchSPS extends AbstractMatchFunction<IndexPhysicalPlanCallContext> {
+
+    public boolean match(RelOptRuleCall call) {
+      final ScanPrel scan = (ScanPrel)call.rel(2);
+      return checkScan(scan.getGroupScan());
+    }
+
+    public IndexPhysicalPlanCallContext onMatch(RelOptRuleCall call) {
+      final ScanPrel scan = call.rel(2);
+      final ProjectPrel proj = call.rel(1);
+      final SortPrel sort = call.rel(0);
+      return new IndexPhysicalPlanCallContext(call, sort, proj, scan, null);
+    }
+  }
+
+  private static class MatchSEPS extends AbstractMatchFunction<IndexPhysicalPlanCallContext> {
+
+    public boolean match(RelOptRuleCall call) {
+      final ScanPrel scan = (ScanPrel)call.rel(3);
+      return checkScan(scan.getGroupScan());
+    }
+
+    public IndexPhysicalPlanCallContext onMatch(RelOptRuleCall call) {
+      final ScanPrel scan = call.rel(3);
+      final SortPrel sort = call.rel(0);
+      final ProjectPrel proj = call.rel(2);
+      final ExchangePrel exch = call.rel(1);
+      return new IndexPhysicalPlanCallContext(call,  sort, proj, scan, exch);
+    }
+  }
+
+  @Override
+  public boolean matches(RelOptRuleCall call) {
+    return match.match(call);
+  }
+
+  @Override
+  public void onMatch(RelOptRuleCall call) {
+    doOnMatch((IndexPhysicalPlanCallContext) match.onMatch(call));
+  }
+
+  private void doOnMatch(IndexPhysicalPlanCallContext indexContext) {
+    Stopwatch indexPlanTimer = Stopwatch.createStarted();
+    final PlannerSettings settings = PrelUtil.getPlannerSettings(indexContext.call.getPlanner());
+
+    DbGroupScan groupScan = (DbGroupScan)indexContext.scan.getGroupScan();
+    boolean isIndexScan = groupScan.isIndexScan();
+    if (!isIndexScan) {
+      // This case generates the index scan and removes the sort if possible.
+      final IndexCollection indexCollection = groupScan.getSecondaryIndexCollection(indexContext.scan);
+      if (indexCollection == null) {
+        return;
+      }
+      if (settings.isStatisticsEnabled()) {
+        groupScan.getStatistics().initialize(null, indexContext.scan, indexContext);
+      }
+      IndexPlanUtils.updateSortExpression(indexContext, indexContext.getSort() != null ?
+              indexContext.getCollation().getFieldCollations() : null);
+      IndexSelector selector = new IndexSelector(indexContext);
+      for (IndexDescriptor indexDesc : indexCollection) {
+        indexDesc.getIndexGroupScan().setStatistics(groupScan.getStatistics());
+        FunctionalIndexInfo functionInfo = indexDesc.getFunctionalInfo();
+        if (IndexPlanUtils.isCoveringIndex(indexContext, functionInfo)) {
+          selector.addIndex(indexDesc, true,
+                  indexContext.lowerProject != null ? indexContext.lowerProject.getRowType().getFieldCount() :
+                          indexContext.scan.getRowType().getFieldCount());
+        }
+      }
+
+      IndexProperties idxProp = selector.getBestIndexNoFilter();
+      if (idxProp != null) {
+        try {
+          //generate a covering plan
+          CoveringPlanNoFilterGenerator planGen =
+                  new CoveringPlanNoFilterGenerator(indexContext, idxProp.getIndexDesc().getFunctionalInfo(),
+                          false, settings);
+          if (planGen.convertChild() != null) {
+            indexContext.getCall().transformTo(planGen.convertChild());
+          }
+        } catch (Exception e) {
+          logger.warn("Exception while trying to generate indexscan to remove sort", e);
+        }
+      }
+    } else {
+      //This case tries to use the already generated index to see if a sort can be removed.
+      if (indexContext.scan.getTraitSet().getTrait(RelCollationTraitDef.INSTANCE).getFieldCollations().size() == 0) {
+        return;
+      }
+      try {
+        RelNode finalRel = indexContext.scan.copy(indexContext.scan.getTraitSet(), indexContext.scan.getInputs());
+        if (indexContext.lowerProject != null) {
+          List<RelNode> inputs = Lists.newArrayList();
+          inputs.add(finalRel);
+          finalRel = indexContext.lowerProject.copy(indexContext.lowerProject.getTraitSet(), inputs);
+        }
+        if (indexContext.getSort() != null) {
+          finalRel = AbstractIndexPlanGenerator.getSortNode(indexContext, finalRel, true,false,
+                  indexContext.exch != null);
+        }
+
+        if (finalRel == null) {
+          return;
+        }
+
+        finalRel = Prule.convert(finalRel, finalRel.getTraitSet().plus(Prel.DRILL_PHYSICAL));
+        indexContext.getCall().transformTo(finalRel);
+      } catch (Exception e) {
+        logger.warn("Exception while trying to use the indexscan to remove the sort", e);
+      }
+    }
+
+    indexPlanTimer.stop();
+    logger.debug("Index Planning took {} ms", indexPlanTimer.elapsed(TimeUnit.MILLISECONDS));
+  }
+}
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/index/rules/DbScanToIndexScanPrule.java b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/index/rules/DbScanToIndexScanPrule.java
new file mode 100644
index 0000000..61191bc
--- /dev/null
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/index/rules/DbScanToIndexScanPrule.java
@@ -0,0 +1,533 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.drill.exec.planner.index.rules;
+
+import org.apache.drill.shaded.guava.com.google.common.base.Stopwatch;
+import org.apache.drill.shaded.guava.com.google.common.collect.Lists;
+
+import org.apache.calcite.plan.RelOptRule;
+import org.apache.calcite.plan.RelOptRuleCall;
+import org.apache.calcite.plan.RelOptRuleOperand;
+import org.apache.calcite.plan.RelOptUtil;
+import org.apache.calcite.rel.RelNode;
+import org.apache.calcite.rex.RexBuilder;
+import org.apache.calcite.rex.RexNode;
+import org.apache.drill.exec.physical.base.DbGroupScan;
+import org.apache.drill.exec.physical.base.GroupScan;
+import org.apache.drill.exec.physical.base.IndexGroupScan;
+import org.apache.drill.exec.planner.common.DrillRelNode;
+import org.apache.drill.exec.planner.index.IndexSelector;
+import org.apache.drill.exec.planner.index.IndexPlanUtils;
+import org.apache.drill.exec.planner.index.IndexCollection;
+import org.apache.drill.exec.planner.index.IndexDescriptor;
+import org.apache.drill.exec.planner.index.FunctionalIndexInfo;
+import org.apache.drill.exec.planner.index.IndexLogicalPlanCallContext;
+import org.apache.drill.exec.planner.index.IndexConditionInfo;
+import org.apache.drill.exec.planner.index.IndexGroup;
+import org.apache.drill.exec.planner.index.IndexProperties;
+import org.apache.drill.exec.planner.index.IndexableExprMarker;
+import org.apache.drill.exec.planner.index.Statistics;
+import org.apache.drill.exec.planner.index.generators.CoveringIndexPlanGenerator;
+import org.apache.drill.exec.planner.index.generators.IndexIntersectPlanGenerator;
+import org.apache.drill.exec.planner.index.generators.NonCoveringIndexPlanGenerator;
+import org.apache.drill.exec.planner.logical.DrillFilterRel;
+import org.apache.drill.exec.planner.logical.DrillProjectRel;
+import org.apache.drill.exec.planner.logical.DrillScanRel;
+import org.apache.drill.exec.planner.logical.DrillSortRel;
+import org.apache.drill.exec.planner.logical.RelOptHelper;
+import org.apache.drill.exec.planner.logical.partition.RewriteAsBinaryOperators;
+import org.apache.drill.exec.planner.physical.PlannerSettings;
+import org.apache.drill.exec.planner.physical.PrelUtil;
+import org.apache.drill.exec.planner.physical.Prule;
+
+import java.util.ArrayList;
+import java.util.List;
+import java.util.Map;
+import java.util.concurrent.TimeUnit;
+
+public class DbScanToIndexScanPrule extends Prule {
+  static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(DbScanToIndexScanPrule.class);
+  final public MatchFunction match;
+
+  public static final RelOptRule REL_FILTER_SCAN = new DbScanToIndexScanPrule(
+      RelOptHelper.some(DrillRelNode.class, RelOptHelper.some(DrillFilterRel.class, RelOptHelper.any(DrillScanRel.class))),
+      "DbScanToIndexScanPrule:Rel_Filter_Scan", new MatchRelFS());
+
+  public static final RelOptRule PROJECT_FILTER_PROJECT_SCAN = new DbScanToIndexScanPrule(
+      RelOptHelper.some(DrillProjectRel.class, RelOptHelper.some(DrillFilterRel.class,
+         RelOptHelper.some(DrillProjectRel.class, RelOptHelper.any(DrillScanRel.class)))),
+     "DbScanToIndexScanPrule:Project_Filter_Project_Scan", new MatchPFPS());
+
+  public static final RelOptRule SORT_FILTER_PROJECT_SCAN = new DbScanToIndexScanPrule(
+     RelOptHelper.some(DrillSortRel.class, RelOptHelper.some(DrillFilterRel.class,
+        RelOptHelper.some(DrillProjectRel.class, RelOptHelper.any(DrillScanRel.class)))),
+    "DbScanToIndexScanPrule:Sort_Filter_Project_Scan", new MatchSFPS());
+
+  public static final RelOptRule SORT_PROJECT_FILTER_PROJECT_SCAN = new DbScanToIndexScanPrule(
+      RelOptHelper.some(DrillSortRel.class, RelOptHelper.some(DrillProjectRel.class, RelOptHelper.some(DrillFilterRel.class,
+          RelOptHelper.some(DrillProjectRel.class, RelOptHelper.any(DrillScanRel.class))))),
+      "DbScanToIndexScanPrule:Sort_Project_Filter_Project_Scan", new MatchSPFPS());
+
+  public static final RelOptRule SORT_PROJECT_FILTER_SCAN = new DbScanToIndexScanPrule(
+      RelOptHelper.some(DrillSortRel.class, RelOptHelper.some(DrillProjectRel.class, RelOptHelper.some(DrillFilterRel.class,
+          RelOptHelper.any(DrillScanRel.class)))),
+      "DbScanToIndexScanPrule:Sort_Project_Filter_Scan", new MatchSPFS());
+
+  public static final RelOptRule FILTER_SCAN = new DbScanToIndexScanPrule(
+      RelOptHelper.some(DrillFilterRel.class, RelOptHelper.any(DrillScanRel.class)),
+      "DbScanToIndexScanPrule:Filter_On_Scan", new MatchFS());
+
+  public static final RelOptRule FILTER_PROJECT_SCAN = new DbScanToIndexScanPrule(
+      RelOptHelper.some(DrillFilterRel.class,
+          RelOptHelper.some(DrillProjectRel.class, RelOptHelper.any(DrillScanRel.class))),
+      "DbScanToIndexScanPrule:Filter_Project_Scan", new MatchFPS());
+
+
+  private DbScanToIndexScanPrule(RelOptRuleOperand operand, String description, MatchFunction match) {
+    super(operand, description);
+    this.match = match;
+  }
+
+  @Override
+  public boolean matches(RelOptRuleCall call) {
+    if (getMatchIfRoot(call) != null) {
+      return true;
+    }
+    return match.match(call);
+  }
+
+  @Override
+  public void onMatch(RelOptRuleCall call) {
+    if (getMatchIfRoot(call) != null) {
+      getMatchIfRoot(call).onMatch(call);
+      return;
+    }
+    doOnMatch((IndexLogicalPlanCallContext) match.onMatch(call));
+  }
+
+  private MatchFunction getMatchIfRoot(RelOptRuleCall call) {
+    List<RelNode> rels = call.getRelList();
+    if (call.getPlanner().getRoot().equals(call.rel(0))) {
+      if (rels.size() == 2) {
+        if ((rels.get(0) instanceof DrillFilterRel) && (rels.get(1) instanceof DrillScanRel)) {
+          return ((DbScanToIndexScanPrule)FILTER_SCAN).match;
+        }
+      }
+      else if (rels.size() == 3) {
+        if ((rels.get(0) instanceof DrillFilterRel) && (rels.get(1) instanceof DrillProjectRel) && (rels.get(2) instanceof DrillScanRel)) {
+          return ((DbScanToIndexScanPrule)FILTER_PROJECT_SCAN).match;
+        }
+      }
+    }
+    return null;
+  }
+
+  private static class MatchFPS extends AbstractMatchFunction<IndexLogicalPlanCallContext> {
+
+    public boolean match(RelOptRuleCall call) {
+      final DrillScanRel scan = (DrillScanRel) call.rel(2);
+      return checkScan(scan);
+    }
+
+    public IndexLogicalPlanCallContext onMatch(RelOptRuleCall call) {
+      final DrillFilterRel filter = call.rel(0);
+      final DrillProjectRel project = call.rel(1);
+      final DrillScanRel scan = call.rel(2);
+      return new IndexLogicalPlanCallContext(call, null, filter, project, scan);
+    }
+
+  }
+
+  private static class MatchFS extends AbstractMatchFunction<IndexLogicalPlanCallContext> {
+    public boolean match(RelOptRuleCall call) {
+      final DrillScanRel scan = call.rel(1);
+      return checkScan(scan);
+    }
+
+    public IndexLogicalPlanCallContext onMatch(RelOptRuleCall call) {
+      final DrillFilterRel filter = call.rel(0);
+      final DrillScanRel scan = call.rel(1);
+      return new IndexLogicalPlanCallContext(call, null, filter, null, scan);
+    }
+  }
+
+  private static class MatchRelFS extends AbstractMatchFunction<IndexLogicalPlanCallContext> {
+    public boolean match(RelOptRuleCall call) {
+      if (call.rel(0) instanceof DrillProjectRel ||
+          call.rel(0) instanceof DrillSortRel) {
+        final DrillScanRel scan = call.rel(2);
+        return checkScan(scan);
+      }
+      return false;
+    }
+
+    public IndexLogicalPlanCallContext onMatch(RelOptRuleCall call) {
+      DrillProjectRel capProject = null;
+      DrillSortRel sort = null;
+      if (call.rel(0) instanceof DrillProjectRel) {
+        capProject = call.rel(0);
+      } else if (call.rel(0) instanceof DrillSortRel) {
+        sort = call.rel(0);
+      }
+      final DrillFilterRel filter = call.rel(1);
+      final DrillScanRel scan = call.rel(2);
+      return new IndexLogicalPlanCallContext(call, sort, capProject, filter, null, scan);
+    }
+  }
+
+  private static class MatchPFPS extends AbstractMatchFunction<IndexLogicalPlanCallContext> {
+    public boolean match(RelOptRuleCall call) {
+      final DrillScanRel scan = call.rel(3);
+      return checkScan(scan);
+    }
+
+    public IndexLogicalPlanCallContext onMatch(RelOptRuleCall call) {
+      final DrillProjectRel capProject = call.rel(0);
+      final DrillFilterRel filter = call.rel(1);
+      final DrillProjectRel project = call.rel(2);
+      final DrillScanRel scan = call.rel(3);
+      return new IndexLogicalPlanCallContext(call, null, capProject, filter, project, scan);
+    }
+  }
+
+  private static class MatchSFPS extends AbstractMatchFunction<IndexLogicalPlanCallContext> {
+    public boolean match(RelOptRuleCall call) {
+      final DrillScanRel scan = call.rel(3);
+      return checkScan(scan);
+    }
+
+    public IndexLogicalPlanCallContext onMatch(RelOptRuleCall call) {
+      final DrillSortRel sort = call.rel(0);
+      final DrillFilterRel filter = call.rel(1);
+      final DrillProjectRel project = call.rel(2);
+      final DrillScanRel scan = call.rel(3);
+      return new IndexLogicalPlanCallContext(call, sort, null, filter, project, scan);
+    }
+  }
+
+  private static class MatchSPFPS extends AbstractMatchFunction<IndexLogicalPlanCallContext> {
+    public boolean match(RelOptRuleCall call) {
+      final DrillScanRel scan = call.rel(4);
+      return checkScan(scan);
+    }
+
+    public IndexLogicalPlanCallContext onMatch(RelOptRuleCall call) {
+      final DrillSortRel sort = call.rel(0);
+      final DrillProjectRel capProject = call.rel(1);
+      final DrillFilterRel filter = call.rel(2);
+      final DrillProjectRel project = call.rel(3);
+      final DrillScanRel scan = call.rel(4);
+      return new IndexLogicalPlanCallContext(call, sort, capProject, filter, project, scan);
+    }
+  }
+
+  private static class MatchSPFS extends AbstractMatchFunction<IndexLogicalPlanCallContext> {
+    public boolean match(RelOptRuleCall call) {
+      final DrillScanRel scan = call.rel(3);
+      return checkScan(scan);
+    }
+
+    public IndexLogicalPlanCallContext onMatch(RelOptRuleCall call) {
+      final DrillSortRel sort = call.rel(0);
+      final DrillProjectRel capProject = call.rel(1);
+      final DrillFilterRel filter = call.rel(2);
+      final DrillScanRel scan = call.rel(3);
+      return new IndexLogicalPlanCallContext(call, sort, capProject, filter, null, scan);
+    }
+  }
+
+  protected void doOnMatch(IndexLogicalPlanCallContext indexContext) {
+
+    Stopwatch indexPlanTimer = Stopwatch.createStarted();
+    final PlannerSettings settings = PrelUtil.getPlannerSettings(indexContext.call.getPlanner());
+    final IndexCollection indexCollection = getIndexCollection(settings, indexContext.scan);
+    if( indexCollection == null ) {
+      return;
+    }
+
+    logger.debug("Index Rule {} starts", this.description);
+
+    RexBuilder builder = indexContext.filter.getCluster().getRexBuilder();
+
+    RexNode condition = null;
+    if (indexContext.lowerProject == null) {
+      condition = indexContext.filter.getCondition();
+    } else {
+      // get the filter as if it were below the projection.
+      condition = RelOptUtil.pushFilterPastProject(indexContext.filter.getCondition(), indexContext.lowerProject);
+    }
+
+    //save this pushed down condition, in case it is needed later to build filter when joining back primary table
+    indexContext.origPushedCondition = condition;
+
+    RewriteAsBinaryOperators visitor = new RewriteAsBinaryOperators(true, builder);
+    condition = condition.accept(visitor);
+
+    if (indexCollection.supportsIndexSelection()) {
+      try {
+        processWithIndexSelection(indexContext, settings, condition,
+            indexCollection, builder);
+      } catch(Exception e) {
+        logger.warn("Exception while doing index planning ", e);
+      }
+    } else {
+      throw new UnsupportedOperationException("Index collection must support index selection");
+    }
+    indexPlanTimer.stop();
+    logger.info("index_plan_info: Index Planning took {} ms", indexPlanTimer.elapsed(TimeUnit.MILLISECONDS));
+  }
+  /**
+   * Return the index collection relevant for the underlying data source
+   * @param settings
+   * @param scan
+   */
+  public IndexCollection getIndexCollection(PlannerSettings settings, DrillScanRel scan) {
+    DbGroupScan groupScan = (DbGroupScan)scan.getGroupScan();
+    return groupScan.getSecondaryIndexCollection(scan);
+  }
+
+  private void processWithIndexSelection(
+      IndexLogicalPlanCallContext indexContext,
+      PlannerSettings settings,
+      RexNode condition,
+      IndexCollection collection,
+      RexBuilder builder) {
+    double totalRows = 0;
+    double filterRows = totalRows;
+    DrillScanRel scan = indexContext.scan;
+    if (! (indexContext.scan.getGroupScan() instanceof DbGroupScan) ) {
+      return;
+    }
+    IndexConditionInfo.Builder infoBuilder = IndexConditionInfo.newBuilder(condition, collection, builder, indexContext.scan);
+    IndexConditionInfo cInfo = infoBuilder.getCollectiveInfo(indexContext);
+    boolean isValidIndexHint = infoBuilder.isValidIndexHint(indexContext);
+
+    if (!cInfo.hasIndexCol) {
+      logger.info("index_plan_info: No index columns are projected from the scan..continue.");
+      return;
+    }
+
+    if (cInfo.indexCondition == null) {
+      logger.info("index_plan_info: No conditions were found eligible for applying index lookup.");
+      return;
+    }
+
+    if (!indexContext.indexHint.equals("") && !isValidIndexHint) {
+      logger.warn("index_plan_info: Index Hint {} is not useful as index with that name is not available", indexContext.indexHint);
+    }
+
+    RexNode indexCondition = cInfo.indexCondition;
+    RexNode remainderCondition = cInfo.remainderCondition;
+
+    if (remainderCondition.isAlwaysTrue()) {
+      remainderCondition = null;
+    }
+    logger.debug("index_plan_info: condition split into indexcondition: {} and remaindercondition: {}", indexCondition, remainderCondition);
+
+    IndexableExprMarker indexableExprMarker = new IndexableExprMarker(indexContext.scan);
+    indexCondition.accept(indexableExprMarker);
+    indexContext.origMarker = indexableExprMarker;
+
+    if (scan.getGroupScan() instanceof DbGroupScan) {
+      // Initialize statistics
+      DbGroupScan dbScan = ((DbGroupScan) scan.getGroupScan());
+      if (settings.isStatisticsEnabled()) {
+        dbScan.getStatistics().initialize(condition, scan, indexContext);
+      }
+      totalRows = dbScan.getRowCount(null, scan);
+      filterRows = dbScan.getRowCount(condition, scan);
+      double sel = filterRows/totalRows;
+      if (totalRows != Statistics.ROWCOUNT_UNKNOWN &&
+          filterRows != Statistics.ROWCOUNT_UNKNOWN &&
+          !settings.isDisableFullTableScan() && !isValidIndexHint &&
+          sel > Math.max(settings.getIndexCoveringSelThreshold(),
+              settings.getIndexNonCoveringSelThreshold() )) {
+        // If full table scan is not disabled, generate full table scan only plans if selectivity
+        // is greater than covering and non-covering selectivity thresholds
+        logger.info("index_plan_info: Skip index planning because filter selectivity: {} is greater than thresholds {}, {}",
+            sel, settings.getIndexCoveringSelThreshold(), settings.getIndexNonCoveringSelThreshold());
+        return;
+      }
+    }
+
+    if (totalRows == Statistics.ROWCOUNT_UNKNOWN ||
+        totalRows == 0 || filterRows == Statistics.ROWCOUNT_UNKNOWN ) {
+      logger.warn("index_plan_info: Total row count is UNKNOWN or 0, or filterRows UNKNOWN; skip index planning");
+      return;
+    }
+
+    List<IndexGroup> coveringIndexes = Lists.newArrayList();
+    List<IndexGroup> nonCoveringIndexes = Lists.newArrayList();
+    List<IndexGroup> intersectIndexes = Lists.newArrayList();
+
+    //update sort expressions in context, it is needed for computing collation, so do it before IndexSelector
+    IndexPlanUtils.updateSortExpression(indexContext, indexContext.sort != null ?
+            indexContext.sort.collation.getFieldCollations() : null);
+
+    IndexSelector selector = new IndexSelector(indexCondition,
+        remainderCondition,
+        indexContext,
+        collection,
+        builder,
+        totalRows);
+
+    for (IndexDescriptor indexDesc : collection) {
+      logger.info("index_plan_info indexDescriptor: {}", indexDesc.toString());
+      // check if any of the indexed fields of the index are present in the filter condition
+      if (IndexPlanUtils.conditionIndexed(indexableExprMarker, indexDesc) != IndexPlanUtils.ConditionIndexed.NONE) {
+        if (isValidIndexHint && !indexContext.indexHint.equals(indexDesc.getIndexName())) {
+          logger.info("index_plan_info: Index {} is being discarded due to index Hint", indexDesc.getIndexName());
+          continue;
+        }
+        FunctionalIndexInfo functionInfo = indexDesc.getFunctionalInfo();
+        selector.addIndex(indexDesc, IndexPlanUtils.isCoveringIndex(indexContext, functionInfo),
+            indexContext.lowerProject != null ? indexContext.lowerProject.getRowType().getFieldCount() :
+                scan.getRowType().getFieldCount());
+      }
+    }
+    // get the candidate indexes based on selection
+    selector.getCandidateIndexes(infoBuilder, coveringIndexes, nonCoveringIndexes, intersectIndexes);
+
+    if (logger.isDebugEnabled()) {
+      StringBuilder strb = new StringBuilder();
+      if (coveringIndexes.size() > 0) {
+        strb.append("Covering indexes:");
+        for (IndexGroup index : coveringIndexes) {
+          strb.append(index.getIndexProps().get(0).getIndexDesc().getIndexName()).append(", ");
+        }
+      }
+      if(nonCoveringIndexes.size() > 0) {
+        strb.append("Non-covering indexes:");
+        for (IndexGroup index : nonCoveringIndexes) {
+          strb.append(index.getIndexProps().get(0).getIndexDesc().getIndexName()).append(", ");
+        }
+      }
+      logger.debug("index_plan_info: IndexSelector return: {}",strb.toString());
+    }
+
+    GroupScan primaryTableScan = indexContext.scan.getGroupScan();
+    // Only non-covering indexes can be intersected. Check if
+    // (a) there are no covering indexes. Intersect plans will almost always be more
+    // expensive than a covering index, so no need to generate one if there is covering.
+    // (b) there is more than 1 non-covering indexes that can be intersected
+    // TODO: this logic for intersect should eventually be migrated to the IndexSelector
+    if (coveringIndexes.size() == 0 && nonCoveringIndexes.size() > 1) {
+      List<IndexDescriptor> indexList = Lists.newArrayList();
+      for (IndexGroup index : nonCoveringIndexes) {
+        IndexDescriptor indexDesc = index.getIndexProps().get(0).getIndexDesc();
+        IndexGroupScan idxScan = indexDesc.getIndexGroupScan();
+        //Copy primary table statistics to index table
+        idxScan.setStatistics(((DbGroupScan) primaryTableScan).getStatistics());
+        indexList.add(index.getIndexProps().get(0).getIndexDesc());
+      }
+
+      Map<IndexDescriptor, IndexConditionInfo> indexInfoMap = infoBuilder.getIndexConditionMap(indexList);
+
+      //no usable index
+      if (indexInfoMap == null || indexInfoMap.size() == 0) {
+        logger.info("index_plan_info: skipping intersect plan generation as there is no usable index");
+        return;
+      }
+
+      //if there is only one index found, no need to do intersect, but just a regular non-covering plan
+      //some part of filter condition needs to apply on primary table.
+      if(indexInfoMap.size() > 1) {
+        logger.info("index_plan_info: intersect plan is generated");
+
+        if (logger.isDebugEnabled()) {
+          List<String> indices = new ArrayList<>(nonCoveringIndexes.size());
+          for (IndexGroup index : nonCoveringIndexes) {
+            indices.add(index.getIndexProps().get(0).getIndexDesc().getIndexName());
+          }
+          logger.debug("index_plan_info: intersect plan is generated on index list {}", indices);
+        }
+        boolean intersectPlanGenerated = false;
+        //multiple indexes, let us try to intersect results from multiple index tables
+        //TODO: make sure the smallest selectivity of these indexes times rowcount smaller than broadcast threshold
+        for (IndexGroup index : intersectIndexes) {
+          List<IndexDescriptor> candidateDesc = Lists.newArrayList();
+          for (IndexProperties candProp : index.getIndexProps()) {
+            candidateDesc.add(candProp.getIndexDesc());
+          }
+          Map<IndexDescriptor, IndexConditionInfo> intersectIdxInfoMap = infoBuilder.getIndexConditionMap(candidateDesc);
+          IndexIntersectPlanGenerator planGen = new IndexIntersectPlanGenerator(
+              indexContext, intersectIdxInfoMap, builder, settings);
+          try {
+            planGen.go();
+            intersectPlanGenerated = true;
+          } catch (Exception e) {
+            // If error while generating intersect plans, continue onto generating non-covering plans
+            logger.warn("index_plan_info: Exception while trying to generate intersect index plan", e);
+          }
+        }
+        // If intersect plans are forced do not generate further non-covering plans
+        if (intersectPlanGenerated && settings.isIndexIntersectPlanPreferred()) {
+          return;
+        }
+      }
+    }
+
+    try {
+      for (IndexGroup index : coveringIndexes) {
+        IndexProperties indexProps = index.getIndexProps().get(0);
+        IndexDescriptor indexDesc = indexProps.getIndexDesc();
+        IndexGroupScan idxScan = indexDesc.getIndexGroupScan();
+        FunctionalIndexInfo indexInfo = indexDesc.getFunctionalInfo();
+
+        indexCondition = indexProps.getLeadingColumnsFilter();
+        remainderCondition = indexProps.getTotalRemainderFilter();
+        //Copy primary table statistics to index table
+        idxScan.setStatistics(((DbGroupScan) scan.getGroupScan()).getStatistics());
+        logger.info("index_plan_info: Generating covering index plan for index: {}, query condition {}", indexDesc.getIndexName(), indexCondition.toString());
+
+        CoveringIndexPlanGenerator planGen = new CoveringIndexPlanGenerator(indexContext, indexInfo, idxScan,
+            indexCondition, remainderCondition, builder, settings);
+
+        planGen.go();
+      }
+    } catch (Exception e) {
+      logger.warn("Exception while trying to generate covering index plan", e);
+    }
+
+    // Create non-covering index plans.
+
+    //First, check if the primary table scan supports creating a restricted scan
+    if (primaryTableScan instanceof DbGroupScan &&
+        (((DbGroupScan) primaryTableScan).supportsRestrictedScan())) {
+      try {
+        for (IndexGroup index : nonCoveringIndexes) {
+          IndexProperties indexProps = index.getIndexProps().get(0);
+          IndexDescriptor indexDesc = indexProps.getIndexDesc();
+          IndexGroupScan idxScan = indexDesc.getIndexGroupScan();
+
+          indexCondition = indexProps.getLeadingColumnsFilter();
+          remainderCondition = indexProps.getTotalRemainderFilter();
+          //Copy primary table statistics to index table
+          idxScan.setStatistics(((DbGroupScan) primaryTableScan).getStatistics());
+          logger.info("index_plan_info: Generating non-covering index plan for index: {}, query condition {}", indexDesc.getIndexName(), indexCondition.toString());
+          NonCoveringIndexPlanGenerator planGen = new NonCoveringIndexPlanGenerator(indexContext, indexDesc,
+            idxScan, indexCondition, remainderCondition, builder, settings);
+          planGen.go();
+        }
+      } catch (Exception e) {
+        logger.warn("Exception while trying to generate non-covering index access plan", e);
+      }
+    }
+  }
+
+}
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/cost/DrillDefaultRelMetadataProvider.java b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/index/rules/MatchFunction.java
similarity index 55%
copy from exec/java-exec/src/main/java/org/apache/drill/exec/planner/cost/DrillDefaultRelMetadataProvider.java
copy to exec/java-exec/src/main/java/org/apache/drill/exec/planner/index/rules/MatchFunction.java
index 5c02e3d..46801c7 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/cost/DrillDefaultRelMetadataProvider.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/index/rules/MatchFunction.java
@@ -15,19 +15,11 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package org.apache.drill.exec.planner.cost;
+package org.apache.drill.exec.planner.index.rules;
 
-import org.apache.drill.shaded.guava.com.google.common.collect.ImmutableList;
-import org.apache.calcite.rel.metadata.ChainedRelMetadataProvider;
-import org.apache.calcite.rel.metadata.DefaultRelMetadataProvider;
-import org.apache.calcite.rel.metadata.RelMetadataProvider;
+import org.apache.calcite.plan.RelOptRuleCall;
 
-public class DrillDefaultRelMetadataProvider {
-  private DrillDefaultRelMetadataProvider() {
-  }
-
-  public static final RelMetadataProvider INSTANCE = ChainedRelMetadataProvider.of(ImmutableList
-      .of(DrillRelMdRowCount.SOURCE,
-          DrillRelMdDistinctRowCount.SOURCE,
-          DefaultRelMetadataProvider.INSTANCE));
-}
+public interface MatchFunction<T> {
+  boolean match(RelOptRuleCall call);
+  T onMatch(RelOptRuleCall call);
+}
\ No newline at end of file
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/logical/DrillMergeProjectRule.java b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/logical/DrillMergeProjectRule.java
index 6c151ed..c8c0db3 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/logical/DrillMergeProjectRule.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/logical/DrillMergeProjectRule.java
@@ -32,6 +32,9 @@ import org.apache.calcite.rel.core.RelFactories.ProjectFactory;
 import org.apache.calcite.rex.RexCall;
 import org.apache.calcite.rex.RexNode;
 import org.apache.drill.exec.planner.DrillRelBuilder;
+import org.apache.drill.exec.planner.physical.PrelFactories;
+
+import java.util.List;
 
 import java.util.ArrayList;
 import java.util.List;
@@ -166,4 +169,25 @@ public class DrillMergeProjectRule extends RelOptRule {
     return list;
   }
 
+  public static Project replace(Project topProject, Project bottomProject) {
+    final List<RexNode> newProjects =
+        RelOptUtil.pushPastProject(topProject.getProjects(), bottomProject);
+
+    // replace the two projects with a combined projection
+    if(topProject instanceof DrillProjectRel) {
+      RelNode newProjectRel = DrillRelFactories.DRILL_LOGICAL_PROJECT_FACTORY.createProject(
+          bottomProject.getInput(), newProjects,
+          topProject.getRowType().getFieldNames());
+
+      return (Project) newProjectRel;
+    }
+    else {
+      RelNode newProjectRel = PrelFactories.PROJECT_FACTORY.createProject(
+          bottomProject.getInput(), newProjects,
+          topProject.getRowType().getFieldNames());
+
+      return (Project) newProjectRel;
+    }
+  }
+
 }
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/logical/DrillOptiq.java b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/logical/DrillOptiq.java
index 571b130..477b03c 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/logical/DrillOptiq.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/logical/DrillOptiq.java
@@ -22,7 +22,7 @@ import java.util.GregorianCalendar;
 import java.util.LinkedList;
 import java.util.List;
 
-import org.apache.drill.shaded.guava.com.google.common.base.Preconditions;
+import org.apache.calcite.rel.type.RelDataType;
 import org.apache.drill.common.exceptions.UserException;
 import org.apache.drill.common.expression.ExpressionPosition;
 import org.apache.drill.common.expression.FieldReference;
@@ -42,6 +42,7 @@ import org.apache.drill.common.types.Types;
 import org.apache.drill.exec.planner.StarColumnHelper;
 import org.apache.calcite.rel.RelNode;
 import org.apache.calcite.rel.type.RelDataTypeField;
+import org.apache.calcite.rex.RexBuilder;
 import org.apache.calcite.rex.RexCall;
 import org.apache.calcite.rex.RexCorrelVariable;
 import org.apache.calcite.rex.RexDynamicParam;
@@ -94,17 +95,32 @@ public class DrillOptiq {
     final RexToDrill visitor = new RexToDrill(context, inputs);
     return expr.accept(visitor);
   }
+  public static LogicalExpression toDrill(DrillParseContext context, RelDataType type,
+                                          RexBuilder builder, RexNode expr) {
+    final RexToDrill visitor = new RexToDrill(context, type, builder);
+    return expr.accept(visitor);
+  }
 
-  private static class RexToDrill extends RexVisitorImpl<LogicalExpression> {
+  public static class RexToDrill extends RexVisitorImpl<LogicalExpression> {
     private final List<RelNode> inputs;
     private final DrillParseContext context;
     private final List<RelDataTypeField> fieldList;
+    private final RelDataType rowType;
+    private final RexBuilder builder;
 
     RexToDrill(DrillParseContext context, List<RelNode> inputs) {
       super(true);
       this.context = context;
       this.inputs = inputs;
       this.fieldList = Lists.newArrayList();
+      if (inputs.size() > 0 && inputs.get(0)!=null) {
+        this.rowType = inputs.get(0).getRowType();
+        this.builder = inputs.get(0).getCluster().getRexBuilder();
+      }
+      else {
+        this.rowType = null;
+        this.builder = null;
+      }
       /*
          Fields are enumerated by their presence order in input. Details {@link org.apache.calcite.rex.RexInputRef}.
          Thus we can merge field list from several inputs by adding them into the list in order of appearance.
@@ -124,12 +140,31 @@ public class DrillOptiq {
         }
       }
     }
+    public RexToDrill(DrillParseContext context, RelNode input) {
+      this(context, Lists.newArrayList(input));
+    }
+
+    public RexToDrill(DrillParseContext context, RelDataType rowType, RexBuilder builder) {
+      super(true);
+      this.context = context;
+      this.rowType = rowType;
+      this.builder = builder;
+      this.inputs = Lists.newArrayList();
+      this.fieldList = rowType.getFieldList();
+    }
+
+    protected RelDataType getRowType() {
+      return rowType;
+    }
+
+    protected RexBuilder getRexBuilder() {
+      return builder;
+    }
 
     @Override
     public LogicalExpression visitInputRef(RexInputRef inputRef) {
       final int index = inputRef.getIndex();
       final RelDataTypeField field = fieldList.get(index);
-      Preconditions.checkNotNull(field, "Unable to find field using input reference");
       return FieldReference.getWithQuotedRef(field.getName());
     }
 
@@ -161,7 +196,6 @@ public class DrillOptiq {
         }
         throw new AssertionError("todo: implement syntax " + syntax + "(" + call + ")");
       case PREFIX:
-        logger.debug("Prefix");
         LogicalExpression arg = call.getOperands().get(0).accept(this);
         switch(call.getKind()){
           case NOT:
@@ -174,7 +208,6 @@ public class DrillOptiq {
         }
         throw new AssertionError("todo: implement syntax " + syntax + "(" + call + ")");
       case SPECIAL:
-        logger.debug("Special");
         switch(call.getKind()){
         case CAST:
           return getDrillCastFunctionFromOptiq(call);
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/logical/DrillPushProjectIntoScanRule.java b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/logical/DrillPushProjectIntoScanRule.java
index ccf8c4d..db20cb7 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/logical/DrillPushProjectIntoScanRule.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/logical/DrillPushProjectIntoScanRule.java
@@ -17,9 +17,6 @@
  */
 package org.apache.drill.exec.planner.logical;
 
-import org.apache.drill.shaded.guava.com.google.common.collect.ImmutableList;
-import org.apache.drill.shaded.guava.com.google.common.collect.ImmutableMap;
-import org.apache.drill.shaded.guava.com.google.common.collect.Sets;
 import org.apache.calcite.adapter.enumerable.EnumerableTableScan;
 import org.apache.calcite.plan.RelOptRule;
 import org.apache.calcite.plan.RelOptRuleCall;
@@ -28,42 +25,33 @@ import org.apache.calcite.rel.core.Project;
 import org.apache.calcite.rel.core.TableScan;
 import org.apache.calcite.rel.logical.LogicalProject;
 import org.apache.calcite.rel.rules.ProjectRemoveRule;
-import org.apache.calcite.rel.type.RelDataType;
-import org.apache.calcite.rel.type.RelDataTypeFactory;
-import org.apache.calcite.rel.type.RelDataTypeField;
-import org.apache.calcite.rex.RexCall;
-import org.apache.calcite.rex.RexInputRef;
-import org.apache.calcite.rex.RexLiteral;
 import org.apache.calcite.rex.RexNode;
-import org.apache.calcite.rex.RexVisitorImpl;
-import org.apache.calcite.sql.fun.SqlStdOperatorTable;
 import org.apache.drill.common.exceptions.DrillRuntimeException;
-import org.apache.drill.common.expression.PathSegment;
-import org.apache.drill.common.expression.SchemaPath;
+import org.apache.drill.exec.planner.common.DrillRelOptUtil;
+import org.apache.drill.exec.planner.common.DrillRelOptUtil.ProjectPushInfo;
 import org.apache.drill.exec.util.Utilities;
 
 import java.io.IOException;
 import java.util.ArrayList;
-import java.util.HashMap;
-import java.util.LinkedHashMap;
 import java.util.List;
-import java.util.Map;
-import java.util.Set;
-
-import static org.apache.drill.exec.planner.logical.FieldsReWriterUtil.DesiredField;
-import static org.apache.drill.exec.planner.logical.FieldsReWriterUtil.FieldsReWriter;
 
 /**
  * When table support project push down, rule can be applied to reduce number of read columns
  * thus improving scan operator performance.
  */
 public class DrillPushProjectIntoScanRule extends RelOptRule {
-  public static final RelOptRule INSTANCE = new DrillPushProjectIntoScanRule(LogicalProject.class, EnumerableTableScan.class);
-
-  private DrillPushProjectIntoScanRule(Class<? extends Project> projectClass,
-                                       Class<? extends TableScan> scanClass) {
-    super(RelOptHelper.some(projectClass, RelOptHelper.any(scanClass)),
-        DrillRelFactories.LOGICAL_BUILDER, "DrillPushProjectIntoScanRule");
+  public static final RelOptRule INSTANCE =
+      new DrillPushProjectIntoScanRule(LogicalProject.class,
+          EnumerableTableScan.class,
+          "DrillPushProjIntoEnumerableScan");
+
+  public static final RelOptRule DRILL_LOGICAL_INSTANCE =
+      new DrillPushProjectIntoScanRule(LogicalProject.class,
+          DrillScanRel.class,
+          "DrillPushProjIntoDrillRelScan");
+
+  private DrillPushProjectIntoScanRule(Class<? extends Project> projectClass, Class<? extends TableScan> scanClass, String description) {
+    super(RelOptHelper.some(projectClass, RelOptHelper.any(scanClass)), description);
   }
 
   @Override
@@ -77,7 +65,7 @@ public class DrillPushProjectIntoScanRule extends RelOptRule {
         return;
       }
 
-      ProjectPushInfo projectPushInfo = getFieldsInformation(scan.getRowType(), project.getProjects());
+      ProjectPushInfo projectPushInfo = DrillRelOptUtil.getFieldsInformation(scan.getRowType(), project.getProjects());
       if (!canPushProjectIntoScan(scan.getTable(), projectPushInfo)) {
         return;
       }
@@ -124,133 +112,4 @@ public class DrillPushProjectIntoScanRule extends RelOptRule {
     return !Utilities.isStarQuery(projectPushInfo.getFields())
         && drillTable.getGroupScan().canPushdownProjects(projectPushInfo.getFields());
   }
-
-  private ProjectPushInfo getFieldsInformation(RelDataType rowType, List<RexNode> projects) {
-    ProjectFieldsVisitor fieldsVisitor = new ProjectFieldsVisitor(rowType);
-    for (RexNode exp : projects) {
-      PathSegment segment = exp.accept(fieldsVisitor);
-      fieldsVisitor.addField(segment);
-    }
-
-    return fieldsVisitor.getInfo();
-  }
-
-  /**
-   * Stores information about fields, their names and types.
-   * Is responsible for creating mapper which used in field re-writer visitor.
-   */
-  private static class ProjectPushInfo {
-    private final List<SchemaPath> fields;
-    private final FieldsReWriter reWriter;
-    private final List<String> fieldNames;
-    private final List<RelDataType> types;
-
-    ProjectPushInfo(List<SchemaPath> fields, Map<String, DesiredField> desiredFields) {
-      this.fields = fields;
-      this.fieldNames = new ArrayList<>();
-      this.types = new ArrayList<>();
-
-      Map<RexNode, Integer> mapper = new HashMap<>();
-
-      int index = 0;
-      for (Map.Entry<String, DesiredField> entry : desiredFields.entrySet()) {
-        fieldNames.add(entry.getKey());
-        DesiredField desiredField = entry.getValue();
-        types.add(desiredField.getType());
-        for (RexNode node : desiredField.getNodes()) {
-          mapper.put(node, index);
-        }
-        index++;
-      }
-      this.reWriter = new FieldsReWriter(mapper);
-    }
-
-    List<SchemaPath> getFields() {
-      return fields;
-    }
-
-    FieldsReWriter getInputReWriter() {
-      return reWriter;
-    }
-
-    /**
-     * Creates new row type based on stores types and field names.
-     *
-     * @param factory factory for data type descriptors.
-     * @return new row type
-     */
-    RelDataType createNewRowType(RelDataTypeFactory factory) {
-      return factory.createStructType(types, fieldNames);
-    }
-  }
-
-  /**
-   * Visitor that finds the set of inputs that are used.
-   */
-  private static class ProjectFieldsVisitor extends RexVisitorImpl<PathSegment> {
-    private final List<String> fieldNames;
-    private final List<RelDataTypeField> fields;
-
-    private final Set<SchemaPath> newFields = Sets.newLinkedHashSet();
-    private final Map<String, DesiredField> desiredFields = new LinkedHashMap<>();
-
-    ProjectFieldsVisitor(RelDataType rowType) {
-      super(true);
-      this.fieldNames = rowType.getFieldNames();
-      this.fields = rowType.getFieldList();
-    }
-
-    void addField(PathSegment segment) {
-      if (segment != null && segment instanceof PathSegment.NameSegment) {
-        newFields.add(new SchemaPath((PathSegment.NameSegment) segment));
-      }
-    }
-
-    ProjectPushInfo getInfo() {
-      return new ProjectPushInfo(ImmutableList.copyOf(newFields), ImmutableMap.copyOf(desiredFields));
-    }
-
-    @Override
-    public PathSegment visitInputRef(RexInputRef inputRef) {
-      int index = inputRef.getIndex();
-      String name = fieldNames.get(index);
-      RelDataTypeField field = fields.get(index);
-      addDesiredField(name, field.getType(), inputRef);
-      return new PathSegment.NameSegment(name);
-    }
-
-    @Override
-    public PathSegment visitCall(RexCall call) {
-      String itemStarFieldName = FieldsReWriterUtil.getFieldNameFromItemStarField(call, fieldNames);
-      if (itemStarFieldName != null) {
-        addDesiredField(itemStarFieldName, call.getType(), call);
-        return new PathSegment.NameSegment(itemStarFieldName);
-      }
-
-      if (SqlStdOperatorTable.ITEM.equals(call.getOperator())) {
-        PathSegment mapOrArray = call.operands.get(0).accept(this);
-        if (mapOrArray != null) {
-          if (call.operands.get(1) instanceof RexLiteral) {
-            return mapOrArray.cloneWithNewChild(Utilities.convertLiteral((RexLiteral) call.operands.get(1)));
-          }
-          return mapOrArray;
-        }
-      } else {
-        for (RexNode operand : call.operands) {
-          addField(operand.accept(this));
-        }
-      }
-      return null;
-    }
-
-    private void addDesiredField(String name, RelDataType type, RexNode originalNode) {
-      DesiredField desiredField = desiredFields.get(name);
-      if (desiredField == null) {
-        desiredFields.put(name, new DesiredField(name, type, originalNode));
-      } else {
-        desiredField.addNode(originalNode);
-      }
-    }
-  }
-
 }
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/logical/DrillScanRel.java b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/logical/DrillScanRel.java
index c0583b5..82658af 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/logical/DrillScanRel.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/logical/DrillScanRel.java
@@ -18,12 +18,11 @@
 package org.apache.drill.exec.planner.logical;
 
 import java.util.ArrayList;
-import java.io.IOException;
 import java.util.List;
 
+import org.apache.calcite.rel.RelWriter;
 import org.apache.calcite.rel.metadata.RelMetadataQuery;
 import org.apache.drill.common.JSONOptions;
-import org.apache.drill.common.exceptions.DrillRuntimeException;
 import org.apache.drill.common.expression.SchemaPath;
 import org.apache.drill.common.logical.data.LogicalOperator;
 import org.apache.drill.common.logical.data.Scan;
@@ -34,7 +33,6 @@ import org.apache.drill.exec.planner.cost.DrillCostBase.DrillCostFactory;
 import org.apache.drill.exec.planner.physical.PlannerSettings;
 import org.apache.drill.exec.planner.physical.PrelUtil;
 import org.apache.drill.exec.planner.torel.ConversionContext;
-import org.apache.calcite.rel.RelWriter;
 import org.apache.calcite.plan.RelOptCluster;
 import org.apache.calcite.plan.RelOptCost;
 import org.apache.calcite.plan.RelOptPlanner;
@@ -50,12 +48,10 @@ import org.apache.drill.exec.util.Utilities;
  */
 public class DrillScanRel extends DrillScanRelBase implements DrillRel {
   private final static int STAR_COLUMN_COST = 10000;
-
-  final private RelDataType rowType;
-  private GroupScan groupScan;
-  private List<SchemaPath> columns;
   private PlannerSettings settings;
+  private List<SchemaPath> columns;
   private final boolean partitionFilterPushdown;
+  final private RelDataType rowType;
 
   /** Creates a DrillScan. */
   public DrillScanRel(final RelOptCluster cluster, final RelTraitSet traits,
@@ -80,17 +76,12 @@ public class DrillScanRel extends DrillScanRelBase implements DrillRel {
   /** Creates a DrillScan. */
   public DrillScanRel(final RelOptCluster cluster, final RelTraitSet traits,
                       final RelOptTable table, final RelDataType rowType, final List<SchemaPath> columns, boolean partitionFilterPushdown) {
-    super(DRILL_LOGICAL, cluster, traits, table);
+    super(cluster, traits, table, columns);
     this.settings = PrelUtil.getPlannerSettings(cluster.getPlanner());
     this.rowType = rowType;
     Preconditions.checkNotNull(columns);
     this.columns = columns;
     this.partitionFilterPushdown = partitionFilterPushdown;
-    try {
-      this.groupScan = drillTable.getGroupScan().clone(this.columns);
-    } catch (final IOException e) {
-      throw new DrillRuntimeException("Failure creating scan.", e);
-    }
   }
 
   /** Creates a DrillScanRel for a particular GroupScan */
@@ -102,10 +93,9 @@ public class DrillScanRel extends DrillScanRelBase implements DrillRel {
   /** Creates a DrillScanRel for a particular GroupScan */
   public DrillScanRel(final RelOptCluster cluster, final RelTraitSet traits,
                       final RelOptTable table, final GroupScan groupScan, final RelDataType rowType, final List<SchemaPath> columns, boolean partitionFilterPushdown) {
-    super(DRILL_LOGICAL, cluster, traits, table);
+    super(cluster, traits, groupScan, table);
     this.rowType = rowType;
     this.columns = columns;
-    this.groupScan = groupScan;
     this.settings = PrelUtil.getPlannerSettings(cluster.getPlanner());
     this.partitionFilterPushdown = partitionFilterPushdown;
   }
@@ -144,12 +134,12 @@ public class DrillScanRel extends DrillScanRelBase implements DrillRel {
 
   @Override
   public RelWriter explainTerms(RelWriter pw) {
-    return super.explainTerms(pw).item("groupscan", groupScan.getDigest());
+    return super.explainTerms(pw).item("groupscan", getGroupScan().getDigest());
   }
 
   @Override
   public double estimateRowCount(RelMetadataQuery mq) {
-    return this.groupScan.getScanStats(settings).getRecordCount();
+    return getGroupScan().getScanStats(settings).getRecordCount();
   }
 
   /// TODO: this method is same as the one for ScanPrel...eventually we should consolidate
@@ -157,7 +147,7 @@ public class DrillScanRel extends DrillScanRelBase implements DrillRel {
   /// by both logical and physical rels.
   @Override
   public RelOptCost computeSelfCost(final RelOptPlanner planner, RelMetadataQuery mq) {
-    final ScanStats stats = groupScan.getScanStats(settings);
+    final ScanStats stats = getGroupScan().getScanStats(settings);
     int columnCount = getRowType().getFieldCount();
     double ioCost = 0;
     boolean isStarQuery = Utilities.isStarQuery(columns);
@@ -176,21 +166,12 @@ public class DrillScanRel extends DrillScanRelBase implements DrillRel {
       return planner.getCostFactory().makeCost(rowCount * columnCount, stats.getCpuCost(), stats.getDiskCost());
     }
 
-    double cpuCost = rowCount * columnCount; // for now, assume cpu cost is proportional to row count.
-    // Even though scan is reading from disk, in the currently generated plans all plans will
-    // need to read the same amount of data, so keeping the disk io cost 0 is ok for now.
-    // In the future we might consider alternative scans that go against projections or
-    // different compression schemes etc that affect the amount of data read. Such alternatives
-    // would affect both cpu and io cost.
+    double cpuCost = rowCount * columnCount; // for now, assume cpu cost is proportional to row count and number of columns
 
     DrillCostFactory costFactory = (DrillCostFactory)planner.getCostFactory();
     return costFactory.makeCost(rowCount, cpuCost, ioCost, 0);
   }
 
-  public GroupScan getGroupScan() {
-    return groupScan;
-  }
-
   public boolean partitionFilterPushdown() {
     return this.partitionFilterPushdown;
   }
diff --git a/logical/src/main/java/org/apache/drill/common/logical/StoragePluginConfig.java b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/physical/AbstractRangePartitionFunction.java
similarity index 55%
copy from logical/src/main/java/org/apache/drill/common/logical/StoragePluginConfig.java
copy to exec/java-exec/src/main/java/org/apache/drill/exec/planner/physical/AbstractRangePartitionFunction.java
index 49335f6..09fdb57 100644
--- a/logical/src/main/java/org/apache/drill/common/logical/StoragePluginConfig.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/physical/AbstractRangePartitionFunction.java
@@ -15,46 +15,30 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package org.apache.drill.common.logical;
+package org.apache.drill.exec.planner.physical;
 
+import java.util.List;
+
+import org.apache.drill.common.expression.FieldReference;
+import org.apache.drill.exec.record.VectorWrapper;
 
 import com.fasterxml.jackson.annotation.JsonIgnore;
 import com.fasterxml.jackson.annotation.JsonTypeInfo;
 
+@JsonTypeInfo(use=JsonTypeInfo.Id.CLASS, include=JsonTypeInfo.As.PROPERTY, property="@class")
+public abstract class AbstractRangePartitionFunction  implements PartitionFunction {
+  public static final String RANGE_PARTITION_EXPR_NAME = "R_A_N_G_E_E_X_P_R";
 
-@JsonTypeInfo(use = JsonTypeInfo.Id.NAME, property = "type")
-public abstract class StoragePluginConfig{
-
-  private Boolean enabled;
+  public abstract List<FieldReference> getPartitionRefList();
 
-  /**
-   * Check for enabled status of the plugin
-   *
-   * @return true, when enabled. False, when disabled or status is absent
-   */
-  public boolean isEnabled() {
-    return enabled != null && enabled;
-  }
+  public abstract void setup(List<VectorWrapper<?>> partitionKeys);
 
+  public abstract int eval(int index, int numPartitions);
 
-  public void setEnabled(Boolean enabled) {
-    this.enabled = enabled;
-  }
-
-  /**
-   * Allows to check whether the enabled status is present in config
-   *
-   * @return true if enabled status is present, false otherwise
-   */
   @JsonIgnore
-  public boolean isEnabledStatusPresent() {
-    return enabled != null;
-  }
-
   @Override
-  public abstract boolean equals(Object o);
-
-  @Override
-  public abstract int hashCode();
+  public FieldReference getPartitionFieldRef() {
+    return new FieldReference(RANGE_PARTITION_EXPR_NAME);
+  }
 
 }
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/physical/BroadcastExchangePrel.java b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/physical/BroadcastExchangePrel.java
index caec426..1ae375b 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/physical/BroadcastExchangePrel.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/physical/BroadcastExchangePrel.java
@@ -57,7 +57,9 @@ public class BroadcastExchangePrel extends ExchangePrel{
 
     final int  rowWidth = child.getRowType().getFieldCount() * DrillCostBase.AVG_FIELD_WIDTH;
     final double cpuCost = broadcastFactor * DrillCostBase.SVR_CPU_COST * inputRows;
-    final double networkCost = broadcastFactor * DrillCostBase.BYTE_NETWORK_COST * inputRows * rowWidth * numEndPoints;
+
+    //we assume localhost network cost is 1/10 of regular network cost
+    final double networkCost = broadcastFactor * DrillCostBase.BYTE_NETWORK_COST * inputRows * rowWidth * (numEndPoints - 0.9);
 
     return new DrillCostBase(inputRows, cpuCost, 0, networkCost);
   }
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/physical/ConvertCountToDirectScan.java b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/physical/ConvertCountToDirectScan.java
index f8cda8c..5204495 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/physical/ConvertCountToDirectScan.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/physical/ConvertCountToDirectScan.java
@@ -128,9 +128,8 @@ public class ConvertCountToDirectScan extends Prule {
     final ScanStats scanStats = new ScanStats(ScanStats.GroupScanProperty.EXACT_ROW_COUNT, 1, 1, scanRowType.getFieldCount());
     final GroupScan directScan = new MetadataDirectGroupScan(reader, oldGrpScan.getFiles(), scanStats);
 
-    final ScanPrel newScan = new ScanPrel(scan,
-        scan.getTraitSet().plus(Prel.DRILL_PHYSICAL).plus(DrillDistributionTrait.SINGLETON), directScan,
-        scanRowType);
+    final ScanPrel newScan = new ScanPrel(scan, scan.getTraitSet().plus(Prel.DRILL_PHYSICAL).plus(DrillDistributionTrait.SINGLETON), 
+        directScan, scanRowType);
 
     final ProjectPrel newProject = new ProjectPrel(agg.getCluster(), agg.getTraitSet().plus(Prel.DRILL_PHYSICAL)
         .plus(DrillDistributionTrait.SINGLETON), newScan, prepareFieldExpressions(scanRowType), agg.getRowType());
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/physical/ScanPrel.java b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/physical/DirectScanPrel.java
similarity index 62%
copy from exec/java-exec/src/main/java/org/apache/drill/exec/planner/physical/ScanPrel.java
copy to exec/java-exec/src/main/java/org/apache/drill/exec/planner/physical/DirectScanPrel.java
index ff901c9..d032255 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/physical/ScanPrel.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/physical/DirectScanPrel.java
@@ -17,57 +17,54 @@
  */
 package org.apache.drill.exec.planner.physical;
 
-import java.io.IOException;
-import java.util.Collections;
-import java.util.Iterator;
-import java.util.List;
-
+import org.apache.calcite.plan.RelOptCluster;
+import org.apache.calcite.plan.RelOptCost;
+import org.apache.calcite.plan.RelOptPlanner;
+import org.apache.calcite.plan.RelTraitSet;
+import org.apache.calcite.rel.AbstractRelNode;
+import org.apache.calcite.rel.RelNode;
+import org.apache.calcite.rel.RelWriter;
 import org.apache.calcite.rel.metadata.RelMetadataQuery;
+import org.apache.calcite.rel.type.RelDataType;
 import org.apache.drill.common.exceptions.DrillRuntimeException;
 import org.apache.drill.common.exceptions.ExecutionSetupException;
 import org.apache.drill.exec.physical.base.GroupScan;
 import org.apache.drill.exec.physical.base.PhysicalOperator;
 import org.apache.drill.exec.physical.base.ScanStats;
-import org.apache.drill.exec.planner.cost.DrillCostBase.DrillCostFactory;
+import org.apache.drill.exec.planner.cost.DrillCostBase;
 import org.apache.drill.exec.planner.fragment.DistributionAffinity;
 import org.apache.drill.exec.planner.physical.visitor.PrelVisitor;
-import org.apache.drill.exec.record.BatchSchema.SelectionVectorMode;
-import org.apache.calcite.rel.AbstractRelNode;
-import org.apache.calcite.rel.RelNode;
-import org.apache.calcite.rel.RelWriter;
-import org.apache.calcite.plan.RelOptCluster;
-import org.apache.calcite.plan.RelOptCost;
-import org.apache.calcite.plan.RelOptPlanner;
-import org.apache.calcite.plan.RelTraitSet;
-import org.apache.calcite.rel.type.RelDataType;
+import org.apache.drill.exec.record.BatchSchema;
+
+import java.io.IOException;
+import java.util.Collections;
+import java.util.Iterator;
+import java.util.List;
 
-public class ScanPrel extends AbstractRelNode implements DrillScanPrel {
+public class DirectScanPrel extends AbstractRelNode implements Prel, HasDistributionAffinity {
   static final org.slf4j.Logger logger = org.slf4j.LoggerFactory
-      .getLogger(ScanPrel.class);
+          .getLogger(DirectScanPrel.class);
 
-  protected final GroupScan groupScan;
+  private final GroupScan groupScan;
   private final RelDataType rowType;
 
-  public ScanPrel(RelNode old, RelTraitSet traitSets, GroupScan scan, RelDataType rowType) {
-    this(old.getCluster(), traitSets, scan, rowType);
-  }
-
-  public ScanPrel(RelOptCluster cluster, RelTraitSet traits, GroupScan groupScan, RelDataType rowType) {
+  DirectScanPrel(RelOptCluster cluster, RelTraitSet traits,
+                 GroupScan groupScan, RelDataType rowType) {
     super(cluster, traits);
-    this.groupScan = getCopy(groupScan);
+    this.groupScan = groupScan;
     this.rowType = rowType;
   }
 
   @Override
   public RelNode copy(RelTraitSet traitSet, List<RelNode> inputs) {
-    return new ScanPrel(this.getCluster(), traitSet, groupScan,
-        this.rowType);
+    return new DirectScanPrel(this.getCluster(), traitSet, this.getGroupScan(),
+            this.rowType);
   }
 
   @Override
   protected Object clone() throws CloneNotSupportedException {
-    return new ScanPrel(this.getCluster(), this.getTraitSet(), getCopy(groupScan),
-        this.rowType);
+    return new DirectScanPrel(this.getCluster(), this.getTraitSet(), getCopy(this.getGroupScan()),
+            this.rowType);
   }
 
   private static GroupScan getCopy(GroupScan scan){
@@ -80,18 +77,42 @@ public class ScanPrel extends AbstractRelNode implements DrillScanPrel {
 
   @Override
   public PhysicalOperator getPhysicalOperator(PhysicalPlanCreator creator)
-      throws IOException {
-    return creator.addMetadata(this, groupScan);
+          throws IOException {
+    return creator.addMetadata(this, this.getGroupScan());
   }
 
-  @Override
   public GroupScan getGroupScan() {
     return groupScan;
   }
 
   @Override
-  public RelWriter explainTerms(RelWriter pw) {
-    return super.explainTerms(pw).item("groupscan", groupScan.getDigest());
+  public boolean needsFinalColumnReordering() {
+    return false;
+  }
+
+  @Override
+  public BatchSchema.SelectionVectorMode[] getSupportedEncodings() {
+    return BatchSchema.SelectionVectorMode.DEFAULT;
+  }
+
+  @Override
+  public DistributionAffinity getDistributionAffinity() {
+    return this.getGroupScan().getDistributionAffinity();
+  }
+
+  @Override
+  public BatchSchema.SelectionVectorMode getEncoding() {
+    return BatchSchema.SelectionVectorMode.NONE;
+  }
+
+  @Override
+  public <T, X, E extends Throwable> T accept(PrelVisitor<T, X, E> logicalVisitor, X value) throws E {
+    return logicalVisitor.visitPrel(this, value);
+  }
+
+  @Override
+  public Iterator<Prel> iterator() {
+    return Collections.emptyIterator();
   }
 
   @Override
@@ -100,71 +121,42 @@ public class ScanPrel extends AbstractRelNode implements DrillScanPrel {
   }
 
   @Override
+  public RelWriter explainTerms(RelWriter pw) {
+    return super.explainTerms(pw).item("groupscan", this.getGroupScan().getDigest());
+  }
+
+  @Override
   public double estimateRowCount(RelMetadataQuery mq) {
     final PlannerSettings settings = PrelUtil.getPlannerSettings(getCluster());
-    return this.groupScan.getScanStats(settings).getRecordCount();
+
+    double rowCount = this.getGroupScan().getScanStats(settings).getRecordCount();
+    logger.debug("#{}.estimateRowCount get rowCount {} from  groupscan {}",
+            this.getId(), rowCount, System.identityHashCode(this.getGroupScan()));
+    return rowCount;
   }
 
   @Override
   public RelOptCost computeSelfCost(final RelOptPlanner planner, RelMetadataQuery mq) {
     final PlannerSettings settings = PrelUtil.getPlannerSettings(planner);
-    final ScanStats stats = this.groupScan.getScanStats(settings);
+    final ScanStats stats = this.getGroupScan().getScanStats(settings);
     final int columnCount = this.getRowType().getFieldCount();
 
     if(PrelUtil.getSettings(getCluster()).useDefaultCosting()) {
       return planner.getCostFactory().makeCost(stats.getRecordCount() * columnCount, stats.getCpuCost(), stats.getDiskCost());
     }
 
-    // double rowCount = RelMetadataQuery.getRowCount(this);
     double rowCount = stats.getRecordCount();
 
-    // As DRILL-4083 points out, when columnCount == 0, cpuCost becomes zero,
-    // which makes the costs of HiveScan and HiveDrillNativeParquetScan the same
-    double cpuCost = rowCount * Math.max(columnCount, 1); // For now, assume cpu cost is proportional to row count.
 
-    // If a positive value for CPU cost is given multiply the default CPU cost by given CPU cost.
+    double cpuCost = rowCount * Math.max(columnCount, 1);
+
     if (stats.getCpuCost() > 0) {
       cpuCost *= stats.getCpuCost();
     }
 
-    // Even though scan is reading from disk, in the currently generated plans all plans will
-    // need to read the same amount of data, so keeping the disk io cost 0 is ok for now.
-    // In the future we might consider alternative scans that go against projections or
-    // different compression schemes etc that affect the amount of data read. Such alternatives
-    // would affect both cpu and io cost.
-    double ioCost = 0;
-    DrillCostFactory costFactory = (DrillCostFactory)planner.getCostFactory();
+    double ioCost = stats.getDiskCost();
+    DrillCostBase.DrillCostFactory costFactory = (DrillCostBase.DrillCostFactory)planner.getCostFactory();
     return costFactory.makeCost(rowCount, cpuCost, ioCost, 0);
   }
 
-  @Override
-  public Iterator<Prel> iterator() {
-    return Collections.emptyIterator();
-  }
-
-  @Override
-  public <T, X, E extends Throwable> T accept(PrelVisitor<T, X, E> logicalVisitor, X value) throws E {
-    return logicalVisitor.visitScan(this, value);
-  }
-
-  @Override
-  public SelectionVectorMode[] getSupportedEncodings() {
-    return SelectionVectorMode.DEFAULT;
-  }
-
-  @Override
-  public SelectionVectorMode getEncoding() {
-    return SelectionVectorMode.NONE;
-  }
-
-  @Override
-  public boolean needsFinalColumnReordering() {
-    return true;
-  }
-
-  @Override
-  public DistributionAffinity getDistributionAffinity() {
-    return groupScan.getDistributionAffinity();
-  }
-
 }
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/physical/DirectScanPrule.java b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/physical/DirectScanPrule.java
index 5e64fc6..19e9fa7 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/physical/DirectScanPrule.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/physical/DirectScanPrule.java
@@ -36,13 +36,9 @@ public class DirectScanPrule extends Prule {
     final DrillDirectScanRel scan = call.rel(0);
     final RelTraitSet traits = scan.getTraitSet().plus(Prel.DRILL_PHYSICAL);
 
-    final ScanPrel newScan = new ScanPrel(scan.getCluster(), traits, scan.getGroupScan(), scan.getRowType()) {
-      // direct scan (no execution) => no accidental column shuffling => no reordering
-      @Override
-      public boolean needsFinalColumnReordering() {
-        return false;
-      }
-    };
+    final DirectScanPrel newScan = new DirectScanPrel(scan.getCluster(), traits, scan.getGroupScan(), scan.getRowType());
+
+    call.transformTo(newScan);
 
     call.transformTo(newScan);
   }
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/physical/DrillDistributionTrait.java b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/physical/DrillDistributionTrait.java
index ff7fbb9..5683c77 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/physical/DrillDistributionTrait.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/physical/DrillDistributionTrait.java
@@ -35,6 +35,7 @@ public class DrillDistributionTrait implements RelTrait {
 
   private DistributionType type;
   private final ImmutableList<DistributionField> fields;
+  private PartitionFunction partitionFunction = null;
 
   public DrillDistributionTrait(DistributionType type) {
     assert (type == DistributionType.SINGLETON || type == DistributionType.RANDOM_DISTRIBUTED || type == DistributionType.ANY
@@ -49,6 +50,14 @@ public class DrillDistributionTrait implements RelTrait {
     this.fields = fields;
   }
 
+  public DrillDistributionTrait(DistributionType type, ImmutableList<DistributionField> fields,
+      PartitionFunction partitionFunction) {
+    assert (type == DistributionType.HASH_DISTRIBUTED || type == DistributionType.RANGE_DISTRIBUTED);
+    this.type = type;
+    this.fields = fields;
+    this.partitionFunction = partitionFunction;
+  }
+
   @Override
   public void register(RelOptPlanner planner) {
   }
@@ -77,6 +86,12 @@ public class DrillDistributionTrait implements RelTrait {
           return true; // hash distribution subsumes random distribution and ANY distribution
         }
       }
+
+      if(this.type == DistributionType.RANGE_DISTRIBUTED) {
+        if (requiredDist == DistributionType.RANDOM_DISTRIBUTED) {
+          return true; // RANGE_DISTRIBUTED distribution subsumes random distribution and ANY distribution
+        }
+      }
     }
 
     return this.equals(trait);
@@ -94,6 +109,19 @@ public class DrillDistributionTrait implements RelTrait {
     return fields;
   }
 
+  public PartitionFunction getPartitionFunction() {
+    return partitionFunction;
+  }
+
+  private boolean arePartitionFunctionsSame(PartitionFunction f1, PartitionFunction f2) {
+    if (f1 != null && f2 != null) {
+      return f1.equals(f2);
+    } else if (f2 == null && f2 == null) {
+      return true;
+    }
+    return false;
+  }
+
   @Override
   public int hashCode() {
     return  fields == null? type.hashCode(): type.hashCode() | fields.hashCode() << 4;
@@ -106,7 +134,8 @@ public class DrillDistributionTrait implements RelTrait {
     }
     if (obj instanceof DrillDistributionTrait) {
       DrillDistributionTrait that = (DrillDistributionTrait) obj;
-      return this.type == that.type && this.fields.equals(that.fields);
+      return this.type == that.type && this.fields.equals(that.fields) &&
+          arePartitionFunctionsSame(this.partitionFunction, that.partitionFunction);
     }
     return false;
   }
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/physical/DrillDistributionTraitDef.java b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/physical/DrillDistributionTraitDef.java
index 94f1b61..0626483 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/physical/DrillDistributionTraitDef.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/physical/DrillDistributionTraitDef.java
@@ -86,7 +86,10 @@ public class DrillDistributionTraitDef extends RelTraitDef<DrillDistributionTrai
         return new HashToRandomExchangePrel(rel.getCluster(), planner.emptyTraitSet().plus(Prel.DRILL_PHYSICAL).plus(toDist), rel,
                                              toDist.getFields());
       case RANGE_DISTRIBUTED:
-        return new OrderedPartitionExchangePrel(rel.getCluster(), planner.emptyTraitSet().plus(Prel.DRILL_PHYSICAL).plus(toDist), rel);
+        // return new OrderedPartitionExchangePrel(rel.getCluster(), planner.emptyTraitSet().plus(Prel.DRILL_PHYSICAL).plus(toDist), rel);
+        return new RangePartitionExchangePrel(rel.getCluster(),
+            planner.emptyTraitSet().plus(Prel.DRILL_PHYSICAL).plus(toDist), rel,
+            toDist.getFields(), toDist.getPartitionFunction());
       case BROADCAST_DISTRIBUTED:
         return new BroadcastExchangePrel(rel.getCluster(), planner.emptyTraitSet().plus(Prel.DRILL_PHYSICAL).plus(toDist), rel);
       case ANY:
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/physical/FilterPrel.java b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/physical/FilterPrel.java
index 33c2944..c65f4fa 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/physical/FilterPrel.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/physical/FilterPrel.java
@@ -36,7 +36,7 @@ import org.apache.calcite.rex.RexNode;
 
 
 public class FilterPrel extends DrillFilterRelBase implements Prel {
-  protected FilterPrel(RelOptCluster cluster, RelTraitSet traits, RelNode child, RexNode condition) {
+  public FilterPrel(RelOptCluster cluster, RelTraitSet traits, RelNode child, RexNode condition) {
     super(Prel.DRILL_PHYSICAL, cluster, traits, child, condition);
   }
 
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/physical/HashJoinPrel.java b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/physical/HashJoinPrel.java
index f874781..0e1fc4e 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/physical/HashJoinPrel.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/physical/HashJoinPrel.java
@@ -23,10 +23,12 @@ import java.util.List;
 import org.apache.calcite.rel.core.Join;
 import org.apache.calcite.rel.metadata.RelMetadataQuery;
 import org.apache.drill.common.logical.data.JoinCondition;
+
 import org.apache.drill.exec.physical.base.PhysicalOperator;
 import org.apache.drill.exec.physical.config.HashJoinPOP;
 import org.apache.drill.exec.physical.impl.join.JoinUtils;
 import org.apache.drill.exec.physical.impl.join.JoinUtils.JoinCategory;
+import org.apache.drill.exec.planner.common.JoinControl;
 import org.apache.drill.exec.record.BatchSchema.SelectionVectorMode;
 import org.apache.calcite.rel.InvalidRelException;
 import org.apache.calcite.rel.core.JoinRelType;
@@ -44,24 +46,30 @@ public class HashJoinPrel  extends JoinPrel {
 
   private boolean swapped = false;
   private RuntimeFilterDef runtimeFilterDef;
+  protected boolean isRowKeyJoin = false;
+  private int joinControl;
 
   public HashJoinPrel(RelOptCluster cluster, RelTraitSet traits, RelNode left, RelNode right, RexNode condition,
                       JoinRelType joinType) throws InvalidRelException {
-    this(cluster, traits, left, right, condition, joinType, false, null);
+    this(cluster, traits, left, right, condition, joinType, false, null, false, JoinControl.DEFAULT);
   }
 
   public HashJoinPrel(RelOptCluster cluster, RelTraitSet traits, RelNode left, RelNode right, RexNode condition,
-      JoinRelType joinType, boolean swapped, RuntimeFilterDef runtimeFilterDef) throws InvalidRelException {
+      JoinRelType joinType, boolean swapped, RuntimeFilterDef runtimeFilterDef,
+      boolean isRowKeyJoin, int joinControl) throws InvalidRelException {
     super(cluster, traits, left, right, condition, joinType);
     this.swapped = swapped;
+    this.isRowKeyJoin = isRowKeyJoin;
     joincategory = JoinUtils.getJoinCategory(left, right, condition, leftKeys, rightKeys, filterNulls);
     this.runtimeFilterDef = runtimeFilterDef;
+    this.joinControl = joinControl;
   }
 
   @Override
   public Join copy(RelTraitSet traitSet, RexNode conditionExpr, RelNode left, RelNode right, JoinRelType joinType, boolean semiJoinDone) {
     try {
-      return new HashJoinPrel(this.getCluster(), traitSet, left, right, conditionExpr, joinType, this.swapped, this.runtimeFilterDef);
+      return new HashJoinPrel(this.getCluster(), traitSet, left, right, conditionExpr, joinType, this.swapped, this.runtimeFilterDef,
+          this.isRowKeyJoin, this.joinControl);
     }catch (InvalidRelException e) {
       throw new AssertionError(e);
     }
@@ -83,9 +91,9 @@ public class HashJoinPrel  extends JoinPrel {
     // Depending on whether the left/right is swapped for hash inner join, pass in different
     // combinations of parameters.
     if (! swapped) {
-      return getHashJoinPop(creator, left, right, leftKeys, rightKeys);
+      return getHashJoinPop(creator, left, right, leftKeys, rightKeys, isRowKeyJoin, joinControl);
     } else {
-      return getHashJoinPop(creator, right, left, rightKeys, leftKeys);
+      return getHashJoinPop(creator, right, left, rightKeys, leftKeys, isRowKeyJoin, joinControl);
     }
   }
 
@@ -100,7 +108,8 @@ public class HashJoinPrel  extends JoinPrel {
   }
 
   private PhysicalOperator getHashJoinPop(PhysicalPlanCreator creator, RelNode left, RelNode right,
-                                          List<Integer> leftKeys, List<Integer> rightKeys) throws IOException{
+                                          List<Integer> leftKeys, List<Integer> rightKeys,
+                                          boolean isRowKeyJoin, int htControl) throws IOException{
     final List<String> fields = getRowType().getFieldNames();
     assert isUnique(fields);
 
@@ -117,7 +126,7 @@ public class HashJoinPrel  extends JoinPrel {
     buildJoinConditions(conditions, leftFields, rightFields, leftKeys, rightKeys);
 
     RuntimeFilterDef runtimeFilterDef = this.getRuntimeFilterDef();
-    HashJoinPOP hjoin = new HashJoinPOP(leftPop, rightPop, conditions, jtype, runtimeFilterDef);
+    HashJoinPOP hjoin = new HashJoinPOP(leftPop, rightPop, conditions, jtype, runtimeFilterDef, isRowKeyJoin, htControl);
     return creator.addMetadata(this, hjoin);
   }
 
@@ -137,5 +146,8 @@ public class HashJoinPrel  extends JoinPrel {
     this.runtimeFilterDef = runtimeFilterDef;
   }
 
+  public boolean isRowKeyJoin() {
+    return this.isRowKeyJoin;
+  }
 
 }
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/physical/LimitUnionExchangeTransposeRule.java b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/physical/LimitExchangeTransposeRule.java
similarity index 51%
rename from exec/java-exec/src/main/java/org/apache/drill/exec/planner/physical/LimitUnionExchangeTransposeRule.java
rename to exec/java-exec/src/main/java/org/apache/drill/exec/planner/physical/LimitExchangeTransposeRule.java
index 6412357..817d6b3 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/physical/LimitUnionExchangeTransposeRule.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/physical/LimitExchangeTransposeRule.java
@@ -17,8 +17,11 @@
  */
 package org.apache.drill.exec.planner.physical;
 
+import org.apache.drill.shaded.guava.com.google.common.collect.ImmutableList;
 import org.apache.calcite.plan.RelOptRule;
 import org.apache.calcite.plan.RelOptRuleCall;
+import org.apache.calcite.plan.hep.HepRelVertex;
+import org.apache.calcite.plan.volcano.RelSubset;
 import org.apache.calcite.rel.RelNode;
 import org.apache.calcite.rex.RexLiteral;
 import org.apache.calcite.rex.RexNode;
@@ -26,29 +29,71 @@ import org.apache.drill.exec.planner.logical.RelOptHelper;
 
 import java.math.BigDecimal;
 
-public class LimitUnionExchangeTransposeRule extends Prule{
-  public static final RelOptRule INSTANCE = new LimitUnionExchangeTransposeRule();
+public class LimitExchangeTransposeRule extends Prule{
+  public static final RelOptRule INSTANCE = new LimitExchangeTransposeRule();
 
-  private LimitUnionExchangeTransposeRule() {
-    super(RelOptHelper.some(LimitPrel.class, RelOptHelper.any(UnionExchangePrel.class)), "LimitUnionExchangeTransposeRule");
+  private boolean findRowKeyJoin(RelNode rel) {
+    if (rel instanceof RowKeyJoinPrel) {
+      return true;
+    }
+    if (rel instanceof RelSubset) {
+      if (((RelSubset) rel).getBest() != null) {
+        if (findRowKeyJoin(((RelSubset) rel).getBest())) {
+          return true;
+        }
+      } else if (((RelSubset) rel).getOriginal() != null) {
+        if (findRowKeyJoin(((RelSubset) rel).getOriginal())) {
+          return true;
+        }
+      }
+    } else if (rel instanceof HepRelVertex) {
+      if (((HepRelVertex) rel).getCurrentRel() != null) {
+        if (findRowKeyJoin(((HepRelVertex) rel).getCurrentRel())) {
+          return true;
+        }
+      }
+    } else {
+      for (RelNode child : rel.getInputs()) {
+        if (findRowKeyJoin(child)) {
+          return true;
+        }
+      }
+    }
+    return false;
+  }
+
+  private LimitExchangeTransposeRule() {
+    super(RelOptHelper.some(LimitPrel.class, RelOptHelper.any(ExchangePrel.class)), "LimitExchangeTransposeRule");
   }
 
   @Override
   public boolean matches(RelOptRuleCall call) {
     final LimitPrel limit = (LimitPrel) call.rel(0);
+    final ExchangePrel exchange = (ExchangePrel) call.rel(1);
+
+    //this rule now works for two exchanges only: UnionExchangePrel and SingleMergeExchangePrel
+    if (!(exchange instanceof UnionExchangePrel) && !(exchange instanceof SingleMergeExchangePrel)) {
+      return false;
+    }
 
     // Two situations we do not fire this rule:
     // 1) limit has been pushed down to its child,
     // 2) the fetch() is null (indicating we have to fetch all the remaining rows starting from offset.
-    return !limit.isPushDown() && limit.getFetch() != null;
+    if (!limit.isPushDown() && limit.getFetch() != null) {
+      if (!(exchange instanceof SingleMergeExchangePrel)
+          || !findRowKeyJoin(exchange)) {
+        return true;
+      }
+    }
+    return false;
   }
 
   @Override
   public void onMatch(RelOptRuleCall call) {
     final LimitPrel limit = (LimitPrel) call.rel(0);
-    final UnionExchangePrel unionExchangePrel = (UnionExchangePrel) call.rel(1);
+    final ExchangePrel exchangePrel = (ExchangePrel) call.rel(1);
 
-    RelNode child = unionExchangePrel.getInput();
+    RelNode child = exchangePrel.getInput();
 
     final int offset = limit.getOffset() != null ? Math.max(0, RexLiteral.intValue(limit.getOffset())) : 0;
     final int fetch = Math.max(0, RexLiteral.intValue(limit.getFetch()));
@@ -57,8 +102,8 @@ public class LimitUnionExchangeTransposeRule extends Prule{
     final RexNode childFetch = limit.getCluster().getRexBuilder().makeExactLiteral(BigDecimal.valueOf(offset + fetch));
 
     final RelNode limitUnderExchange = new LimitPrel(child.getCluster(), child.getTraitSet(), child, null, childFetch);
-    final RelNode newUnionExch = new UnionExchangePrel(unionExchangePrel.getCluster(), unionExchangePrel.getTraitSet(), limitUnderExchange);
-    final RelNode limitAboveExchange = new LimitPrel(limit.getCluster(), limit.getTraitSet(), newUnionExch, limit.getOffset(), limit.getFetch(), true);
+    final RelNode newExch = exchangePrel.copy(exchangePrel.getTraitSet(), ImmutableList.of(limitUnderExchange));
+    final RelNode limitAboveExchange = new LimitPrel(limit.getCluster(), limit.getTraitSet(), newExch, limit.getOffset(), limit.getFetch(), true);
 
     call.transformTo(limitAboveExchange);
   }
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/physical/PlannerSettings.java b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/physical/PlannerSettings.java
index 9dae286..63f884c 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/physical/PlannerSettings.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/physical/PlannerSettings.java
@@ -115,6 +115,28 @@ public class PlannerSettings implements Context{
   public static final String UNIONALL_DISTRIBUTE_KEY = "planner.enable_unionall_distribute";
   public static final BooleanValidator UNIONALL_DISTRIBUTE = new BooleanValidator(UNIONALL_DISTRIBUTE_KEY, null);
 
+  // ------------------------------------------- Index planning related options BEGIN --------------------------------------------------------------
+  public static final String USE_SIMPLE_OPTIMIZER_KEY = "planner.use_simple_optimizer";
+  public static final BooleanValidator USE_SIMPLE_OPTIMIZER = new BooleanValidator(USE_SIMPLE_OPTIMIZER_KEY, null);
+  public static final BooleanValidator INDEX_PLANNING = new BooleanValidator("planner.enable_index_planning", null);
+  public static final BooleanValidator ENABLE_STATS = new BooleanValidator("planner.enable_statistics", null);
+  public static final BooleanValidator DISABLE_FULL_TABLE_SCAN = new BooleanValidator("planner.disable_full_table_scan", null);
+  public static final RangeLongValidator INDEX_MAX_CHOSEN_INDEXES_PER_TABLE = new RangeLongValidator("planner.index.max_chosen_indexes_per_table", 0, 100, null);
+  public static final BooleanValidator INDEX_FORCE_SORT_NONCOVERING = new BooleanValidator("planner.index.force_sort_noncovering", null);
+  public static final BooleanValidator INDEX_USE_HASHJOIN_NONCOVERING = new BooleanValidator("planner.index.use_hashjoin_noncovering", null);
+  public static final RangeDoubleValidator INDEX_COVERING_SELECTIVITY_THRESHOLD =
+      new RangeDoubleValidator("planner.index.covering_selectivity_threshold", 0.0, 1.0, null);
+  public static final RangeDoubleValidator INDEX_NONCOVERING_SELECTIVITY_THRESHOLD =
+      new RangeDoubleValidator("planner.index.noncovering_selectivity_threshold", 0.0, 1.0, null);
+  public static final RangeDoubleValidator INDEX_ROWKEYJOIN_COST_FACTOR =
+      new RangeDoubleValidator("planner.index.rowkeyjoin_cost_factor", 0, Double.MAX_VALUE, null);
+  // TODO: Deprecate the following 2 (also in SystemOptionManager.java)
+  public static final BooleanValidator INDEX_PREFER_INTERSECT_PLANS = new BooleanValidator("planner.index.prefer_intersect_plans", null);
+  public static final RangeLongValidator INDEX_MAX_INDEXES_TO_INTERSECT = new RangeLongValidator("planner.index.max_indexes_to_intersect", 2, 100, null);
+  public static final RangeDoubleValidator INDEX_STATS_ROWCOUNT_SCALING_FACTOR =
+      new RangeDoubleValidator("planner.index.statistics_rowcount_scaling_factor", 0.0, 1.0, null);
+  // ------------------------------------------- Index planning related options END ----------------------------------------------------------------
+
   public static final OptionValidator IDENTIFIER_MAX_LENGTH =
       new RangeLongValidator("planner.identifier_max_length", 128 /* A minimum length is needed because option names are identifiers themselves */,
                               Integer.MAX_VALUE,
@@ -365,6 +387,54 @@ public class PlannerSettings implements Context{
     return options.getOption(ENABLE_UNNEST_LATERAL);
   }
 
+  public boolean isIndexPlanningEnabled() {
+    return options.getOption(INDEX_PLANNING);
+  }
+
+  public boolean isStatisticsEnabled() {
+    return options.getOption(ENABLE_STATS);
+  }
+
+  public boolean isDisableFullTableScan() {
+    return options.getOption(DISABLE_FULL_TABLE_SCAN);
+  }
+
+  public long getIndexMaxChosenIndexesPerTable() {
+    return options.getOption(INDEX_MAX_CHOSEN_INDEXES_PER_TABLE);
+  }
+
+  public boolean isIndexForceSortNonCovering() {
+    return options.getOption(INDEX_FORCE_SORT_NONCOVERING);
+  }
+
+  public boolean isIndexUseHashJoinNonCovering() {
+    return options.getOption(INDEX_USE_HASHJOIN_NONCOVERING);
+  }
+
+  public double getIndexCoveringSelThreshold() {
+    return options.getOption(INDEX_COVERING_SELECTIVITY_THRESHOLD);
+  }
+
+  public double getIndexNonCoveringSelThreshold() {
+    return options.getOption(INDEX_NONCOVERING_SELECTIVITY_THRESHOLD);
+  }
+
+  public double getIndexRowKeyJoinCostFactor() {
+    return options.getOption(INDEX_ROWKEYJOIN_COST_FACTOR);
+  }
+
+  public boolean isIndexIntersectPlanPreferred() {
+    return options.getOption(INDEX_PREFER_INTERSECT_PLANS);
+  }
+
+  public long getMaxIndexesToIntersect() {
+    return options.getOption(INDEX_MAX_INDEXES_TO_INTERSECT);
+  }
+
+  public double getIndexStatsRowCountScalingFactor() {
+    return options.getOption(INDEX_STATS_ROWCOUNT_SCALING_FACTOR);
+  }
+
   @Override
   public <T> T unwrap(Class<T> clazz) {
     if(clazz == PlannerSettings.class){
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/physical/PrelFactories.java b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/physical/PrelFactories.java
new file mode 100644
index 0000000..321398b
--- /dev/null
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/physical/PrelFactories.java
@@ -0,0 +1,52 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.drill.exec.planner.physical;
+
+
+import org.apache.drill.shaded.guava.com.google.common.collect.Lists;
+import org.apache.calcite.plan.RelOptCluster;
+import org.apache.calcite.rel.RelNode;
+import org.apache.calcite.rel.core.RelFactories;
+import org.apache.calcite.rel.type.RelDataType;
+import org.apache.calcite.rex.RexNode;
+import org.apache.calcite.rex.RexUtil;
+
+
+import java.util.List;
+
+public class PrelFactories {
+  public static final RelFactories.ProjectFactory PROJECT_FACTORY =
+      new DrillProjectPrelFactory();
+
+  /**
+   * Implementation of {@link RelFactories.ProjectFactory} that returns a vanilla
+   * {@link org.apache.calcite.rel.logical.LogicalProject}.
+   */
+  private static class DrillProjectPrelFactory implements RelFactories.ProjectFactory {
+    @Override
+    public RelNode createProject(RelNode child,
+                                 List<? extends RexNode> childExprs, List<String> fieldNames) {
+      final RelOptCluster cluster = child.getCluster();
+      final RelDataType rowType = RexUtil.createStructType(cluster.getTypeFactory(), childExprs, fieldNames);
+      final RelNode project = new ProjectPrel(cluster, child.getTraitSet().plus(Prel.DRILL_PHYSICAL),
+          child, Lists.newArrayList(childExprs), rowType);
+
+      return project;
+    }
+  }
+}
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/physical/PrelUtil.java b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/physical/PrelUtil.java
index e1cc661..4381c00 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/physical/PrelUtil.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/physical/PrelUtil.java
@@ -18,9 +18,7 @@
 package org.apache.drill.exec.planner.physical;
 
 import org.apache.drill.shaded.guava.com.google.common.collect.Lists;
-import org.apache.drill.shaded.guava.com.google.common.collect.Sets;
-import org.apache.drill.shaded.guava.com.google.common.collect.ImmutableList;
-import com.carrotsearch.hppc.IntIntHashMap;
+
 import org.apache.calcite.plan.RelOptCluster;
 import org.apache.calcite.plan.RelOptPlanner;
 import org.apache.calcite.plan.RelOptRuleCall;
@@ -30,28 +28,18 @@ import org.apache.calcite.rel.RelCollation;
 import org.apache.calcite.rel.RelFieldCollation;
 import org.apache.calcite.rel.RelNode;
 import org.apache.calcite.rel.type.RelDataType;
-import org.apache.calcite.rel.type.RelDataTypeFactory;
-import org.apache.calcite.rel.type.RelDataTypeField;
 import org.apache.calcite.rex.RexCall;
 import org.apache.calcite.rex.RexInputRef;
-import org.apache.calcite.rex.RexLiteral;
-import org.apache.calcite.rex.RexLocalRef;
 import org.apache.calcite.rex.RexNode;
-import org.apache.calcite.rex.RexShuttle;
 import org.apache.calcite.rex.RexVisitorImpl;
 import org.apache.drill.common.expression.ExpressionPosition;
 import org.apache.drill.common.expression.FieldReference;
-import org.apache.drill.common.expression.PathSegment;
-import org.apache.drill.common.expression.PathSegment.ArraySegment;
-import org.apache.drill.common.expression.PathSegment.NameSegment;
-import org.apache.drill.common.expression.SchemaPath;
 import org.apache.drill.common.logical.data.Order.Ordering;
 import org.apache.drill.exec.record.BatchSchema.SelectionVectorMode;
 
 import java.util.Arrays;
 import java.util.Iterator;
 import java.util.List;
-import java.util.Set;
 
 public class PrelUtil {
 
@@ -107,121 +95,6 @@ public class PrelUtil {
     return lastUsed.getLastUsedReference();
   }
 
-  public static ProjectPushInfo getColumns(RelDataType rowType, List<RexNode> projects) {
-    final List<String> fieldNames = rowType.getFieldNames();
-    if (fieldNames.isEmpty()) {
-      return null;
-    }
-
-    RefFieldsVisitor v = new RefFieldsVisitor(rowType);
-    for (RexNode exp : projects) {
-      PathSegment segment = exp.accept(v);
-      v.addColumn(segment);
-    }
-
-    return v.getInfo();
-
-  }
-
-  public static class DesiredField {
-    public final int origIndex;
-    public final String name;
-    public final RelDataTypeField field;
-
-    public DesiredField(int origIndex, String name, RelDataTypeField field) {
-      super();
-      this.origIndex = origIndex;
-      this.name = name;
-      this.field = field;
-    }
-
-    @Override
-    public int hashCode() {
-      final int prime = 31;
-      int result = 1;
-      result = prime * result + ((field == null) ? 0 : field.hashCode());
-      result = prime * result + ((name == null) ? 0 : name.hashCode());
-      result = prime * result + origIndex;
-      return result;
-    }
-
-    @Override
-    public boolean equals(Object obj) {
-      if (this == obj) {
-        return true;
-      }
-      if (obj == null) {
-        return false;
-      }
-      if (getClass() != obj.getClass()) {
-        return false;
-      }
-      DesiredField other = (DesiredField) obj;
-      if (field == null) {
-        if (other.field != null) {
-          return false;
-        }
-      } else if (!field.equals(other.field)) {
-        return false;
-      }
-      if (name == null) {
-        if (other.name != null) {
-          return false;
-        }
-      } else if (!name.equals(other.name)) {
-        return false;
-      }
-      if (origIndex != other.origIndex) {
-        return false;
-      }
-      return true;
-    }
-
-  }
-
-  public static class ProjectPushInfo {
-    public final List<SchemaPath> columns;
-    public final List<DesiredField> desiredFields;
-    public final InputRewriter rewriter;
-    private final List<String> fieldNames;
-    private final List<RelDataType> types;
-
-    public ProjectPushInfo(List<SchemaPath> columns, ImmutableList<DesiredField> desiredFields) {
-      super();
-      this.columns = columns;
-      this.desiredFields = desiredFields;
-
-      this.fieldNames = Lists.newArrayListWithCapacity(desiredFields.size());
-      this.types = Lists.newArrayListWithCapacity(desiredFields.size());
-      IntIntHashMap oldToNewIds = new IntIntHashMap();
-
-      int i =0;
-      for (DesiredField f : desiredFields) {
-        fieldNames.add(f.name);
-        types.add(f.field.getType());
-        oldToNewIds.put(f.origIndex, i);
-        i++;
-      }
-      this.rewriter = new InputRewriter(oldToNewIds);
-    }
-
-    public InputRewriter getInputRewriter() {
-      return rewriter;
-    }
-
-    public boolean isStarQuery() {
-      for (SchemaPath column : columns) {
-        if (column.getRootSegment().getPath().startsWith("*")) {
-          return true;
-        }
-      }
-      return false;
-    }
-
-    public RelDataType createNewRowType(RelDataTypeFactory factory) {
-      return factory.createStructType(types, fieldNames);
-    }
-  }
 
   // Simple visitor class to determine the last used reference in the expression
   private static class LastUsedRefVisitor extends RexVisitorImpl<Void> {
@@ -251,69 +124,6 @@ public class PrelUtil {
     }
   }
 
-  /** Visitor that finds the set of inputs that are used. */
-  private static class RefFieldsVisitor extends RexVisitorImpl<PathSegment> {
-    final Set<SchemaPath> columns = Sets.newLinkedHashSet();
-    final private List<String> fieldNames;
-    final private List<RelDataTypeField> fields;
-    final private Set<DesiredField> desiredFields = Sets.newLinkedHashSet();
-
-    public RefFieldsVisitor(RelDataType rowType) {
-      super(true);
-      this.fieldNames = rowType.getFieldNames();
-      this.fields = rowType.getFieldList();
-    }
-
-    public void addColumn(PathSegment segment) {
-      if (segment != null && segment instanceof NameSegment) {
-        columns.add(new SchemaPath((NameSegment)segment));
-      }
-    }
-
-    public ProjectPushInfo getInfo() {
-      return new ProjectPushInfo(ImmutableList.copyOf(columns), ImmutableList.copyOf(desiredFields));
-    }
-
-    @Override
-    public PathSegment visitInputRef(RexInputRef inputRef) {
-      int index = inputRef.getIndex();
-      String name = fieldNames.get(index);
-      RelDataTypeField field = fields.get(index);
-      DesiredField f = new DesiredField(index, name, field);
-      desiredFields.add(f);
-      return new NameSegment(name);
-    }
-
-    @Override
-    public PathSegment visitCall(RexCall call) {
-      if ("ITEM".equals(call.getOperator().getName())) {
-        PathSegment mapOrArray = call.operands.get(0).accept(this);
-        if (mapOrArray != null) {
-          if (call.operands.get(1) instanceof RexLiteral) {
-            return mapOrArray.cloneWithNewChild(convertLiteral((RexLiteral) call.operands.get(1)));
-          }
-          return mapOrArray;
-        }
-      } else {
-        for (RexNode operand : call.operands) {
-          addColumn(operand.accept(this));
-        }
-      }
-      return null;
-    }
-
-    private PathSegment convertLiteral(RexLiteral literal) {
-      switch (literal.getType().getSqlTypeName()) {
-      case CHAR:
-        return new NameSegment(RexLiteral.stringValue(literal));
-      case INTEGER:
-        return new ArraySegment(RexLiteral.intValue(literal));
-      default:
-        return null;
-      }
-    }
-
-  }
 
   public static RelTraitSet fixTraits(RelOptRuleCall call, RelTraitSet set) {
     return fixTraits(call.getPlanner(), set);
@@ -339,44 +149,4 @@ public class PrelUtil {
 
     return newTraitSet;
   }
-
-  public static class InputRefRemap {
-    private int oldIndex;
-    private int newIndex;
-
-    public InputRefRemap(int oldIndex, int newIndex) {
-      super();
-      this.oldIndex = oldIndex;
-      this.newIndex = newIndex;
-    }
-    public int getOldIndex() {
-      return oldIndex;
-    }
-    public int getNewIndex() {
-      return newIndex;
-    }
-
-  }
-
-  public static class InputRewriter extends RexShuttle {
-
-    final IntIntHashMap map;
-
-    public InputRewriter(IntIntHashMap map) {
-      super();
-      this.map = map;
-    }
-
-    @Override
-    public RexNode visitInputRef(RexInputRef inputRef) {
-      return new RexInputRef(map.get(inputRef.getIndex()), inputRef.getType());
-    }
-
-    @Override
-    public RexNode visitLocalRef(RexLocalRef localRef) {
-      return new RexInputRef(map.get(localRef.getIndex()), localRef.getType());
-    }
-
-  }
-
 }
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/physical/RangePartitionExchangePrel.java b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/physical/RangePartitionExchangePrel.java
new file mode 100644
index 0000000..76568c0
--- /dev/null
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/physical/RangePartitionExchangePrel.java
@@ -0,0 +1,130 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.drill.exec.planner.physical;
+
+import java.io.IOException;
+import java.util.List;
+
+import org.apache.drill.exec.physical.base.PhysicalOperator;
+import org.apache.drill.exec.physical.config.RangePartitionExchange;
+import org.apache.drill.exec.planner.cost.DrillCostBase.DrillCostFactory;
+import org.apache.drill.exec.planner.physical.DrillDistributionTrait.DistributionField;
+import org.apache.drill.exec.record.BatchSchema.SelectionVectorMode;
+import org.apache.calcite.rel.RelNode;
+import org.apache.calcite.rel.RelWriter;
+import org.apache.calcite.rel.metadata.RelMetadataQuery;
+import org.apache.calcite.linq4j.Ord;
+import org.apache.calcite.plan.RelOptCluster;
+import org.apache.calcite.plan.RelOptCost;
+import org.apache.calcite.plan.RelOptPlanner;
+import org.apache.calcite.plan.RelTraitSet;
+
+import static org.apache.drill.exec.planner.index.Statistics.ROWCOUNT_UNKNOWN;
+
+/**
... 570 lines suppressed ...