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:56 UTC

[drill] branch master updated (61e8b46 -> 7571d52)

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

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


    from 61e8b46  DRILL-6715: Update descriptions for System Options table
     new 0abcbe3  DRILL-6381: (Part 1) Secondary Index framework
     new a4f62e9  DRILL-6381: (Part 2) MapRDB plugin update to 6.0.1
     new 12f57f0  DRILL-6381: (Part 3) Planner and Execution implementation to support Secondary Indexes
     new 2a9e51f  DRILL-6381: (Part 4) Enhance MapR-DB plugin to support querying secondary indexes
     new 3c2f9ab  DRILL-6381: (Part 5) Update Javadoc for a few interfaces.
     new 5fa9c80  DRILL-6381: Address code review comments.
     new 387bc4f  DRILL-6381: Address review comments (part 2): fix formatting issues and add javadoc.
     new 7571d52  DRILL-6381: Address code review comments (part 3).

The 8 revisions listed above as "new" are entirely new to this
repository and will be described in separate emails.  The revisions
listed as "add" were already present in the repository and have only
been added to this reference.


Summary of changes:
 contrib/format-maprdb/pom.xml                      |   89 ++
 .../planner/index/MapRDBFunctionalIndexInfo.java   |  168 ++
 .../exec/planner/index/MapRDBIndexDescriptor.java  |  222 +++
 .../exec/planner/index/MapRDBIndexDiscover.java    |  376 +++++
 .../drill/exec/planner/index/MapRDBStatistics.java | 1006 ++++++++++++
 .../planner/index/MapRDBStatisticsPayload.java     |   44 +-
 .../drill/exec/store/mapr/PluginConstants.java     |   93 ++
 .../drill/exec/store/mapr/PluginErrorHandler.java  |   50 +
 .../drill/exec/store/mapr/db/MapRDBCost.java       |  104 ++
 .../exec/store/mapr/db/MapRDBFormatMatcher.java    |   46 +-
 .../exec/store/mapr/db/MapRDBFormatPlugin.java     |   69 +-
 .../store/mapr/db/MapRDBFormatPluginConfig.java    |    4 +
 .../drill/exec/store/mapr/db/MapRDBGroupScan.java  |   83 +-
 .../store/mapr/db/MapRDBPushFilterIntoScan.java    |   20 +-
 .../store/mapr/db/MapRDBPushLimitIntoScan.java     |  203 +++
 .../store/mapr/db/MapRDBPushProjectIntoScan.java   |  142 ++
 ....java => MapRDBRestrictedScanBatchCreator.java} |   41 +-
 .../exec/store/mapr/db/MapRDBScanBatchCreator.java |    6 +-
 .../drill/exec/store/mapr/db/MapRDBSubScan.java    |   23 +-
 .../exec/store/mapr/db/MapRDBSubScanSpec.java      |   54 +-
 .../drill/exec/store/mapr/db/MapRDBTableCache.java |  232 +++
 .../store/mapr/db/RestrictedMapRDBSubScan.java     |   80 +
 .../store/mapr/db/RestrictedMapRDBSubScanSpec.java |  191 +++
 .../store/mapr/db/binary/BinaryTableGroupScan.java |   34 +-
 .../store/mapr/db/json/AllTextValueWriter.java     |   80 +
 .../mapr/db/json/CompareFunctionsProcessor.java    |   15 +-
 .../mapr/db/json/DocumentReaderVectorWriter.java   |   42 +
 .../exec/store/mapr/db/json/FieldPathHelper.java   |   75 +
 .../mapr/db/json/FieldTransferVectorWriter.java    |   49 +
 .../store/mapr/db/json/IdOnlyVectorWriter.java     |   65 +
 .../store/mapr/db/json/JsonConditionBuilder.java   |   59 +-
 .../exec/store/mapr/db/json/JsonScanSpec.java      |   93 +-
 .../exec/store/mapr/db/json/JsonSubScanSpec.java   |   69 +-
 .../store/mapr/db/json/JsonTableGroupScan.java     |  676 +++++++-
 .../db/json/JsonTableRangePartitionFunction.java   |  237 +++
 .../store/mapr/db/json/MaprDBJsonRecordReader.java |  538 +++----
 .../mapr/db/json/NumbersAsDoubleValueWriter.java   |   51 +
 .../store/mapr/db/json/OjaiFunctionsProcessor.java |  214 +++
 .../exec/store/mapr/db/json/OjaiValueWriter.java   |  194 +++
 .../db/json/ProjectionPassthroughVectorWriter.java |   83 +
 .../mapr/db/json/RestrictedJsonRecordReader.java   |  248 +++
 .../mapr/db/json/RestrictedJsonTableGroupScan.java |  184 +++
 .../store/mapr/db/json/RowCountVectorWriter.java   |   29 +-
 .../exec/udf/mapr/db/ConditionPlaceholder.java     |   54 +
 .../drill/exec/udf/mapr/db/DecodeFieldPath.java    |   65 +
 .../drill/exec/udf/mapr/db/MatchesPlaceholder.java |   53 +
 .../exec/udf/mapr/db/NotMatchesPlaceholder.java    |   53 +
 .../exec/udf/mapr/db/NotTypeOfPlaceholder.java     |   53 +
 .../drill/exec/udf/mapr/db/SizeOfPlaceholder.java  |   54 +
 .../drill/exec/udf/mapr/db/TypeOfPlaceholder.java  |   53 +
 .../src/main/resources/drill-module.conf           |   20 +-
 .../mapr/drill/maprdb/tests/MaprDBTestsSuite.java  |   63 +-
 .../maprdb/tests/index/IndexHintPlanTest.java      |  166 ++
 .../drill/maprdb/tests/index/IndexPlanTest.java    | 1690 ++++++++++++++++++++
 .../drill/maprdb/tests/index/LargeTableGen.java    |  172 ++
 .../maprdb/tests/index/LargeTableGenBase.java      |  185 +++
 .../drill/maprdb/tests/index/StatisticsTest.java   |  108 ++
 .../drill/maprdb/tests/index/TableIndexCmd.java    |  123 ++
 .../mapr/drill/maprdb/tests/json/BaseJsonTest.java |   13 +
 .../maprdb/tests/json/TestEncodedFieldPaths.java   |  128 ++
 .../maprdb/tests/json/TestFieldPathHelper.java     |   52 +
 .../drill/maprdb/tests/json/TestScanRanges.java    |  158 ++
 .../drill/maprdb/tests/json/TestSimpleJson.java    |  216 ++-
 .../{ => com/mapr/drill}/json/business.json        |    0
 .../mapr/drill/json/encoded_fields_userdata.json   |    5 +
 .../resources/{hbase-site.xml => core-site.xml}    |    5 +
 contrib/native/client/src/protobuf/BitData.pb.cc   |   55 +-
 contrib/native/client/src/protobuf/BitData.pb.h    |   36 +-
 .../native/client/src/protobuf/UserBitShared.pb.cc |   57 +-
 .../native/client/src/protobuf/UserBitShared.pb.h  |    7 +-
 .../exec/store/hbase/HBasePushFilterIntoScan.java  |    2 +-
 ...ertHiveMapRDBJsonScanToDrillMapRDBJsonScan.java |    2 +-
 .../store/kafka/KafkaPushDownFilterIntoScan.java   |    3 +-
 .../store/mongo/MongoPushDownFilterForScan.java    |    5 +-
 .../java/org/apache/drill/exec/ExecConstants.java  |    2 +
 .../exec/physical/base/AbstractDbGroupScan.java    |   95 ++
 .../base/AbstractDbSubScan.java}                   |   26 +-
 .../exec/physical/base/AbstractGroupScan.java      |    6 +
 .../physical/base/AbstractPhysicalVisitor.java     |   18 +
 .../drill/exec/physical/base/DbGroupScan.java      |  129 ++
 .../apache/drill/exec/physical/base/DbSubScan.java |   41 +-
 .../apache/drill/exec/physical/base/GroupScan.java |    3 +
 .../drill/exec/physical/base/IndexGroupScan.java   |   75 +
 .../drill/exec/physical/base/PhysicalVisitor.java  |    6 +
 .../drill/exec/physical/config/HashJoinPOP.java    |   62 +-
 .../exec/physical/config/HashPartitionSender.java  |   20 +-
 .../physical/config/RangePartitionExchange.java    |   60 +
 ...{RangeSender.java => RangePartitionSender.java} |   52 +-
 .../drill/exec/physical/config/RowKeyJoinPOP.java  |   96 ++
 .../apache/drill/exec/physical/impl/ScanBatch.java |  145 +-
 .../exec/physical/impl/common/HashPartition.java   |   15 +-
 .../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  |   27 +-
 .../drill/exec/physical/impl/join/RowKeyJoin.java  |   79 +
 .../exec/physical/impl/join/RowKeyJoinBatch.java   |  290 ++++
 ...tchCreator.java => RowKeyJoinBatchCreator.java} |   18 +-
 .../impl/mergereceiver/MergingRecordBatch.java     |    5 +-
 .../physical/impl/partitionsender/Partitioner.java |    5 +
 .../RangePartitionRecordBatch.java                 |  193 +++
 .../RangePartitionSenderCreator.java               |   46 +
 .../apache/drill/exec/planner/PlannerPhase.java    |   34 +-
 .../exec/planner/common/DrillJoinRelBase.java      |    4 +
 .../drill/exec/planner/common/DrillRelOptUtil.java |  154 ++
 .../exec/planner/common/DrillScanRelBase.java      |   39 +-
 .../drill/exec/planner/common/JoinControl.java     |   53 +
 .../drill/exec/planner/common/OrderedRel.java      |   53 +
 .../cost/DrillDefaultRelMetadataProvider.java      |    1 +
 .../exec/planner/cost/DrillRelMdSelectivity.java   |   90 ++
 .../apache/drill/exec/planner/cost/PluginCost.java |   79 +
 .../drill/exec/planner/fragment/Materializer.java  |   37 +-
 .../planner/index/AbstractIndexCollection.java     |   96 ++
 .../planner/index/AbstractIndexDescriptor.java     |   80 +
 .../planner/index/AbstractIndexStatistics.java     |   51 +
 .../CollationContext.java}                         |   26 +-
 .../exec/planner/index/DrillIndexCollection.java   |   75 +
 .../exec/planner/index/DrillIndexDefinition.java   |  278 ++++
 .../exec/planner/index/DrillIndexDescriptor.java   |  122 ++
 .../apache/drill/exec/planner/index/ExprToRex.java |  107 ++
 .../planner/index/FindFiltersForCollation.java     |  199 +++
 .../exec/planner/index/FunctionalIndexHelper.java  |  215 +++
 .../exec/planner/index/FunctionalIndexInfo.java    |   85 +
 .../drill/exec/planner/index/IndexCallContext.java |   76 +
 .../drill/exec/planner/index/IndexCollection.java  |  101 ++
 .../exec/planner/index/IndexConditionInfo.java     |  258 +++
 .../drill/exec/planner/index/IndexDefinition.java  |  119 ++
 .../drill/exec/planner/index/IndexDescriptor.java  |   90 ++
 .../IndexDiscover.java}                            |   16 +-
 .../exec/planner/index/IndexDiscoverBase.java      |  110 ++
 .../exec/planner/index/IndexDiscoverFactory.java   |   75 +
 .../IndexDiscoverable.java}                        |   28 +-
 .../drill/exec/planner/index/IndexGroup.java       |   59 +
 .../planner/index/IndexLogicalPlanCallContext.java |  178 +++
 .../index/IndexPhysicalPlanCallContext.java        |  193 +++
 .../drill/exec/planner/index/IndexPlanUtils.java   |  872 ++++++++++
 .../drill/exec/planner/index/IndexProperties.java  |   69 +
 .../drill/exec/planner/index/IndexSelector.java    |  766 +++++++++
 .../IndexStatistics.java}                          |   27 +-
 .../exec/planner/index/IndexableExprMarker.java    |  262 +++
 .../InvalidIndexDefinitionException.java}          |   20 +-
 .../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       |   65 +
 .../StatisticsPayload.java}                        |   19 +-
 .../generators/AbstractIndexPlanGenerator.java     |  274 ++++
 .../generators/CoveringIndexPlanGenerator.java     |  208 +++
 .../generators/CoveringPlanNoFilterGenerator.java  |  125 ++
 .../generators/IndexIntersectPlanGenerator.java    |  350 ++++
 .../generators/NonCoveringIndexPlanGenerator.java  |  336 ++++
 .../planner/index/rules/AbstractMatchFunction.java |   58 +
 .../planner/index/rules/DbScanSortRemovalRule.java |  239 +++
 .../index/rules/DbScanToIndexScanPrule.java        |  533 ++++++
 .../rules/MatchFunction.java}                      |   20 +-
 .../planner/logical/DrillMergeProjectRule.java     |   32 +
 .../drill/exec/planner/logical/DrillOptiq.java     |   43 +-
 .../logical/DrillPushProjectIntoScanRule.java      |  171 +-
 .../drill/exec/planner/logical/DrillScanRel.java   |   37 +-
 .../drill/exec/planner/logical/DrillSortRel.java   |   17 +-
 .../drill/exec/planner/logical/DrillTable.java     |    4 +
 .../physical/AbstractRangePartitionFunction.java   |   44 +-
 .../planner/physical/BroadcastExchangePrel.java    |    7 +-
 .../planner/physical/ConvertCountToDirectScan.java |    6 +-
 .../{ScanPrel.java => DirectScanPrel.java}         |  146 +-
 .../exec/planner/physical/DirectScanPrule.java     |   10 +-
 .../planner/physical/DrillDistributionTrait.java   |   31 +-
 .../physical/DrillDistributionTraitDef.java        |   11 +-
 .../drill/exec/planner/physical/FilterPrel.java    |    2 +-
 .../drill/exec/planner/physical/HashJoinPrel.java  |   26 +-
 ...seRule.java => LimitExchangeTransposeRule.java} |   63 +-
 .../exec/planner/physical/PartitionFunction.java   |   56 +
 .../exec/planner/physical/PlannerSettings.java     |   70 +
 .../drill/exec/planner/physical/PrelFactories.java |   52 +
 .../drill/exec/planner/physical/PrelUtil.java      |    1 +
 .../physical/RangePartitionExchangePrel.java       |  130 ++
 .../exec/planner/physical/RowKeyJoinPrel.java      |  120 ++
 .../drill/exec/planner/physical/ScanPrel.java      |   50 +-
 .../drill/exec/planner/physical/ScanPrule.java     |    2 +-
 .../drill/exec/planner/physical/SortPrel.java      |   28 +-
 .../drill/exec/planner/physical/SortPrule.java     |   16 +-
 .../exec/planner/physical/SubsetTransformer.java   |   13 +
 .../drill/exec/planner/physical/TopNPrel.java      |   31 +-
 .../visitor/ExcessiveExchangeIdentifier.java       |   28 +
 .../physical/visitor/SwapHashJoinVisitor.java      |    3 +-
 .../drill/exec/record/AbstractRecordBatch.java     |    2 +-
 .../exec/server/options/SystemOptionManager.java   |   19 +-
 .../drill/exec/store/AbstractRecordReader.java     |    5 +
 .../org/apache/drill/exec/store/RecordReader.java  |    8 +
 .../drill/exec/store/dfs/FileSystemConfig.java     |    8 +-
 .../InfoSchemaPushFilterIntoRecordGenerator.java   |    2 +-
 .../exec/store/parquet/ParquetPushDownFilter.java  |    2 +-
 .../drill/exec/util/EncodedSchemaPathSet.java      |  291 ++++
 .../java-exec/src/main/resources/drill-module.conf |   16 +
 .../physical/impl/common/HashPartitionTest.java    |    2 +-
 .../java/org/apache/drill/test/BaseTestQuery.java  |   12 +-
 .../drill/common/expression/CastExpression.java    |   23 +
 .../apache/drill/common/expression/SchemaPath.java |    3 +
 .../drill/common/logical/StoragePluginConfig.java  |    9 +-
 pom.xml                                            |    4 +-
 .../org/apache/drill/exec/proto/UserBitShared.java |   75 +-
 .../drill/exec/proto/beans/CoreOperatorType.java   |    8 +-
 protocol/src/main/protobuf/UserBitShared.proto     |    3 +-
 207 files changed, 20202 insertions(+), 1436 deletions(-)
 create mode 100644 contrib/format-maprdb/src/main/java/org/apache/drill/exec/planner/index/MapRDBFunctionalIndexInfo.java
 create mode 100644 contrib/format-maprdb/src/main/java/org/apache/drill/exec/planner/index/MapRDBIndexDescriptor.java
 create mode 100644 contrib/format-maprdb/src/main/java/org/apache/drill/exec/planner/index/MapRDBIndexDiscover.java
 create mode 100644 contrib/format-maprdb/src/main/java/org/apache/drill/exec/planner/index/MapRDBStatistics.java
 copy exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/join/HashJoinBatchCreator.java => contrib/format-maprdb/src/main/java/org/apache/drill/exec/planner/index/MapRDBStatisticsPayload.java (50%)
 create mode 100644 contrib/format-maprdb/src/main/java/org/apache/drill/exec/store/mapr/PluginConstants.java
 create mode 100644 contrib/format-maprdb/src/main/java/org/apache/drill/exec/store/mapr/PluginErrorHandler.java
 create mode 100644 contrib/format-maprdb/src/main/java/org/apache/drill/exec/store/mapr/db/MapRDBCost.java
 create mode 100644 contrib/format-maprdb/src/main/java/org/apache/drill/exec/store/mapr/db/MapRDBPushLimitIntoScan.java
 create mode 100644 contrib/format-maprdb/src/main/java/org/apache/drill/exec/store/mapr/db/MapRDBPushProjectIntoScan.java
 copy contrib/format-maprdb/src/main/java/org/apache/drill/exec/store/mapr/db/{MapRDBScanBatchCreator.java => MapRDBRestrictedScanBatchCreator.java} (51%)
 create mode 100644 contrib/format-maprdb/src/main/java/org/apache/drill/exec/store/mapr/db/MapRDBTableCache.java
 create mode 100644 contrib/format-maprdb/src/main/java/org/apache/drill/exec/store/mapr/db/RestrictedMapRDBSubScan.java
 create mode 100644 contrib/format-maprdb/src/main/java/org/apache/drill/exec/store/mapr/db/RestrictedMapRDBSubScanSpec.java
 create mode 100644 contrib/format-maprdb/src/main/java/org/apache/drill/exec/store/mapr/db/json/AllTextValueWriter.java
 create mode 100644 contrib/format-maprdb/src/main/java/org/apache/drill/exec/store/mapr/db/json/DocumentReaderVectorWriter.java
 create mode 100644 contrib/format-maprdb/src/main/java/org/apache/drill/exec/store/mapr/db/json/FieldPathHelper.java
 create mode 100644 contrib/format-maprdb/src/main/java/org/apache/drill/exec/store/mapr/db/json/FieldTransferVectorWriter.java
 create mode 100644 contrib/format-maprdb/src/main/java/org/apache/drill/exec/store/mapr/db/json/IdOnlyVectorWriter.java
 create mode 100644 contrib/format-maprdb/src/main/java/org/apache/drill/exec/store/mapr/db/json/JsonTableRangePartitionFunction.java
 create mode 100644 contrib/format-maprdb/src/main/java/org/apache/drill/exec/store/mapr/db/json/NumbersAsDoubleValueWriter.java
 create mode 100644 contrib/format-maprdb/src/main/java/org/apache/drill/exec/store/mapr/db/json/OjaiFunctionsProcessor.java
 create mode 100644 contrib/format-maprdb/src/main/java/org/apache/drill/exec/store/mapr/db/json/OjaiValueWriter.java
 create mode 100644 contrib/format-maprdb/src/main/java/org/apache/drill/exec/store/mapr/db/json/ProjectionPassthroughVectorWriter.java
 create mode 100644 contrib/format-maprdb/src/main/java/org/apache/drill/exec/store/mapr/db/json/RestrictedJsonRecordReader.java
 create mode 100644 contrib/format-maprdb/src/main/java/org/apache/drill/exec/store/mapr/db/json/RestrictedJsonTableGroupScan.java
 copy exec/java-exec/src/main/java/org/apache/drill/exec/planner/cost/DrillDefaultRelMetadataProvider.java => contrib/format-maprdb/src/main/java/org/apache/drill/exec/store/mapr/db/json/RowCountVectorWriter.java (53%)
 create mode 100644 contrib/format-maprdb/src/main/java/org/apache/drill/exec/udf/mapr/db/ConditionPlaceholder.java
 create mode 100644 contrib/format-maprdb/src/main/java/org/apache/drill/exec/udf/mapr/db/DecodeFieldPath.java
 create mode 100644 contrib/format-maprdb/src/main/java/org/apache/drill/exec/udf/mapr/db/MatchesPlaceholder.java
 create mode 100644 contrib/format-maprdb/src/main/java/org/apache/drill/exec/udf/mapr/db/NotMatchesPlaceholder.java
 create mode 100644 contrib/format-maprdb/src/main/java/org/apache/drill/exec/udf/mapr/db/NotTypeOfPlaceholder.java
 create mode 100644 contrib/format-maprdb/src/main/java/org/apache/drill/exec/udf/mapr/db/SizeOfPlaceholder.java
 create mode 100644 contrib/format-maprdb/src/main/java/org/apache/drill/exec/udf/mapr/db/TypeOfPlaceholder.java
 create mode 100644 contrib/format-maprdb/src/test/java/com/mapr/drill/maprdb/tests/index/IndexHintPlanTest.java
 create mode 100644 contrib/format-maprdb/src/test/java/com/mapr/drill/maprdb/tests/index/IndexPlanTest.java
 create mode 100644 contrib/format-maprdb/src/test/java/com/mapr/drill/maprdb/tests/index/LargeTableGen.java
 create mode 100644 contrib/format-maprdb/src/test/java/com/mapr/drill/maprdb/tests/index/LargeTableGenBase.java
 create mode 100644 contrib/format-maprdb/src/test/java/com/mapr/drill/maprdb/tests/index/StatisticsTest.java
 create mode 100644 contrib/format-maprdb/src/test/java/com/mapr/drill/maprdb/tests/index/TableIndexCmd.java
 create mode 100644 contrib/format-maprdb/src/test/java/com/mapr/drill/maprdb/tests/json/TestEncodedFieldPaths.java
 create mode 100644 contrib/format-maprdb/src/test/java/com/mapr/drill/maprdb/tests/json/TestFieldPathHelper.java
 create mode 100644 contrib/format-maprdb/src/test/java/com/mapr/drill/maprdb/tests/json/TestScanRanges.java
 rename contrib/format-maprdb/src/test/resources/{ => com/mapr/drill}/json/business.json (100%)
 create mode 100644 contrib/format-maprdb/src/test/resources/com/mapr/drill/json/encoded_fields_userdata.json
 rename contrib/format-maprdb/src/test/resources/{hbase-site.xml => core-site.xml} (90%)
 create mode 100644 exec/java-exec/src/main/java/org/apache/drill/exec/physical/base/AbstractDbGroupScan.java
 copy exec/java-exec/src/main/java/org/apache/drill/exec/{planner/cost/DrillDefaultRelMetadataProvider.java => physical/base/AbstractDbSubScan.java} (56%)
 create mode 100644 exec/java-exec/src/main/java/org/apache/drill/exec/physical/base/DbGroupScan.java
 copy logical/src/main/java/org/apache/drill/common/logical/StoragePluginConfig.java => exec/java-exec/src/main/java/org/apache/drill/exec/physical/base/DbSubScan.java (52%)
 create mode 100644 exec/java-exec/src/main/java/org/apache/drill/exec/physical/base/IndexGroupScan.java
 create mode 100644 exec/java-exec/src/main/java/org/apache/drill/exec/physical/config/RangePartitionExchange.java
 rename exec/java-exec/src/main/java/org/apache/drill/exec/physical/config/{RangeSender.java => RangePartitionSender.java} (50%)
 create mode 100644 exec/java-exec/src/main/java/org/apache/drill/exec/physical/config/RowKeyJoinPOP.java
 create mode 100644 exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/join/RowKeyJoin.java
 create mode 100644 exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/join/RowKeyJoinBatch.java
 copy exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/join/{HashJoinBatchCreator.java => RowKeyJoinBatchCreator.java} (63%)
 create mode 100644 exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/rangepartitioner/RangePartitionRecordBatch.java
 create mode 100644 exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/rangepartitioner/RangePartitionSenderCreator.java
 create mode 100644 exec/java-exec/src/main/java/org/apache/drill/exec/planner/common/JoinControl.java
 create mode 100644 exec/java-exec/src/main/java/org/apache/drill/exec/planner/common/OrderedRel.java
 create mode 100644 exec/java-exec/src/main/java/org/apache/drill/exec/planner/cost/DrillRelMdSelectivity.java
 create mode 100644 exec/java-exec/src/main/java/org/apache/drill/exec/planner/cost/PluginCost.java
 create mode 100644 exec/java-exec/src/main/java/org/apache/drill/exec/planner/index/AbstractIndexCollection.java
 create mode 100644 exec/java-exec/src/main/java/org/apache/drill/exec/planner/index/AbstractIndexDescriptor.java
 create mode 100644 exec/java-exec/src/main/java/org/apache/drill/exec/planner/index/AbstractIndexStatistics.java
 copy exec/java-exec/src/main/java/org/apache/drill/exec/planner/{cost/DrillDefaultRelMetadataProvider.java => index/CollationContext.java} (56%)
 create mode 100644 exec/java-exec/src/main/java/org/apache/drill/exec/planner/index/DrillIndexCollection.java
 create mode 100644 exec/java-exec/src/main/java/org/apache/drill/exec/planner/index/DrillIndexDefinition.java
 create mode 100644 exec/java-exec/src/main/java/org/apache/drill/exec/planner/index/DrillIndexDescriptor.java
 create mode 100644 exec/java-exec/src/main/java/org/apache/drill/exec/planner/index/ExprToRex.java
 create mode 100644 exec/java-exec/src/main/java/org/apache/drill/exec/planner/index/FindFiltersForCollation.java
 create mode 100644 exec/java-exec/src/main/java/org/apache/drill/exec/planner/index/FunctionalIndexHelper.java
 create mode 100644 exec/java-exec/src/main/java/org/apache/drill/exec/planner/index/FunctionalIndexInfo.java
 create mode 100644 exec/java-exec/src/main/java/org/apache/drill/exec/planner/index/IndexCallContext.java
 create mode 100644 exec/java-exec/src/main/java/org/apache/drill/exec/planner/index/IndexCollection.java
 create mode 100644 exec/java-exec/src/main/java/org/apache/drill/exec/planner/index/IndexConditionInfo.java
 create mode 100644 exec/java-exec/src/main/java/org/apache/drill/exec/planner/index/IndexDefinition.java
 create mode 100644 exec/java-exec/src/main/java/org/apache/drill/exec/planner/index/IndexDescriptor.java
 copy exec/java-exec/src/main/java/org/apache/drill/exec/planner/{cost/DrillDefaultRelMetadataProvider.java => index/IndexDiscover.java} (55%)
 create mode 100644 exec/java-exec/src/main/java/org/apache/drill/exec/planner/index/IndexDiscoverBase.java
 create mode 100644 exec/java-exec/src/main/java/org/apache/drill/exec/planner/index/IndexDiscoverFactory.java
 copy exec/java-exec/src/main/java/org/apache/drill/exec/planner/{cost/DrillDefaultRelMetadataProvider.java => index/IndexDiscoverable.java} (55%)
 create mode 100644 exec/java-exec/src/main/java/org/apache/drill/exec/planner/index/IndexGroup.java
 create mode 100644 exec/java-exec/src/main/java/org/apache/drill/exec/planner/index/IndexLogicalPlanCallContext.java
 create mode 100644 exec/java-exec/src/main/java/org/apache/drill/exec/planner/index/IndexPhysicalPlanCallContext.java
 create mode 100644 exec/java-exec/src/main/java/org/apache/drill/exec/planner/index/IndexPlanUtils.java
 create mode 100644 exec/java-exec/src/main/java/org/apache/drill/exec/planner/index/IndexProperties.java
 create mode 100644 exec/java-exec/src/main/java/org/apache/drill/exec/planner/index/IndexSelector.java
 copy exec/java-exec/src/main/java/org/apache/drill/exec/planner/{cost/DrillDefaultRelMetadataProvider.java => index/IndexStatistics.java} (55%)
 create mode 100644 exec/java-exec/src/main/java/org/apache/drill/exec/planner/index/IndexableExprMarker.java
 copy exec/java-exec/src/main/java/org/apache/drill/exec/planner/{cost/DrillDefaultRelMetadataProvider.java => index/InvalidIndexDefinitionException.java} (56%)
 create mode 100644 exec/java-exec/src/main/java/org/apache/drill/exec/planner/index/PathInExpr.java
 create mode 100644 exec/java-exec/src/main/java/org/apache/drill/exec/planner/index/RexSeparator.java
 create mode 100644 exec/java-exec/src/main/java/org/apache/drill/exec/planner/index/RexToExpression.java
 create mode 100644 exec/java-exec/src/main/java/org/apache/drill/exec/planner/index/SimpleRexRemap.java
 create mode 100644 exec/java-exec/src/main/java/org/apache/drill/exec/planner/index/Statistics.java
 copy exec/java-exec/src/main/java/org/apache/drill/exec/planner/{cost/DrillDefaultRelMetadataProvider.java => index/StatisticsPayload.java} (55%)
 create mode 100644 exec/java-exec/src/main/java/org/apache/drill/exec/planner/index/generators/AbstractIndexPlanGenerator.java
 create mode 100644 exec/java-exec/src/main/java/org/apache/drill/exec/planner/index/generators/CoveringIndexPlanGenerator.java
 create mode 100644 exec/java-exec/src/main/java/org/apache/drill/exec/planner/index/generators/CoveringPlanNoFilterGenerator.java
 create mode 100644 exec/java-exec/src/main/java/org/apache/drill/exec/planner/index/generators/IndexIntersectPlanGenerator.java
 create mode 100644 exec/java-exec/src/main/java/org/apache/drill/exec/planner/index/generators/NonCoveringIndexPlanGenerator.java
 create mode 100644 exec/java-exec/src/main/java/org/apache/drill/exec/planner/index/rules/AbstractMatchFunction.java
 create mode 100644 exec/java-exec/src/main/java/org/apache/drill/exec/planner/index/rules/DbScanSortRemovalRule.java
 create mode 100644 exec/java-exec/src/main/java/org/apache/drill/exec/planner/index/rules/DbScanToIndexScanPrule.java
 copy exec/java-exec/src/main/java/org/apache/drill/exec/planner/{cost/DrillDefaultRelMetadataProvider.java => index/rules/MatchFunction.java} (55%)
 copy logical/src/main/java/org/apache/drill/common/logical/StoragePluginConfig.java => exec/java-exec/src/main/java/org/apache/drill/exec/planner/physical/AbstractRangePartitionFunction.java (55%)
 copy exec/java-exec/src/main/java/org/apache/drill/exec/planner/physical/{ScanPrel.java => DirectScanPrel.java} (62%)
 rename exec/java-exec/src/main/java/org/apache/drill/exec/planner/physical/{LimitUnionExchangeTransposeRule.java => LimitExchangeTransposeRule.java} (51%)
 create mode 100644 exec/java-exec/src/main/java/org/apache/drill/exec/planner/physical/PartitionFunction.java
 create mode 100644 exec/java-exec/src/main/java/org/apache/drill/exec/planner/physical/PrelFactories.java
 create mode 100644 exec/java-exec/src/main/java/org/apache/drill/exec/planner/physical/RangePartitionExchangePrel.java
 create mode 100644 exec/java-exec/src/main/java/org/apache/drill/exec/planner/physical/RowKeyJoinPrel.java
 create mode 100644 exec/java-exec/src/main/java/org/apache/drill/exec/util/EncodedSchemaPathSet.java


[drill] 06/08: DRILL-6381: Address code review comments.

Posted by am...@apache.org.
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 5fa9c808daef8ea70a39ac2248daa99055955b72
Author: Aman Sinha <as...@maprtech.com>
AuthorDate: Mon Oct 1 12:06:39 2018 -0700

    DRILL-6381: Address code review comments.
---
 .../planner/index/MapRDBFunctionalIndexInfo.java   |  5 ++++
 .../exec/planner/index/MapRDBIndexDiscover.java    | 14 ++++++-----
 .../store/mapr/db/RestrictedMapRDBSubScanSpec.java | 28 ----------------------
 .../db/json/JsonTableRangePartitionFunction.java   |  2 +-
 .../drill/exec/physical/config/HashJoinPOP.java    | 12 ----------
 .../planner/index/AbstractIndexDescriptor.java     |  6 +++++
 .../drill/exec/planner/index/IndexDescriptor.java  | 11 +++++++--
 .../drill/exec/planner/index/IndexGroup.java       |  6 +----
 .../generators/IndexIntersectPlanGenerator.java    | 22 +++++++++++++----
 .../generators/NonCoveringIndexPlanGenerator.java  | 21 ++++++++--------
 .../physical/DrillDistributionTraitDef.java        |  8 ++++++-
 .../exec/planner/physical/RowKeyJoinPrel.java      |  7 ++++--
 12 files changed, 70 insertions(+), 72 deletions(-)

diff --git a/contrib/format-maprdb/src/main/java/org/apache/drill/exec/planner/index/MapRDBFunctionalIndexInfo.java b/contrib/format-maprdb/src/main/java/org/apache/drill/exec/planner/index/MapRDBFunctionalIndexInfo.java
index 01561a3..ec38636 100644
--- a/contrib/format-maprdb/src/main/java/org/apache/drill/exec/planner/index/MapRDBFunctionalIndexInfo.java
+++ b/contrib/format-maprdb/src/main/java/org/apache/drill/exec/planner/index/MapRDBFunctionalIndexInfo.java
@@ -138,6 +138,11 @@ public class MapRDBFunctionalIndexInfo implements FunctionalIndexInfo {
   }
 
   /**
+   * Suppose the index key has functions (rather than plain columns): CAST(a as int), CAST(b as varchar(10)),
+   * then we want to maintain a mapping of the logical expression of that function to the schema path of the
+   * base column involved in the function. In this example map has 2 entries:
+   *   CAST(a as int)  --> 'a'
+   *   CAST(b as varchar(10)) --> 'b'
    * @return the map of indexed expression --> the involved schema paths in a indexed expression
    */
   public Map<LogicalExpression, Set<SchemaPath>> getPathsInFunctionExpr() {
diff --git a/contrib/format-maprdb/src/main/java/org/apache/drill/exec/planner/index/MapRDBIndexDiscover.java b/contrib/format-maprdb/src/main/java/org/apache/drill/exec/planner/index/MapRDBIndexDiscover.java
index e1b8a61..c231e11 100644
--- a/contrib/format-maprdb/src/main/java/org/apache/drill/exec/planner/index/MapRDBIndexDiscover.java
+++ b/contrib/format-maprdb/src/main/java/org/apache/drill/exec/planner/index/MapRDBIndexDiscover.java
@@ -37,6 +37,7 @@ import org.apache.drill.common.expression.parser.ExprLexer;
 import org.apache.drill.common.expression.parser.ExprParser;
 import org.apache.drill.common.types.TypeProtos;
 import org.apache.drill.common.types.Types;
+import org.apache.drill.common.util.DrillFileUtils;
 import org.apache.drill.exec.physical.base.AbstractDbGroupScan;
 import org.apache.drill.exec.physical.base.GroupScan;
 import org.apache.drill.exec.planner.common.DrillScanRelBase;
@@ -67,6 +68,7 @@ import java.util.Set;
 public class MapRDBIndexDiscover extends IndexDiscoverBase implements IndexDiscover {
 
   static final String DEFAULT_STRING_CAST_LEN_STR = "256";
+  static final String FIELD_DELIMITER = ":";
 
   public MapRDBIndexDiscover(GroupScan inScan, DrillScanRelBase scanRel) {
     super((AbstractDbGroupScan) inScan, scanRel);
@@ -78,14 +80,14 @@ public class MapRDBIndexDiscover extends IndexDiscoverBase implements IndexDisco
 
   @Override
   public IndexCollection getTableIndex(String tableName) {
-    //return getTableIndexFromCommandLine(tableName);
     return getTableIndexFromMFS(tableName);
   }
 
   /**
-   *
+   * For a given table name get the list of indexes defined on the table according to the visibility of
+   * the indexes based on permissions.
    * @param tableName
-   * @return
+   * @return an IndexCollection representing the list of indexes for that table
    */
   private IndexCollection getTableIndexFromMFS(String tableName) {
     try {
@@ -120,8 +122,8 @@ public class MapRDBIndexDiscover extends IndexDiscoverBase implements IndexDisco
 
   FileSelection deriveFSSelection(DrillFileSystem fs, IndexDescriptor idxDesc) throws IOException {
     String tableName = idxDesc.getTableName();
-    String[] tablePath = tableName.split("/");
-    String tableParent = tableName.substring(0, tableName.lastIndexOf("/"));
+    String[] tablePath = tableName.split(DrillFileUtils.SEPARATOR);
+    String tableParent = tableName.substring(0, tableName.lastIndexOf(DrillFileUtils.SEPARATOR));
 
     return FileSelection.create(fs, tableParent, tablePath[tablePath.length - 1], false);
   }
@@ -318,7 +320,7 @@ public class MapRDBIndexDiscover extends IndexDiscoverBase implements IndexDisco
   private DrillIndexDescriptor buildIndexDescriptor(String tableName, IndexDesc desc)
       throws InvalidIndexDefinitionException {
     if (desc.isExternal()) {
-      //XX: not support external index
+      // External index is not currently supported
       return null;
     }
 
diff --git a/contrib/format-maprdb/src/main/java/org/apache/drill/exec/store/mapr/db/RestrictedMapRDBSubScanSpec.java b/contrib/format-maprdb/src/main/java/org/apache/drill/exec/store/mapr/db/RestrictedMapRDBSubScanSpec.java
index bd8a32a..596699f 100644
--- a/contrib/format-maprdb/src/main/java/org/apache/drill/exec/store/mapr/db/RestrictedMapRDBSubScanSpec.java
+++ b/contrib/format-maprdb/src/main/java/org/apache/drill/exec/store/mapr/db/RestrictedMapRDBSubScanSpec.java
@@ -188,32 +188,4 @@ public class RestrictedMapRDBSubScanSpec extends MapRDBSubScanSpec {
     currentIndex += numKeys;
   }
 
-  /**
-   * Returns the next row key in the iteration.
-   * @return the next row key in the iteration or null if no more row keys
-   */
-  @JsonIgnore
-  public String nextRowKey() {
-    if (hasRowKey()) {
-      // get the entry at the current index within this batch
-      Object o = rowKeyVector.getAccessor().getObject(currentIndex++);
-      if (o == null) {
-        throw new DrillRuntimeException("Encountered a null row key during restricted subscan !");
-      }
-
-      // this is specific to the way the hash join maintains its entries. once we have reached the max
-      // occupied index within a batch, move to the next one and reset the current index to 0
-      // TODO: we should try to abstract this out
-      if (currentIndex > maxOccupiedIndex) {
-        Pair<ValueVector, Integer> currentBatch = rjbatch.nextRowKeyBatch();
-        if (currentBatch != null) {
-          init(currentBatch);
-        }
-      }
-
-      return o.toString();
-    }
-    return null;
-  }
-
 }
diff --git a/contrib/format-maprdb/src/main/java/org/apache/drill/exec/store/mapr/db/json/JsonTableRangePartitionFunction.java b/contrib/format-maprdb/src/main/java/org/apache/drill/exec/store/mapr/db/json/JsonTableRangePartitionFunction.java
index acaa6ca..436347f 100644
--- a/contrib/format-maprdb/src/main/java/org/apache/drill/exec/store/mapr/db/json/JsonTableRangePartitionFunction.java
+++ b/contrib/format-maprdb/src/main/java/org/apache/drill/exec/store/mapr/db/json/JsonTableRangePartitionFunction.java
@@ -186,7 +186,7 @@ public class JsonTableRangePartitionFunction extends AbstractRangePartitionFunct
     // get the table handle from the table cache
     Table table = plugin.getJsonTableCache().getTable(tableName, userName);
 
-    // Set the condition to null such that all scan ranges are retrieved for the primary table.
+    // Get all scan ranges for the primary table.
     // The reason is the row keys could typically belong to any one of the tablets of the table, so
     // there is no use trying to get only limited set of scan ranges.
     // NOTE: here we use the restrictedScanRangeSizeMB because the range partitioning should be parallelized
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 21d6092..4df9c38 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
@@ -50,16 +50,6 @@ public class HashJoinPOP extends AbstractJoinPop {
   @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,
@@ -80,8 +70,6 @@ public class HashJoinPOP extends AbstractJoinPop {
                      List<JoinCondition> conditions,
                      JoinRelType joinType) {
     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
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/index/AbstractIndexDescriptor.java b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/index/AbstractIndexDescriptor.java
index f908ead..dd042da 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/index/AbstractIndexDescriptor.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/index/AbstractIndexDescriptor.java
@@ -71,4 +71,10 @@ public abstract class AbstractIndexDescriptor extends DrillIndexDefinition imple
       int numProjectedFields, GroupScan primaryGroupScan) {
     throw new UnsupportedOperationException("getCost() not supported for this index.");
   }
+
+  @Override
+  public boolean isAsyncIndex() {
+    return true;
+  }
+
 }
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/index/IndexDescriptor.java b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/index/IndexDescriptor.java
index d43ba81..3b63230 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/index/IndexDescriptor.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/index/IndexDescriptor.java
@@ -72,12 +72,19 @@ public interface IndexDescriptor extends IndexDefinition {
    * @param primaryGroupScan Primary table's GroupScan instance
    * @return a RelOptCost instance representing the total cost
    */
-  public RelOptCost getCost(IndexProperties indexProps, RelOptPlanner planner,
+  RelOptCost getCost(IndexProperties indexProps, RelOptPlanner planner,
       int numProjectedFields, GroupScan primaryGroupScan);
 
   /**
    * Get the costing factors associated with the storage/format plugin
    */
-  public PluginCost getPluginCostModel();
+  PluginCost getPluginCostModel();
+
+  /**
+   * Whether this index is maintained synchronously (i.e primary table updates are propagated to the index
+   * synchronously) or asynchronously with some delay.  The latter is more common for distributed NoSQL databases.
+   * @return True if the index is maintained asynchronously, False otherwise
+   */
+  boolean isAsyncIndex();
 
 }
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/index/IndexGroup.java b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/index/IndexGroup.java
index ea34ea5..3d3aeeb 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/index/IndexGroup.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/index/IndexGroup.java
@@ -33,11 +33,7 @@ public class IndexGroup {
   }
 
   public boolean isIntersectIndex() {
-    if (indexProps.size() > 1) {
-      return true;
-    } else {
-      return false;
-    }
+    return indexProps.size() > 1;
   }
 
   public int numIndexes() {
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
index f2091f6..b380c28 100644
--- 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
@@ -237,7 +237,8 @@ public class IndexIntersectPlanGenerator extends AbstractIndexPlanGenerator {
     return finalRel;
   }
 
-  private Pair<RelNode, DbGroupScan> buildRestrictedDBScan(RexNode remnant) {
+  private Pair<RelNode, DbGroupScan> buildRestrictedDBScan(RexNode remnant,
+      boolean isAnyIndexAsync) {
 
     DbGroupScan origDbGroupScan = (DbGroupScan)IndexPlanUtils.getGroupScan(origScan);
     List<SchemaPath> cols = new ArrayList<SchemaPath>(origDbGroupScan.getColumns());
@@ -266,9 +267,16 @@ public class IndexIntersectPlanGenerator extends AbstractIndexPlanGenerator {
     final RelDataTypeFactory.FieldInfoBuilder leftFieldTypeBuilder =
         dbScan.getCluster().getTypeFactory().builder();
 
-    FilterPrel leftIndexFilterPrel = new FilterPrel(dbScan.getCluster(), dbScan.getTraitSet(),
-          dbScan, indexContext.getOrigCondition());
-    lastRelNode = leftIndexFilterPrel;
+    FilterPrel leftIndexFilterPrel = null;
+
+    // See NonCoveringIndexPlanGenerator for why we are re-applying index filter condition in case of async indexes.
+    // For intersect planning, any one of the intersected indexes may be async but to keep it simple we re-apply the
+    // full original condition.
+    if (isAnyIndexAsync) {
+      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;
@@ -301,8 +309,12 @@ public class IndexIntersectPlanGenerator extends AbstractIndexPlanGenerator {
   @Override
   public RelNode convertChild(final RelNode filter, final RelNode input) throws InvalidRelException {
     Map<IndexDescriptor, RexNode> idxConditionMap = Maps.newLinkedHashMap();
+    boolean isAnyIndexAsync = false;
     for(IndexDescriptor idx : indexInfoMap.keySet()) {
       idxConditionMap.put(idx, indexInfoMap.get(idx).indexCondition);
+      if (!isAnyIndexAsync && idx.isAsyncIndex()) {
+        isAnyIndexAsync = true;
+      }
     }
 
     RelNode indexPlan = null;
@@ -323,7 +335,7 @@ public class IndexIntersectPlanGenerator extends AbstractIndexPlanGenerator {
 
     //now with index plan constructed, build plan of left(probe) side to use restricted db scan
 
-    Pair<RelNode, DbGroupScan> leftRelAndScan = buildRestrictedDBScan(remnant);
+    Pair<RelNode, DbGroupScan> leftRelAndScan = buildRestrictedDBScan(remnant, isAnyIndexAsync);
 
     RelNode finalRel = buildRowKeyJoin(leftRelAndScan.left, rangeDistRight, true, JoinControl.DEFAULT);
     if ( upperProject != null) {
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
index e1337bc..c1bcf68 100644
--- 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
@@ -205,16 +205,17 @@ public class NonCoveringIndexPlanGenerator extends AbstractIndexPlanGenerator {
     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;
+    // We are applying the same index condition to primary table's restricted scan. The reason is, the index may be an async
+    // index .. i.e it is not synchronously updated along with the primary table update as part of a single transaction, so it
+    // is possible that after or during index scan, the primary table rows may have been updated and no longer satisfy the index
+    // condition. By re-applying the index condition here, we will ensure non-qualifying records are filtered out.
+    // The remainder condition will be applied on top of RowKeyJoin.
+    FilterPrel leftIndexFilterPrel = null;
+    if (indexDesc.isAsyncIndex()) {
+      leftIndexFilterPrel = new FilterPrel(dbScan.getCluster(), dbScan.getTraitSet(),
+            dbScan, indexContext.getOrigCondition());
+      lastLeft = leftIndexFilterPrel;
+    }
 
     RelDataType origRowType = origProject == null ? origScan.getRowType() : origProject.getRowType();
 
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 0626483..6d52184 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,13 @@ 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);
+        // NOTE: earlier, for Range Distribution we were creating an OrderedPartitionExchange; however that operator is not actually
+        // used in any of the query plans because Drill's Sort does not do range based sorting (it does a HashToRandomExchange followed
+        // by a Sort).  Here, we are generating a RangePartitionExchange instead of OrderedPartitionExchange. The run-time implementation
+        // of RPE is a much simpler operator..it just does 'bucketing' based on ranges.  Also, it allows a parameter to specify the
+        // partitioning function whereas the OPE does a much more complex inferencing to determine which partition goes where. In future,
+        // if we do want to leverage OPE then we could create a new type of distribution trait or make the DistributionType a
+        // class instead of a simple enum and then we can distinguish whether an OPE or RPE is needed.
         return new RangePartitionExchangePrel(rel.getCluster(),
             planner.emptyTraitSet().plus(Prel.DRILL_PHYSICAL).plus(toDist), rel,
             toDist.getFields(), toDist.getPartitionFunction());
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/physical/RowKeyJoinPrel.java b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/physical/RowKeyJoinPrel.java
index 616eb56..7e8f77e 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/physical/RowKeyJoinPrel.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/physical/RowKeyJoinPrel.java
@@ -82,8 +82,11 @@ public class RowKeyJoinPrel extends JoinPrel implements Prel {
     }
     double rowCount = mq.getRowCount(this.getRight());
     DrillCostFactory costFactory = (DrillCostFactory) planner.getCostFactory();
-    return costFactory.makeCost(rowCount, 0, 0, 0,
-        0 /* mem cost is 0 because this operator does not make any extra copy of either the left or right batches */);
+
+    // RowKeyJoin operator by itself incurs negligible CPU and I/O cost since it is not doing a real join.
+    // The actual cost is attributed to the skip-scan (random I/O). The RK join will hold 1 batch in memory but
+    // it is not making any extra copy of either the left or right batches, so the memory cost is 0
+    return costFactory.makeCost(rowCount, 0, 0, 0, 0);
   }
 
   @Override


[drill] 04/08: DRILL-6381: (Part 4) Enhance MapR-DB plugin to support querying secondary indexes

Posted by am...@apache.org.
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 2a9e51f8a68872a77e38ee91be107868f60fd334
Author: rebase <bu...@mapr.com>
AuthorDate: Fri Mar 16 14:24:20 2018 -0700

    DRILL-6381: (Part 4) Enhance MapR-DB plugin to support querying secondary indexes
    
      1. Implementation of the index descriptor for MapR-DB.
      2. MapR-DB specific costing for covering and non-covering indexes.
      3. Discovery componenent to discover the indexes available for a MapR-DB table including CAST functional indexes.
      4. Utility functions to build a canonical index descriptor.
      5. Statistics: fetch and initialize statistcs from MapR-DB for a query condition. Maintain a query-scoped cache for the statistics. Utility functions to compute selectivity.
      6. Range Partitioning: partitioning function that takes into account the tablet map to find out where a particular rowkey belongs.
      7. Restricted Scan: support doing restricted (i.e skip) scan through lookups on the rowkey. Added a group-scan and record reader for this.
      8. MD-3726: Simple Order by queries (without limit) when an index is used are showing regression.
      9. MD-3995: Do not pushdown limit 0 past project with CONVERT_FROMJSON
      10. MD-4259 : Account for limit during hashcode computation
    
    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:
    	contrib/format-maprdb/src/main/java/org/apache/drill/exec/store/mapr/db/MapRDBFormatMatcher.java
    	contrib/format-maprdb/src/main/java/org/apache/drill/exec/store/mapr/db/MapRDBPushProjectIntoScan.java
    	contrib/format-maprdb/src/main/java/org/apache/drill/exec/store/mapr/db/json/JsonTableGroupScan.java
    	exec/java-exec/src/main/java/org/apache/drill/exec/planner/index/rules/DbScanSortRemovalRule.java
    	exec/java-exec/src/main/java/org/apache/drill/exec/planner/physical/SortPrel.java
    	exec/java-exec/src/main/java/org/apache/drill/exec/planner/physical/TopNPrel.java
    
    Fix additional compilation issues.
---
 .../exec/planner/index/MapRDBIndexDescriptor.java  |  222 +++
 .../exec/planner/index/MapRDBIndexDiscover.java    |  374 +++++
 .../drill/exec/planner/index/MapRDBStatistics.java |  689 +++++++-
 .../exec/store/mapr/db/MapRDBFormatMatcher.java    |   42 +
 .../exec/store/mapr/db/MapRDBFormatPlugin.java     |    2 +-
 .../drill/exec/store/mapr/db/MapRDBGroupScan.java  |   12 +-
 .../store/mapr/db/MapRDBPushLimitIntoScan.java     |   50 +-
 .../store/mapr/db/MapRDBPushProjectIntoScan.java   |    3 +-
 .../store/mapr/db/json/JsonTableGroupScan.java     |  198 ++-
 .../db/json/JsonTableRangePartitionFunction.java   |  237 +++
 .../mapr/db/json/RestrictedJsonTableGroupScan.java |  184 +++
 .../maprdb/tests/index/IndexHintPlanTest.java      |  171 ++
 .../drill/maprdb/tests/index/IndexPlanTest.java    | 1715 ++++++++++++++++++++
 .../drill/maprdb/tests/index/LargeTableGen.java    |  176 ++
 .../maprdb/tests/index/LargeTableGenBase.java      |  186 +++
 .../drill/maprdb/tests/index/StatisticsTest.java   |  115 ++
 .../drill/maprdb/tests/index/TableIndexCmd.java    |  127 ++
 .../drill/exec/planner/common/OrderedRel.java      |   53 +
 .../drill/exec/planner/index/IndexCallContext.java |    4 +-
 .../planner/index/IndexLogicalPlanCallContext.java |    4 +-
 .../index/IndexPhysicalPlanCallContext.java        |   12 +-
 .../drill/exec/planner/index/IndexPlanUtils.java   |   17 +
 .../generators/AbstractIndexPlanGenerator.java     |   49 +-
 .../generators/CoveringPlanNoFilterGenerator.java  |   16 +-
 .../generators/NonCoveringIndexPlanGenerator.java  |    2 +-
 .../planner/index/rules/DbScanSortRemovalRule.java |   53 +-
 .../drill/exec/planner/logical/DrillSortRel.java   |   17 +-
 .../drill/exec/planner/physical/SortPrel.java      |   28 +-
 .../drill/exec/planner/physical/SortPrule.java     |   16 +-
 .../drill/exec/planner/physical/TopNPrel.java      |   31 +-
 .../visitor/ExcessiveExchangeIdentifier.java       |   28 +
 31 files changed, 4746 insertions(+), 87 deletions(-)

diff --git a/contrib/format-maprdb/src/main/java/org/apache/drill/exec/planner/index/MapRDBIndexDescriptor.java b/contrib/format-maprdb/src/main/java/org/apache/drill/exec/planner/index/MapRDBIndexDescriptor.java
new file mode 100644
index 0000000..a57f5b5
--- /dev/null
+++ b/contrib/format-maprdb/src/main/java/org/apache/drill/exec/planner/index/MapRDBIndexDescriptor.java
@@ -0,0 +1,222 @@
+/*
+ * 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.Collection;
+import java.util.List;
+import java.util.Set;
+
+import com.google.common.collect.Lists;
+import com.google.common.collect.Sets;
+
+import org.apache.calcite.plan.RelOptCost;
+import org.apache.calcite.plan.RelOptPlanner;
+import org.apache.calcite.rel.RelFieldCollation.NullDirection;
+import org.apache.drill.common.expression.SchemaPath;
+import org.apache.drill.exec.expr.CloneVisitor;
+import org.apache.drill.exec.physical.base.DbGroupScan;
+import org.apache.drill.exec.physical.base.GroupScan;
+import org.apache.drill.exec.planner.cost.DrillCostBase;
+import org.apache.drill.exec.planner.cost.DrillCostBase.DrillCostFactory;
+import org.apache.drill.exec.planner.cost.PluginCost;
+import org.apache.drill.exec.planner.index.IndexProperties;
+import org.apache.drill.exec.store.mapr.PluginConstants;
+import org.apache.drill.exec.util.EncodedSchemaPathSet;
+import org.apache.drill.common.expression.LogicalExpression;
+
+import com.google.common.base.Preconditions;
+import com.google.common.collect.ImmutableSet;
+
+public class MapRDBIndexDescriptor extends DrillIndexDescriptor {
+
+  protected final Object desc;
+  protected final Set<LogicalExpression> allFields;
+  protected final Set<LogicalExpression> indexedFields;
+  protected MapRDBFunctionalIndexInfo functionalInfo;
+  protected PluginCost pluginCost;
+
+  public MapRDBIndexDescriptor(List<LogicalExpression> indexCols,
+                               CollationContext indexCollationContext,
+                               List<LogicalExpression> nonIndexCols,
+                               List<LogicalExpression> rowKeyColumns,
+                               String indexName,
+                               String tableName,
+                               IndexType type,
+                               Object desc,
+                               DbGroupScan scan,
+                               NullDirection nullsDirection) {
+    super(indexCols, indexCollationContext, nonIndexCols, rowKeyColumns, indexName, tableName, type, nullsDirection);
+    this.desc = desc;
+    this.indexedFields = ImmutableSet.copyOf(indexColumns);
+    this.allFields = new ImmutableSet.Builder<LogicalExpression>()
+        .add(PluginConstants.DOCUMENT_SCHEMA_PATH)
+        .addAll(indexColumns)
+        .addAll(nonIndexColumns)
+        .build();
+    this.pluginCost = scan.getPluginCostModel();
+  }
+
+  public Object getOriginalDesc(){
+    return desc;
+  }
+
+  @Override
+  public boolean isCoveringIndex(List<LogicalExpression> expressions) {
+    List<LogicalExpression> decodedCols = new DecodePathinExpr().parseExpressions(expressions);
+    return columnsInIndexFields(decodedCols, allFields);
+  }
+
+  @Override
+  public boolean allColumnsIndexed(Collection<LogicalExpression> expressions) {
+    List<LogicalExpression> decodedCols = new DecodePathinExpr().parseExpressions(expressions);
+    return columnsInIndexFields(decodedCols, indexedFields);
+  }
+
+  @Override
+  public boolean someColumnsIndexed(Collection<LogicalExpression> columns) {
+    return columnsIndexed(columns, false);
+  }
+
+  private boolean columnsIndexed(Collection<LogicalExpression> expressions, boolean allColsIndexed) {
+    List<LogicalExpression> decodedCols = new DecodePathinExpr().parseExpressions(expressions);
+    if (allColsIndexed) {
+      return columnsInIndexFields(decodedCols, indexedFields);
+    } else {
+      return someColumnsInIndexFields(decodedCols, indexedFields);
+    }
+  }
+
+  public FunctionalIndexInfo getFunctionalInfo() {
+    if (this.functionalInfo == null) {
+      this.functionalInfo = new MapRDBFunctionalIndexInfo(this);
+    }
+    return this.functionalInfo;
+  }
+
+  /**
+   * Search through a LogicalExpression, finding all referenced schema paths
+   * and replace them with decoded paths.
+   * If one encoded path could be decoded to multiple paths, add these decoded paths to
+   * the end of returned list of expressions from parseExpressions.
+   */
+  private class DecodePathinExpr extends CloneVisitor {
+    Set<SchemaPath> schemaPathSet = Sets.newHashSet();
+
+    public List<LogicalExpression> parseExpressions(Collection<LogicalExpression> expressions) {
+      List<LogicalExpression> allCols = Lists.newArrayList();
+      Collection<SchemaPath> decoded;
+
+      for(LogicalExpression expr : expressions) {
+        LogicalExpression nonDecoded = expr.accept(this, null);
+        if(nonDecoded != null) {
+          allCols.add(nonDecoded);
+        }
+      }
+
+      if (schemaPathSet.size() > 0) {
+        decoded = EncodedSchemaPathSet.decode(schemaPathSet);
+        allCols.addAll(decoded);
+      }
+      return allCols;
+    }
+
+    @Override
+    public LogicalExpression visitSchemaPath(SchemaPath path, Void value) {
+      if (EncodedSchemaPathSet.isEncodedSchemaPath(path)) {
+        // if decoded size is not one, incoming path is encoded path thus there is no cast or other function applied on it,
+        // since users won't pass in encoded fields, so it is safe to return null,
+        schemaPathSet.add(path);
+        return null;
+      } else {
+        return path;
+      }
+    }
+
+  }
+
+  @Override
+  public RelOptCost getCost(IndexProperties indexProps, RelOptPlanner planner,
+      int numProjectedFields, GroupScan primaryTableGroupScan) {
+    Preconditions.checkArgument(primaryTableGroupScan instanceof DbGroupScan);
+    DbGroupScan dbGroupScan = (DbGroupScan) primaryTableGroupScan;
+    DrillCostFactory costFactory = (DrillCostFactory)planner.getCostFactory();
+    double totalRows = indexProps.getTotalRows();
+    double leadRowCount = indexProps.getLeadingSelectivity() * totalRows;
+    double avgRowSize = indexProps.getAvgRowSize();
+    if (indexProps.isCovering()) { // covering index
+      // int numIndexCols = allFields.size();
+      // for disk i/o, all index columns are going to be read into memory
+      double numBlocks = Math.ceil((leadRowCount * avgRowSize)/pluginCost.getBlockSize(primaryTableGroupScan));
+      double diskCost = numBlocks * pluginCost.getSequentialBlockReadCost(primaryTableGroupScan);
+      // cpu cost is cost of filter evaluation for the remainder condition
+      double cpuCost = 0.0;
+      if (indexProps.getTotalRemainderFilter() != null) {
+        cpuCost = leadRowCount * DrillCostBase.COMPARE_CPU_COST;
+      }
+      double networkCost = 0.0; // TODO: add network cost once full table scan also considers network cost
+      return costFactory.makeCost(leadRowCount, cpuCost, diskCost, networkCost);
+
+    } else { // non-covering index
+      // int numIndexCols = allFields.size();
+      double numBlocksIndex = Math.ceil((leadRowCount * avgRowSize)/pluginCost.getBlockSize(primaryTableGroupScan));
+      double diskCostIndex = numBlocksIndex * pluginCost.getSequentialBlockReadCost(primaryTableGroupScan);
+      // 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
+      double totalBlocksPrimary = Math.ceil((dbGroupScan.getColumns().size() *
+          pluginCost.getAverageColumnSize(primaryTableGroupScan) * totalRows)/
+          pluginCost.getBlockSize(primaryTableGroupScan));
+      double diskBlocksPrimary = Math.min(totalBlocksPrimary, leadRowCount);
+      double diskCostPrimary = diskBlocksPrimary * pluginCost.getRandomBlockReadCost(primaryTableGroupScan);
+      double diskCostTotal = diskCostIndex + diskCostPrimary;
+
+      // cpu cost of remainder condition evaluation over the selected rows
+      double cpuCost = 0.0;
+      if (indexProps.getTotalRemainderFilter() != null) {
+        cpuCost = leadRowCount * DrillCostBase.COMPARE_CPU_COST;
+      }
+      double networkCost = 0.0; // TODO: add network cost once full table scan also considers network cost
+      return costFactory.makeCost(leadRowCount, cpuCost, diskCostTotal, networkCost);
+    }
+  }
+
+  // Future use once full table scan also includes network cost
+  private double getNetworkCost(double leadRowCount, int numProjectedFields, boolean isCovering,
+      GroupScan primaryTableGroupScan) {
+    if (isCovering) {
+      // db server will send only the projected columns to the db client for the selected
+      // number of rows, so network cost is based on the number of actual projected columns
+      double networkCost = leadRowCount * numProjectedFields * pluginCost.getAverageColumnSize(primaryTableGroupScan);
+      return networkCost;
+    } else {
+      // only the rowkey column is projected from the index and sent over the network
+      double networkCostIndex = leadRowCount * 1 * pluginCost.getAverageColumnSize(primaryTableGroupScan);
+
+      // after join-back to primary table, all projected columns are sent over the network
+      double networkCostPrimary = leadRowCount * numProjectedFields * pluginCost.getAverageColumnSize(primaryTableGroupScan);
+
+      return networkCostIndex + networkCostPrimary;
+    }
+
+  }
+
+  @Override
+  public PluginCost getPluginCostModel() {
+    return pluginCost;
+  }
+}
diff --git a/contrib/format-maprdb/src/main/java/org/apache/drill/exec/planner/index/MapRDBIndexDiscover.java b/contrib/format-maprdb/src/main/java/org/apache/drill/exec/planner/index/MapRDBIndexDiscover.java
new file mode 100644
index 0000000..e1b8a61
--- /dev/null
+++ b/contrib/format-maprdb/src/main/java/org/apache/drill/exec/planner/index/MapRDBIndexDiscover.java
@@ -0,0 +1,374 @@
+/*
+ * 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 com.google.common.collect.Maps;
+import com.mapr.db.Admin;
+import com.mapr.db.MapRDB;
+import com.mapr.db.exceptions.DBException;
+import com.mapr.db.index.IndexDesc;
+import com.mapr.db.index.IndexDesc.MissingAndNullOrdering;
+import com.mapr.db.index.IndexFieldDesc;
+import org.antlr.runtime.ANTLRStringStream;
+import org.antlr.runtime.CommonTokenStream;
+import org.apache.calcite.rel.RelFieldCollation;
+import org.apache.calcite.rel.RelFieldCollation.NullDirection;
+import org.apache.calcite.sql.type.SqlTypeName;
+import org.apache.drill.common.exceptions.DrillRuntimeException;
+import org.apache.drill.common.expression.LogicalExpression;
+import org.apache.drill.common.expression.SchemaPath;
+import org.apache.drill.common.expression.parser.ExprLexer;
+import org.apache.drill.common.expression.parser.ExprParser;
+import org.apache.drill.common.types.TypeProtos;
+import org.apache.drill.common.types.Types;
+import org.apache.drill.exec.physical.base.AbstractDbGroupScan;
+import org.apache.drill.exec.physical.base.GroupScan;
+import org.apache.drill.exec.planner.common.DrillScanRelBase;
+import org.apache.drill.exec.planner.logical.DrillTable;
+import org.apache.drill.exec.planner.physical.ScanPrel;
+import org.apache.drill.exec.store.dfs.DrillFileSystem;
+import org.apache.drill.exec.store.dfs.FileSelection;
+import org.apache.drill.exec.store.dfs.FileSystemPlugin;
+import org.apache.drill.exec.store.mapr.db.MapRDBFormatMatcher;
+import org.apache.drill.exec.store.mapr.db.MapRDBFormatPlugin;
+import org.apache.drill.exec.store.mapr.db.MapRDBGroupScan;
+import org.apache.drill.exec.store.mapr.db.json.FieldPathHelper;
+import org.apache.drill.exec.util.ImpersonationUtil;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.security.UserGroupInformation;
+import org.ojai.FieldPath;
+
+import java.io.IOException;
+import java.security.PrivilegedExceptionAction;
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+
+public class MapRDBIndexDiscover extends IndexDiscoverBase implements IndexDiscover {
+
+  static final String DEFAULT_STRING_CAST_LEN_STR = "256";
+
+  public MapRDBIndexDiscover(GroupScan inScan, DrillScanRelBase scanRel) {
+    super((AbstractDbGroupScan) inScan, scanRel);
+  }
+
+  public MapRDBIndexDiscover(GroupScan inScan, ScanPrel scanRel) {
+    super((AbstractDbGroupScan) inScan, scanRel);
+  }
+
+  @Override
+  public IndexCollection getTableIndex(String tableName) {
+    //return getTableIndexFromCommandLine(tableName);
+    return getTableIndexFromMFS(tableName);
+  }
+
+  /**
+   *
+   * @param tableName
+   * @return
+   */
+  private IndexCollection getTableIndexFromMFS(String tableName) {
+    try {
+      Set<DrillIndexDescriptor> idxSet = new HashSet<>();
+      Collection<IndexDesc> indexes = admin().getTableIndexes(new Path(tableName));
+      if (indexes.size() == 0 ) {
+        logger.error("No index returned from Admin.getTableIndexes for table {}", tableName);
+        return null;
+      }
+      for (IndexDesc idx : indexes) {
+        DrillIndexDescriptor hbaseIdx = buildIndexDescriptor(tableName, idx);
+        if (hbaseIdx == null) {
+          //not able to build a valid index based on the index info from MFS
+          logger.error("Not able to build index for {}", idx.toString());
+          continue;
+        }
+        idxSet.add(hbaseIdx);
+      }
+      if (idxSet.size() == 0) {
+        logger.error("No index found for table {}.", tableName);
+        return null;
+      }
+      return new DrillIndexCollection(getOriginalScanRel(), idxSet);
+    } catch (DBException ex) {
+      logger.error("Could not get table index from File system.", ex);
+    }
+    catch(InvalidIndexDefinitionException ex) {
+      logger.error("Invalid index definition detected.", ex);
+    }
+    return null;
+  }
+
+  FileSelection deriveFSSelection(DrillFileSystem fs, IndexDescriptor idxDesc) throws IOException {
+    String tableName = idxDesc.getTableName();
+    String[] tablePath = tableName.split("/");
+    String tableParent = tableName.substring(0, tableName.lastIndexOf("/"));
+
+    return FileSelection.create(fs, tableParent, tablePath[tablePath.length - 1], false);
+  }
+
+  @Override
+  public DrillTable getNativeDrillTable(IndexDescriptor idxDescriptor) {
+
+    try {
+      final AbstractDbGroupScan origScan = getOriginalScan();
+      if (!(origScan instanceof MapRDBGroupScan)) {
+        return null;
+      }
+      MapRDBFormatPlugin maprFormatPlugin = ((MapRDBGroupScan) origScan).getFormatPlugin();
+      FileSystemPlugin fsPlugin = (FileSystemPlugin) (((MapRDBGroupScan) origScan).getStoragePlugin());
+
+      DrillFileSystem fs = ImpersonationUtil.createFileSystem(origScan.getUserName(), fsPlugin.getFsConf());
+      MapRDBFormatMatcher matcher = (MapRDBFormatMatcher) (maprFormatPlugin.getMatcher());
+      FileSelection fsSelection = deriveFSSelection(fs, idxDescriptor);
+      return matcher.isReadableIndex(fs, fsSelection, fsPlugin, fsPlugin.getName(),
+          origScan.getUserName(), idxDescriptor);
+
+    } catch (Exception e) {
+      logger.error("Failed to get native DrillTable.", e);
+    }
+    return null;
+  }
+
+  private SchemaPath fieldName2SchemaPath(String fieldName) {
+    if (fieldName.contains(":")) {
+      fieldName = fieldName.split(":")[1];
+    }
+    if (fieldName.contains(".")) {
+      return FieldPathHelper.fieldPath2SchemaPath(FieldPath.parseFrom(fieldName));
+    }
+    return SchemaPath.getSimplePath(fieldName);
+  }
+
+  String getDrillTypeStr(String maprdbTypeStr) {
+    String typeStr = maprdbTypeStr.toUpperCase();
+    String[] typeTokens = typeStr.split("[)(]");
+    String typeData = DEFAULT_STRING_CAST_LEN_STR;
+    if(typeTokens.length > 1) {
+      typeStr = typeTokens[0];
+      typeData = typeTokens[1];
+    }
+    switch(typeStr){
+      case "STRING":
+        // set default width since it is not specified
+        return "VARCHAR("+typeData+")";
+      case "LONG":
+        return "BIGINT";
+      case "INT":
+      case "INTEGER":
+        return "INT";
+      case "FLOAT":
+        return "FLOAT4";
+      case "DOUBLE":
+        return "FLOAT8";
+      case "INTERVAL_YEAR_MONTH":
+        return "INTERVALYEAR";
+      case "INTERVAL_DAY_TIME":
+        return "INTERVALDAY";
+      case "BOOLEAN":
+        return "BIT";
+      case "BINARY":
+        return "VARBINARY";
+      case "ANY":
+      case "DECIMAL":
+        return null;
+      default: return typeStr;
+    }
+
+  }
+
+  TypeProtos.MajorType getDrillType(String typeStr) {
+    switch(typeStr){
+      case "VARCHAR":
+      case "CHAR":
+      case "STRING":
+        // set default width since it is not specified
+        return
+            Types.required(TypeProtos.MinorType.VARCHAR).toBuilder().setWidth(
+                getOriginalScanRel().getCluster().getTypeFactory().createSqlType(SqlTypeName.VARCHAR).getPrecision()).build();
+      case "LONG":
+      case "BIGINT":
+        return Types.required(TypeProtos.MinorType.BIGINT);
+      case "INT":
+      case "INTEGER":
+        return Types.required(TypeProtos.MinorType.INT);
+      case "FLOAT":
+        return Types.required(TypeProtos.MinorType.FLOAT4);
+      case "DOUBLE":
+        return Types.required(TypeProtos.MinorType.FLOAT8);
+      case "INTERVAL_YEAR_MONTH":
+        return Types.required(TypeProtos.MinorType.INTERVALYEAR);
+      case "INTERVAL_DAY_TIME":
+        return Types.required(TypeProtos.MinorType.INTERVALDAY);
+      case "BOOLEAN":
+        return Types.required(TypeProtos.MinorType.BIT);
+      case "BINARY":
+        return Types.required(TypeProtos.MinorType.VARBINARY).toBuilder().build();
+      case "ANY":
+      case "DECIMAL":
+        return null;
+      default: return Types.required(TypeProtos.MinorType.valueOf(typeStr));
+    }
+  }
+
+  private LogicalExpression castFunctionSQLSyntax(String field, String type) throws InvalidIndexDefinitionException {
+    //get castTypeStr so we can construct SQL syntax string before MapRDB could provide such syntax
+    String castTypeStr = getDrillTypeStr(type);
+    if(castTypeStr == null) {//no cast
+      throw new InvalidIndexDefinitionException("cast function type not recognized: " + type + "for field " + field);
+    }
+    try {
+      String castFunc = String.format("cast( %s as %s)", field, castTypeStr);
+      final ExprLexer lexer = new ExprLexer(new ANTLRStringStream(castFunc));
+      final CommonTokenStream tokens = new CommonTokenStream(lexer);
+      final ExprParser parser = new ExprParser(tokens);
+      final ExprParser.parse_return ret = parser.parse();
+      logger.trace("{}, {}", tokens, ret);
+      return ret.e;
+    }catch(Exception ex) {
+      logger.error("parse failed{}", ex);
+    }
+    return null;
+  }
+
+  private LogicalExpression getIndexExpression(IndexFieldDesc desc) throws InvalidIndexDefinitionException {
+    final String fieldName = desc.getFieldPath().asPathString();
+    final String functionDef = desc.getFunctionName();
+    if ((functionDef != null)) {//this is a function
+      String[] tokens = functionDef.split("\\s+");
+      if (tokens[0].equalsIgnoreCase("cast")) {
+        if (tokens.length != 3) {
+          throw new InvalidIndexDefinitionException("cast function definition not recognized: " + functionDef);
+        }
+        LogicalExpression idxExpr = castFunctionSQLSyntax(fieldName, tokens[2]);
+        if (idxExpr == null) {
+          throw new InvalidIndexDefinitionException("got null expression for function definition: " + functionDef);
+        }
+        return idxExpr;
+      } else {
+        throw new InvalidIndexDefinitionException("function definition is not supported for indexing: " + functionDef);
+      }
+    }
+    //else it is a schemaPath
+    return fieldName2SchemaPath(fieldName);
+  }
+
+  private List<LogicalExpression> field2SchemaPath(Collection<IndexFieldDesc> descCollection)
+      throws InvalidIndexDefinitionException {
+    List<LogicalExpression> listSchema = new ArrayList<>();
+    for (IndexFieldDesc field : descCollection) {
+        listSchema.add(getIndexExpression(field));
+    }
+    return listSchema;
+  }
+
+  private List<RelFieldCollation> getFieldCollations(IndexDesc desc, Collection<IndexFieldDesc> descCollection) {
+    List<RelFieldCollation> fieldCollations = new ArrayList<>();
+    int i=0;
+    for (IndexFieldDesc field : descCollection) {
+      RelFieldCollation.Direction direction = (field.getSortOrder() == IndexFieldDesc.Order.Asc) ?
+          RelFieldCollation.Direction.ASCENDING : (field.getSortOrder() == IndexFieldDesc.Order.Desc ?
+              RelFieldCollation.Direction.DESCENDING : null);
+      if (direction != null) {
+        // assume null direction of NULLS UNSPECIFIED for now until MapR-DB adds that to the APIs
+        RelFieldCollation.NullDirection nulldir =
+            desc.getMissingAndNullOrdering() == MissingAndNullOrdering.MissingAndNullFirst ? NullDirection.FIRST :
+            (desc.getMissingAndNullOrdering() == MissingAndNullOrdering.MissingAndNullLast ?
+                NullDirection.LAST : NullDirection.UNSPECIFIED);
+        RelFieldCollation c = new RelFieldCollation(i++, direction, nulldir);
+        fieldCollations.add(c);
+      } else {
+        // if the direction is not present for a field, no need to examine remaining fields
+        break;
+      }
+    }
+    return fieldCollations;
+  }
+
+  private CollationContext buildCollationContext(List<LogicalExpression> indexFields,
+      List<RelFieldCollation> indexFieldCollations) {
+    assert indexFieldCollations.size() <= indexFields.size();
+    Map<LogicalExpression, RelFieldCollation> collationMap = Maps.newHashMap();
+    for (int i = 0; i < indexFieldCollations.size(); i++) {
+      collationMap.put(indexFields.get(i), indexFieldCollations.get(i));
+    }
+    CollationContext collationContext = new CollationContext(collationMap, indexFieldCollations);
+    return collationContext;
+  }
+
+  private DrillIndexDescriptor buildIndexDescriptor(String tableName, IndexDesc desc)
+      throws InvalidIndexDefinitionException {
+    if (desc.isExternal()) {
+      //XX: not support external index
+      return null;
+    }
+
+    IndexDescriptor.IndexType idxType = IndexDescriptor.IndexType.NATIVE_SECONDARY_INDEX;
+    List<LogicalExpression> indexFields = field2SchemaPath(desc.getIndexedFields());
+    List<LogicalExpression> coveringFields = field2SchemaPath(desc.getIncludedFields());
+    coveringFields.add(SchemaPath.getSimplePath("_id"));
+    CollationContext collationContext = null;
+    if (!desc.isHashed()) { // hash index has no collation property
+      List<RelFieldCollation> indexFieldCollations = getFieldCollations(desc, desc.getIndexedFields());
+      collationContext = buildCollationContext(indexFields, indexFieldCollations);
+    }
+
+    DrillIndexDescriptor idx = new MapRDBIndexDescriptor (
+        indexFields,
+        collationContext,
+        coveringFields,
+        null,
+        desc.getIndexName(),
+        tableName,
+        idxType,
+        desc,
+        this.getOriginalScan(),
+        desc.getMissingAndNullOrdering() == MissingAndNullOrdering.MissingAndNullFirst ? NullDirection.FIRST :
+            (desc.getMissingAndNullOrdering() == MissingAndNullOrdering.MissingAndNullLast ?
+                NullDirection.LAST : NullDirection.UNSPECIFIED));
+
+    String storageName = this.getOriginalScan().getStoragePlugin().getName();
+    materializeIndex(storageName, idx);
+    return idx;
+  }
+
+  @SuppressWarnings("deprecation")
+  private Admin admin() {
+    assert getOriginalScan() instanceof MapRDBGroupScan;
+
+    final MapRDBGroupScan dbGroupScan = (MapRDBGroupScan) getOriginalScan();
+    final UserGroupInformation currentUser = ImpersonationUtil.createProxyUgi(dbGroupScan.getUserName());
+    final Configuration conf = dbGroupScan.getFormatPlugin().getFsConf();
+
+    final Admin admin;
+    try {
+      admin = currentUser.doAs(new PrivilegedExceptionAction<Admin>() {
+        public Admin run() throws Exception {
+          return MapRDB.getAdmin(conf);
+        }
+      });
+    } catch (Exception e) {
+      throw new DrillRuntimeException("Failed to get Admin instance for user: " + currentUser.getUserName(), e);
+    }
+    return admin;
+  }
+}
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 3b8de34..e129b96 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
@@ -17,20 +17,49 @@
  */
 package org.apache.drill.exec.planner.index;
 
+import com.google.common.base.Charsets;
+import com.google.common.base.Preconditions;
+import com.google.common.collect.Lists;
+
+import com.google.common.collect.Maps;
 import org.apache.calcite.plan.RelOptUtil;
 import org.apache.calcite.rel.RelNode;
+import org.apache.calcite.rel.metadata.RelMdUtil;
 import org.apache.calcite.rel.type.RelDataType;
+import org.apache.calcite.rex.RexBuilder;
 import org.apache.calcite.rex.RexCall;
+import org.apache.calcite.rex.RexLiteral;
 import org.apache.calcite.rex.RexNode;
 import org.apache.calcite.rex.RexInputRef;
+import org.apache.calcite.rex.RexUtil;
 import org.apache.calcite.sql.SqlKind;
+import org.apache.calcite.sql.fun.SqlStdOperatorTable;
+import org.apache.calcite.sql.type.SqlTypeName;
+import org.apache.calcite.util.Pair;
+import org.apache.drill.common.expression.ExpressionStringBuilder;
+import org.apache.drill.common.expression.LogicalExpression;
 import org.apache.drill.exec.physical.base.DbGroupScan;
+import org.apache.drill.exec.physical.base.GroupScan;
+import org.apache.drill.exec.planner.common.DrillRelOptUtil;
 import org.apache.drill.exec.planner.common.DrillScanRelBase;
+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.PlannerSettings;
+import org.apache.drill.exec.planner.physical.PrelUtil;
 import org.apache.drill.exec.planner.physical.ScanPrel;
+import org.apache.drill.exec.store.hbase.HBaseRegexParser;
+import org.apache.drill.exec.store.mapr.db.json.JsonTableGroupScan;
+import org.apache.hadoop.hbase.HConstants;
 import org.ojai.store.QueryCondition;
 
+import java.io.UnsupportedEncodingException;
+import java.util.Arrays;
+import java.util.ArrayList;
+import java.util.Iterator;
+import java.util.List;
 import java.util.HashMap;
+import java.util.HashSet;
 import java.util.Map;
 
 public class MapRDBStatistics implements Statistics {
@@ -260,16 +289,332 @@ public class MapRDBStatistics implements Statistics {
   }
 
   public boolean initialize(RexNode condition, DrillScanRelBase scanRel, IndexCallContext context) {
-    //XXX to implement for complete secondary index framework
+    GroupScan scan = IndexPlanUtils.getGroupScan(scanRel);
+
+    PlannerSettings settings = PrelUtil.getPlannerSettings(scanRel.getCluster().getPlanner());
+    rowKeyJoinBackIOFactor = settings.getIndexRowKeyJoinCostFactor();
+    if (scan instanceof DbGroupScan) {
+      String conditionAsStr = convertRexToString(condition, scanRel.getRowType());
+      if (statsCache.get(conditionAsStr) == null) {
+        IndexCollection indexes = ((DbGroupScan)scan).getSecondaryIndexCollection(scanRel);
+        populateStats(condition, indexes, scanRel, context);
+        logger.info("index_plan_info: initialize: scanRel #{} and groupScan {} got fulltable {}, statsCache: {}, fiStatsCache: {}",
+            scanRel.getId(), System.identityHashCode(scan), fullTableScanPayload, statsCache, fIStatsCache);
+        return true;
+      }
+    }
     return false;
   }
 
+  /**
+   * This function computes statistics when there is no query condition
+   * @param jTabGrpScan - The current group scan
+   * @param indexes - The collection of indexes to use for getting statistics
+   * @param scanRel - The current scanRel
+   * @param context - The index plan call context
+   */
+  private void populateStatsForNoFilter(JsonTableGroupScan jTabGrpScan, IndexCollection indexes, RelNode scanRel,
+                                   IndexCallContext context) {
+    // Get the stats payload for full table (has total rows in the table)
+    StatisticsPayload ftsPayload = jTabGrpScan.getFirstKeyEstimatedStats(null, null, scanRel);
+    addToCache(null, null, context, ftsPayload, jTabGrpScan, scanRel, scanRel.getRowType());
+    addToCache(null, jTabGrpScan.getAverageRowSizeStats(null), ftsPayload);
+    // Get the stats for all indexes
+    for (IndexDescriptor idx: indexes) {
+      StatisticsPayload idxPayload = jTabGrpScan.getFirstKeyEstimatedStats(null, idx, scanRel);
+      StatisticsPayload idxRowSizePayload = jTabGrpScan.getAverageRowSizeStats(idx);
+      RelDataType newRowType;
+      FunctionalIndexInfo functionInfo = idx.getFunctionalInfo();
+      if (functionInfo.hasFunctional()) {
+        newRowType = FunctionalIndexHelper.rewriteFunctionalRowType(scanRel, context, functionInfo);
+      } else {
+        newRowType = scanRel.getRowType();
+      }
+      addToCache(null, idx, context, idxPayload, jTabGrpScan, scanRel, newRowType);
+      addToCache(idx, idxRowSizePayload, ftsPayload);
+    }
+  }
+
+  /**
+   * This is the core statistics function for populating the statistics. The statistics populated correspond to the query
+   * condition. Based on different types of plans, we would need statistics for different combinations of predicates. Currently,
+   * we do not have a tree-walker for {@link QueryCondition}. Hence, instead of using the individual predicates stats, to construct
+   * the stats for the overall predicates, we rely on using the final predicates. Hence, this has a limitation(susceptible) to
+   * predicate modification post stats generation. Statistics computed/stored are rowcounts, leading rowcounts, average rowsize.
+   * Rowcounts and leading rowcounts (i.e. corresponding to predicates on the leading index columns) are stored in the statsCache.
+   * Average rowsizes are stored in the fiStatsCache (FI stands for Filter Independent).
+   *
+   * @param condition - The condition for which to obtain statistics
+   * @param indexes - The collection of indexes to use for getting statistics
+   * @param scanRel - The current scanRel
+   * @param context - The index plan call context
+   */
+  private void populateStats(RexNode condition, IndexCollection indexes, DrillScanRelBase scanRel,
+                               IndexCallContext context) {
+    JsonTableGroupScan jTabGrpScan;
+    Map<IndexDescriptor, IndexConditionInfo> firstKeyIdxConditionMap;
+    Map<IndexDescriptor, IndexConditionInfo> idxConditionMap;
+    /* Map containing the individual base conditions of an ANDed/ORed condition and their selectivities.
+     * This is used to compute the overall selectivity of a complex ANDed/ORed condition using its base
+     * conditions. Helps prevent over/under estimates and guessed selectivity for ORed predicates.
+     */
+    Map<String, Double> baseConditionMap;
+    GroupScan grpScan = IndexPlanUtils.getGroupScan(scanRel);
+
+    if ((scanRel instanceof DrillScanRel || scanRel instanceof ScanPrel) &&
+        grpScan instanceof JsonTableGroupScan) {
+      jTabGrpScan = (JsonTableGroupScan) grpScan;
+    } else {
+      logger.debug("Statistics: populateStats exit early - not an instance of JsonTableGroupScan!");
+      return;
+    }
+    if (condition == null) {
+      populateStatsForNoFilter(jTabGrpScan, indexes, scanRel, context);
+      statsAvailable = true;
+      return;
+    }
+
+    RexBuilder builder = scanRel.getCluster().getRexBuilder();
+    PlannerSettings settings = PrelUtil.getSettings(scanRel.getCluster());
+    // Get the stats payload for full table (has total rows in the table)
+    StatisticsPayload ftsPayload = jTabGrpScan.getFirstKeyEstimatedStats(null, null, scanRel);
+
+    // Get the average row size for table and all indexes
+    addToCache(null, jTabGrpScan.getAverageRowSizeStats(null), ftsPayload);
+    if (ftsPayload == null || ftsPayload.getRowCount() == 0) {
+      return;
+    }
+    for (IndexDescriptor idx : indexes) {
+      StatisticsPayload idxRowSizePayload = jTabGrpScan.getAverageRowSizeStats(idx);
+      addToCache(idx, idxRowSizePayload, ftsPayload);
+    }
+
+    /* Only use indexes with distinct first key */
+    IndexCollection distFKeyIndexes = distinctFKeyIndexes(indexes, scanRel);
+    IndexConditionInfo.Builder infoBuilder = IndexConditionInfo.newBuilder(condition,
+        distFKeyIndexes, builder, scanRel);
+    idxConditionMap = infoBuilder.getIndexConditionMap();
+    firstKeyIdxConditionMap = infoBuilder.getFirstKeyIndexConditionMap();
+    baseConditionMap = new HashMap<>();
+    for (IndexDescriptor idx : firstKeyIdxConditionMap.keySet()) {
+      if(IndexPlanUtils.conditionIndexed(context.getOrigMarker(), idx) == IndexPlanUtils.ConditionIndexed.NONE) {
+        continue;
+      }
+      RexNode idxCondition = firstKeyIdxConditionMap.get(idx).indexCondition;
+      /* Use the pre-processed condition only for getting actual statistic from MapR-DB APIs. Use the
+       * original condition everywhere else (cache store/lookups) since the RexNode condition and its
+       * corresponding QueryCondition will be used to get statistics. e.g. we convert LIKE into RANGE
+       * condition to get statistics. However, statistics are always asked for LIKE and NOT the RANGE
+       */
+      RexNode preProcIdxCondition = convertToStatsCondition(idxCondition, idx, context, scanRel,
+          Arrays.asList(SqlKind.CAST, SqlKind.LIKE));
+      RelDataType newRowType;
+      FunctionalIndexInfo functionInfo = idx.getFunctionalInfo();
+      if (functionInfo.hasFunctional()) {
+        newRowType = FunctionalIndexHelper.rewriteFunctionalRowType(scanRel, context, functionInfo);
+      } else {
+        newRowType = scanRel.getRowType();
+      }
+
+      QueryCondition queryCondition = jTabGrpScan.convertToQueryCondition(
+          convertToLogicalExpression(preProcIdxCondition, newRowType, settings, builder));
+      // Cap rows/size at total rows in case of issues with DB APIs
+      StatisticsPayload idxPayload = jTabGrpScan.getFirstKeyEstimatedStats(queryCondition, idx, scanRel);
+      double rowCount = Math.min(idxPayload.getRowCount(), ftsPayload.getRowCount());
+      double leadingRowCount = Math.min(idxPayload.getLeadingRowCount(), rowCount);
+      double avgRowSize = Math.min(idxPayload.getAvgRowSize(), ftsPayload.getAvgRowSize());
+      StatisticsPayload payload = new MapRDBStatisticsPayload(rowCount, leadingRowCount, avgRowSize);
+      addToCache(idxCondition, idx, context, payload, jTabGrpScan, scanRel, newRowType);
+      addBaseConditions(idxCondition, payload, false, baseConditionMap, scanRel.getRowType());
+    }
+    /* Add the row count for index conditions on all indexes. Stats are only computed for leading
+     * keys but index conditions can be pushed and would be required for access path costing
+     */
+    for (IndexDescriptor idx : idxConditionMap.keySet()) {
+      if(IndexPlanUtils.conditionIndexed(context.getOrigMarker(), idx) == IndexPlanUtils.ConditionIndexed.NONE) {
+        continue;
+      }
+      Map<LogicalExpression, RexNode> leadingPrefixMap = Maps.newHashMap();
+      double rowCount, leadingRowCount, avgRowSize;
+      RexNode idxCondition = idxConditionMap.get(idx).indexCondition;
+      // Ignore conditions which always evaluate to true
+      if (idxCondition.isAlwaysTrue()) {
+        continue;
+      }
+      RexNode idxIncColCondition = idxConditionMap.get(idx).remainderCondition;
+      RexNode idxRemColCondition = IndexPlanUtils.getLeadingPrefixMap(leadingPrefixMap, idx.getIndexColumns(), infoBuilder, idxCondition);
+      RexNode idxLeadColCondition = IndexPlanUtils.getLeadingColumnsFilter(
+          IndexPlanUtils.getLeadingFilters(leadingPrefixMap, idx.getIndexColumns()), builder);
+      RexNode idxTotRemColCondition = IndexPlanUtils.getTotalRemainderFilter(idxRemColCondition, idxIncColCondition, builder);
+      RexNode idxTotColCondition = IndexPlanUtils.getTotalFilter(idxLeadColCondition, idxTotRemColCondition, builder);
+      FunctionalIndexInfo functionInfo = idx.getFunctionalInfo();
+      RelDataType newRowType = scanRel.getRowType();
+      if (functionInfo.hasFunctional()) {
+        newRowType = FunctionalIndexHelper.rewriteFunctionalRowType(scanRel, context, functionInfo);
+      }
+      /* For non-covering plans we would need the index leading condition */
+      rowCount = ftsPayload.getRowCount() * computeSelectivity(idxLeadColCondition, idx,
+          ftsPayload.getRowCount(), scanRel, baseConditionMap).left;
+      leadingRowCount = rowCount;
+      avgRowSize = fIStatsCache.get(buildUniqueIndexIdentifier(idx)).getAvgRowSize();
+      addToCache(idxLeadColCondition, idx, context, new MapRDBStatisticsPayload(rowCount, leadingRowCount, avgRowSize),
+          jTabGrpScan, scanRel, newRowType);
+      /* For covering plans we would need the full condition */
+      rowCount = ftsPayload.getRowCount() * computeSelectivity(idxTotColCondition, idx,
+          ftsPayload.getRowCount(), scanRel, baseConditionMap).left;
+      addToCache(idxTotColCondition, idx, context, new MapRDBStatisticsPayload(rowCount, leadingRowCount, avgRowSize),
+          jTabGrpScan, scanRel, newRowType);
+      /* For intersect plans we would need the index condition */
+      rowCount = ftsPayload.getRowCount() * computeSelectivity(idxCondition, idx,
+          ftsPayload.getRowCount(), scanRel, baseConditionMap).left;
+      addToCache(idxCondition, idx, context, new MapRDBStatisticsPayload(rowCount, leadingRowCount, avgRowSize),
+          jTabGrpScan, scanRel, newRowType);
+      /* Add the rowCount for condition on only included columns - no leading columns here! */
+      if (idxIncColCondition != null) {
+        rowCount = ftsPayload.getRowCount() * computeSelectivity(idxIncColCondition, null,
+            ftsPayload.getRowCount(), scanRel, baseConditionMap).left;
+        addToCache(idxIncColCondition, idx, context, new MapRDBStatisticsPayload(rowCount, rowCount, avgRowSize),
+            jTabGrpScan, scanRel, newRowType);
+      }
+    }
+
+    // Add the rowCount for the complete condition - based on table
+    double rowCount = ftsPayload.getRowCount() * computeSelectivity(condition, null,
+        ftsPayload.getRowCount(), scanRel, baseConditionMap).left;
+    // Here, ftsLeadingKey rowcount is based on _id predicates
+    StatisticsPayload ftsLeadingKeyPayload = jTabGrpScan.getFirstKeyEstimatedStats(jTabGrpScan.convertToQueryCondition(
+        convertToLogicalExpression(condition, scanRel.getRowType(), settings, builder)), null, scanRel);
+    addToCache(condition, null, null, new MapRDBStatisticsPayload(rowCount, ftsLeadingKeyPayload.getRowCount(),
+        ftsPayload.getAvgRowSize()), jTabGrpScan, scanRel, scanRel.getRowType());
+    // Add the full table rows while we are at it - represented by <NULL> RexNode, <NULL> QueryCondition.
+    // No ftsLeadingKey so leadingKeyRowcount = totalRowCount
+    addToCache(null, null, null, new MapRDBStatisticsPayload(ftsPayload.getRowCount(), ftsPayload.getRowCount(),
+        ftsPayload.getAvgRowSize()), jTabGrpScan, scanRel, scanRel.getRowType());
+    // mark stats has been statsAvailable
+    statsAvailable = true;
+  }
+
+  private boolean addBaseConditions(RexNode condition, StatisticsPayload payload, boolean redundant,
+      Map<String, Double> baseConditionMap, RelDataType rowType) {
+    boolean res = redundant;
+    if (condition.getKind() == SqlKind.AND) {
+      for(RexNode pred : RelOptUtil.conjunctions(condition)) {
+        res = addBaseConditions(pred, payload, res, baseConditionMap, rowType);
+      }
+    } else if (condition.getKind() == SqlKind.OR) {
+      for(RexNode pred : RelOptUtil.disjunctions(condition)) {
+        res = addBaseConditions(pred, payload, res, baseConditionMap, rowType);
+      }
+    } else {
+      // base condition
+      String conditionAsStr = convertRexToString(condition, rowType);
+      if (!redundant) {
+        baseConditionMap.put(conditionAsStr, payload.getRowCount());
+        return true;
+      } else {
+        baseConditionMap.put(conditionAsStr, -1.0);
+        return false;
+      }
+    }
+    return res;
+  }
   /*
- * Convert the given RexNode to a String representation while also replacing the RexInputRef references
- * to actual column names. Since, we compare String representations of RexNodes, two equivalent RexNode
- * expressions may differ in the RexInputRef positions but otherwise the same.
- * e.g. $1 = 'CA' projection (State, Country) , $2 = 'CA' projection (Country, State)
- */
+   * Adds the statistic(row count) to the cache. Also adds the corresponding QueryCondition->RexNode
+   * condition mapping.
+   */
+  private void addToCache(RexNode condition, IndexDescriptor idx, IndexCallContext context,
+      StatisticsPayload payload, JsonTableGroupScan jTabGrpScan, RelNode scanRel, RelDataType rowType) {
+    if (condition != null
+        && !condition.isAlwaysTrue()) {
+      RexBuilder builder = scanRel.getCluster().getRexBuilder();
+      PlannerSettings settings = PrelUtil.getSettings(scanRel.getCluster());
+      String conditionAsStr = convertRexToString(condition, scanRel.getRowType());
+      if (statsCache.get(conditionAsStr) == null
+              && payload.getRowCount() != Statistics.ROWCOUNT_UNKNOWN) {
+        Map<String, StatisticsPayload> payloadMap = new HashMap<>();
+        payloadMap.put(buildUniqueIndexIdentifier(idx), payload);
+        statsCache.put(conditionAsStr, payloadMap);
+        logger.debug("Statistics: StatsCache:<{}, {}>",conditionAsStr, payload);
+        // Always pre-process CAST conditions - Otherwise queryCondition will not be generated correctly
+        RexNode preProcIdxCondition = convertToStatsCondition(condition, idx, context, scanRel,
+            Arrays.asList(SqlKind.CAST));
+        QueryCondition queryCondition =
+            jTabGrpScan.convertToQueryCondition(convertToLogicalExpression(preProcIdxCondition,
+                rowType, settings, builder));
+        if (queryCondition != null) {
+          String queryConditionAsStr = queryCondition.toString();
+          if (conditionRexNodeMap.get(queryConditionAsStr) == null) {
+            conditionRexNodeMap.put(queryConditionAsStr, conditionAsStr);
+            logger.debug("Statistics: QCRNCache:<{}, {}>",queryConditionAsStr, conditionAsStr);
+          }
+        } else {
+          logger.debug("Statistics: QCRNCache: Unable to generate QueryCondition for {}", conditionAsStr);
+          logger.debug("Statistics: QCRNCache: Unable to generate QueryCondition for {}", conditionAsStr);
+        }
+      } else {
+        Map<String, StatisticsPayload> payloadMap = statsCache.get(conditionAsStr);
+        if (payloadMap != null) {
+          if (payloadMap.get(buildUniqueIndexIdentifier(idx)) == null) {
+            payloadMap.put(buildUniqueIndexIdentifier(idx), payload);
+
+            // rowCount for the same condition should be the same on primary table or index,
+            // let us sync them to the smallest since currently both are over-estimated.
+            // DO NOT sync the leading rowCount since it is based on the leading condition and not the
+            // condition (key for this cache). Hence, for the same condition the leading condition and
+            // consequently the leading rowCount will vary with the index. Syncing them may lead to
+            // unintended side-effects e.g. given a covering index and full table scan and a condition
+            // on a non-id field which happens to be the leading key in the index, the leading rowcount
+            // for the full table scan should be the full table rowcount. Syncing them would incorrectly
+            // make the full table scan cheaper! If required, syncing should be only done based on
+            // leading condition and NOT the condition
+            double minimalRowCount = payload.getRowCount();
+            for (StatisticsPayload existing : payloadMap.values()) {
+              if (existing.getRowCount() < minimalRowCount) {
+                minimalRowCount = existing.getRowCount();
+              }
+            }
+            for (StatisticsPayload existing : payloadMap.values()) {
+              if (existing instanceof MapRDBStatisticsPayload) {
+                ((MapRDBStatisticsPayload)existing).rowCount = minimalRowCount;
+              }
+            }
+          } else {
+            logger.debug("Statistics: Filter row count already exists for filter: {}. Skip!", conditionAsStr);
+          }
+        } else {
+          logger.debug("Statistics: Filter row count is UNKNOWN for filter: {}", conditionAsStr);
+        }
+      }
+    } else if (condition == null && idx == null) {
+      fullTableScanPayload = new MapRDBStatisticsPayload(payload.getRowCount(),
+          payload.getLeadingRowCount(), payload.getAvgRowSize());
+      logger.debug("Statistics: StatsCache:<{}, {}>","NULL", fullTableScanPayload);
+    }
+  }
+
+  private void addToCache(IndexDescriptor idx, StatisticsPayload payload, StatisticsPayload ftsPayload) {
+    String tabIdxIdentifier = buildUniqueIndexIdentifier(idx);
+    if (fIStatsCache.get(tabIdxIdentifier) == null) {
+      if (ftsPayload.getAvgRowSize() >= payload.getAvgRowSize()) {
+        fIStatsCache.put(tabIdxIdentifier, payload);
+        logger.debug("Statistics: fIStatsCache:<{}, {}>",tabIdxIdentifier, payload);
+      } else {
+        StatisticsPayload cappedPayload =
+            new MapRDBStatisticsPayload(ROWCOUNT_UNKNOWN, ROWCOUNT_UNKNOWN, ftsPayload.getAvgRowSize());
+        fIStatsCache.put(tabIdxIdentifier,cappedPayload);
+        logger.debug("Statistics: fIStatsCache:<{}, {}> (Capped)",tabIdxIdentifier, cappedPayload);
+      }
+    } else {
+      logger.debug("Statistics: Average row size already exists for :<{}, {}>. Skip!",tabIdxIdentifier, payload);
+    }
+  }
+
+  /*
+   * Convert the given RexNode to a String representation while also replacing the RexInputRef references
+   * to actual column names. Since, we compare String representations of RexNodes, two equivalent RexNode
+   * expressions may differ in the RexInputRef positions but otherwise the same.
+   * e.g. $1 = 'CA' projection (State, Country) , $2 = 'CA' projection (Country, State)
+   */
   private String convertRexToString(RexNode condition, RelDataType rowType) {
     StringBuilder sb = new StringBuilder();
     if (condition == null) {
@@ -320,11 +665,11 @@ public class MapRDBStatistics implements Statistics {
   }
 
   /*
- * Generate the input reference to column mapping for reference replacement. Please
- * look at the usage in convertRexToString() to understand why this mapping is required.
- */
+   * Generate the input reference to column mapping for reference replacement. Please
+   * look at the usage in convertRexToString() to understand why this mapping is required.
+   */
   private void getInputRefMapping(RexNode condition, RelDataType rowType,
-                                  HashMap<String, String> mapping) {
+      HashMap<String, String> mapping) {
     if (condition instanceof RexCall) {
       for (RexNode op : ((RexCall) condition).getOperands()) {
         getInputRefMapping(op, rowType, mapping);
@@ -334,4 +679,328 @@ public class MapRDBStatistics implements Statistics {
           rowType.getFieldNames().get(condition.hashCode()));
     }
   }
+
+  /*
+   * Additional pre-processing may be required for LIKE/CAST predicates in order to compute statistics.
+   * e.g. A LIKE predicate should be converted to a RANGE predicate for statistics computation. MapR-DB
+   * does not yet support computing statistics for LIKE predicates.
+   */
+  private RexNode convertToStatsCondition(RexNode condition, IndexDescriptor index,
+      IndexCallContext context, RelNode scanRel, List<SqlKind>typesToProcess) {
+    RexBuilder builder = scanRel.getCluster().getRexBuilder();
+    if (condition.getKind() == SqlKind.AND) {
+      final List<RexNode> conditions = Lists.newArrayList();
+      for(RexNode pred : RelOptUtil.conjunctions(condition)) {
+        conditions.add(convertToStatsCondition(pred, index, context, scanRel, typesToProcess));
+      }
+      return RexUtil.composeConjunction(builder, conditions, false);
+    } else if (condition.getKind() == SqlKind.OR) {
+      final List<RexNode> conditions = Lists.newArrayList();
+      for(RexNode pred : RelOptUtil.disjunctions(condition)) {
+        conditions.add(convertToStatsCondition(pred, index, context, scanRel, typesToProcess));
+      }
+      return RexUtil.composeDisjunction(builder, conditions, false);
+    } else if (condition instanceof RexCall) {
+      // LIKE operator - convert to a RANGE predicate, if possible
+      if (typesToProcess.contains(SqlKind.LIKE)
+          && ((RexCall) condition).getOperator().getKind() == SqlKind.LIKE) {
+        return convertLikeToRange((RexCall)condition, builder);
+      } else if (typesToProcess.contains(SqlKind.CAST)
+          && hasCastExpression(condition)) {
+        return convertCastForFIdx(((RexCall) condition), index, context, scanRel);
+      }
+      else {
+        return condition;
+      }
+    }
+    return condition;
+  }
+
+  /*
+   * Determines whether the given expression contains a CAST expression. Assumes that the
+   * given expression is a valid expression.
+   * Returns TRUE, if it finds at least one instance of CAST operator.
+   */
+  private boolean hasCastExpression(RexNode condition) {
+    if (condition instanceof RexCall) {
+      if (((RexCall) condition).getOperator().getKind() == SqlKind.CAST) {
+        return true;
+      }
+      for (RexNode op : ((RexCall) condition).getOperands()) {
+        if (hasCastExpression(op)) {
+          return true;
+        }
+      }
+    }
+    return false;
+  }
+  /*
+   * CAST expressions are not understood by MAPR-DB as-is. Hence, we must convert them before passing them
+   * onto MAPR-DB for statistics. Given a functional index, the given expression is converted into an
+   * expression on the `expression` column of the functional index.
+   */
+  private RexNode convertCastForFIdx(RexCall condition, IndexDescriptor index,
+                                     IndexCallContext context, RelNode origScan) {
+    if (index == null) {
+      return condition;
+    }
+    FunctionalIndexInfo functionInfo = index.getFunctionalInfo();
+    if (!functionInfo.hasFunctional()) {
+      return condition;
+    }
+    // The functional index has a different row-type than the original scan. Use the index row-type when
+    // converting the condition
+    RelDataType newRowType = FunctionalIndexHelper.rewriteFunctionalRowType(origScan, context, functionInfo);
+    RexBuilder builder = origScan.getCluster().getRexBuilder();
+    return FunctionalIndexHelper.convertConditionForIndexScan(condition,
+        origScan, newRowType, builder, functionInfo);
+  }
+
+  /*
+   * Helper function to perform additional pre-processing for LIKE predicates
+   */
+  private RexNode convertLikeToRange(RexCall condition, RexBuilder builder) {
+    Preconditions.checkArgument(condition.getOperator().getKind() == SqlKind.LIKE,
+        "Unable to convertLikeToRange: argument is not a LIKE condition!");
+    HBaseRegexParser parser = null;
+    RexNode arg = null;
+    RexLiteral pattern = null, escape = null;
+    String patternStr = null, escapeStr = null;
+    if (condition.getOperands().size() == 2) {
+      // No escape character specified
+      for (RexNode op : condition.getOperands()) {
+        if (op.getKind() == SqlKind.LITERAL) {
+          pattern = (RexLiteral) op;
+        } else {
+          arg = op;
+        }
+      }
+      // Get the PATTERN strings from the corresponding RexLiteral
+      if (pattern.getTypeName() == SqlTypeName.DECIMAL ||
+          pattern.getTypeName() == SqlTypeName.INTEGER) {
+        patternStr = pattern.getValue().toString();
+      } else if (pattern.getTypeName() == SqlTypeName.CHAR) {
+        patternStr = pattern.getValue2().toString();
+      }
+      if (patternStr != null) {
+        parser = new HBaseRegexParser(patternStr);
+      }
+    } else if (condition.getOperands().size() == 3) {
+      // Escape character specified
+      for (RexNode op : condition.getOperands()) {
+        if (op.getKind() == SqlKind.LITERAL) {
+          // Assume first literal specifies PATTERN and the second literal specifies the ESCAPE char
+          if (pattern == null) {
+            pattern = (RexLiteral) op;
+          } else {
+            escape = (RexLiteral) op;
+          }
+        } else {
+          arg = op;
+        }
+      }
+      // Get the PATTERN and ESCAPE strings from the corresponding RexLiteral
+      if (pattern.getTypeName() == SqlTypeName.DECIMAL ||
+          pattern.getTypeName() == SqlTypeName.INTEGER) {
+        patternStr = pattern.getValue().toString();
+      } else if (pattern.getTypeName() == SqlTypeName.CHAR) {
+        patternStr = pattern.getValue2().toString();
+      }
+      if (escape.getTypeName() == SqlTypeName.DECIMAL ||
+          escape.getTypeName() == SqlTypeName.INTEGER) {
+        escapeStr = escape.getValue().toString();
+      } else if (escape.getTypeName() == SqlTypeName.CHAR) {
+        escapeStr = escape.getValue2().toString();
+      }
+      if (patternStr != null && escapeStr != null) {
+        parser = new HBaseRegexParser(patternStr, escapeStr.toCharArray()[0]);
+      }
+    }
+    if (parser != null) {
+      parser.parse();
+      String prefix = parser.getPrefixString();
+      /*
+       * If there is a literal prefix, convert it into an EQUALITY or RANGE predicate
+       */
+      if (prefix != null) {
+        if (prefix.equals(parser.getLikeString())) {
+          // No WILDCARD present. This turns the LIKE predicate to EQUALITY predicate
+          if (arg != null) {
+            return builder.makeCall(SqlStdOperatorTable.EQUALS, arg, pattern);
+          }
+        } else {
+          // WILDCARD present. This turns the LIKE predicate to RANGE predicate
+          byte[] startKey = HConstants.EMPTY_START_ROW;
+          byte[] stopKey = HConstants.EMPTY_END_ROW;
+          startKey = prefix.getBytes(Charsets.UTF_8);
+          stopKey = startKey.clone();
+          boolean isMaxVal = true;
+          for (int i = stopKey.length - 1; i >= 0 ; --i) {
+            int nextByteValue = (0xff & stopKey[i]) + 1;
+            if (nextByteValue < 0xff) {
+              stopKey[i] = (byte) nextByteValue;
+              isMaxVal = false;
+              break;
+            } else {
+              stopKey[i] = 0;
+            }
+          }
+          if (isMaxVal) {
+            stopKey = HConstants.EMPTY_END_ROW;
+          }
+          try {
+            // TODO: This maybe a potential bug since we assume UTF-8 encoding. However, we follow the
+            // current DB implementation. See HBaseFilterBuilder.createHBaseScanSpec "like" CASE statement
+            RexLiteral startKeyLiteral = builder.makeLiteral(new String(startKey,
+                Charsets.UTF_8.toString()));
+            RexLiteral stopKeyLiteral = builder.makeLiteral(new String(stopKey,
+                Charsets.UTF_8.toString()));
+            if (arg != null) {
+              RexNode startPred = builder.makeCall(SqlStdOperatorTable.GREATER_THAN_OR_EQUAL,
+                  arg, startKeyLiteral);
+              RexNode stopPred = builder.makeCall(SqlStdOperatorTable.LESS_THAN, arg, stopKeyLiteral);
+              return builder.makeCall(SqlStdOperatorTable.AND, startPred, stopPred);
+            }
+          } catch (UnsupportedEncodingException ex) {
+            // Encoding not supported - Do nothing!
+            logger.debug("Statistics: convertLikeToRange: Unsupported Encoding Exception -> {}", ex.getMessage());
+          }
+        }
+      }
+    }
+    // Could not convert - return condition as-is.
+    return condition;
+  }
+
+  /*
+   * Compute the selectivity of the given rowCondition. Retrieve the selectivity
+   * for index conditions from the cache
+   */
+  private Pair<Double, Boolean> computeSelectivity(RexNode condition, IndexDescriptor idx, double totalRows,
+      RelNode scanRel, Map<String, Double> baseConditionMap) {
+    double selectivity;
+    boolean guess = false;
+    if (totalRows <= 0) {
+      return new Pair<>(1.0, true);
+    }
+    String conditionAsStr = convertRexToString(condition, scanRel.getRowType());
+    if (condition.getKind() == SqlKind.AND) {
+      selectivity = 1.0;
+      for (RexNode pred : RelOptUtil.conjunctions(condition)) {
+        Pair<Double, Boolean> selPayload = computeSelectivity(pred, idx, totalRows, scanRel, baseConditionMap);
+        if (selPayload.left > 0) {
+          // At least one AND branch is a guess
+          if (selPayload.right == true) {
+            guess = true;
+          }
+          selectivity *= selPayload.left;
+        }
+      }
+    } else if (condition.getKind() == SqlKind.OR) {
+      selectivity = 0.0;
+      for (RexNode pred : RelOptUtil.disjunctions(condition)) {
+        Pair<Double, Boolean> selPayload = computeSelectivity(pred, idx, totalRows, scanRel, baseConditionMap);
+        if (selPayload.left > 0.0) {
+          // At least one OR branch is a guess
+          if (selPayload.right == true) {
+            guess = true;
+          }
+          selectivity += selPayload.left;
+        }
+      }
+      // Cap selectivity of OR'ed predicates at 0.25 if at least one predicate is a guess (Calcite does the same)
+      if (guess && selectivity > 0.25) {
+        selectivity = 0.25;
+      }
+    } else {
+      guess = false;
+      if (baseConditionMap.get(conditionAsStr) != null) {
+        double rowCount = baseConditionMap.get(conditionAsStr);
+        if (rowCount != -1.0) {
+          selectivity = rowCount / totalRows;
+        } else {
+          // Ignore
+          selectivity = -1.0;
+          guess = true;
+        }
+      } else {
+        selectivity = RelMdUtil.guessSelectivity(condition);
+        guess = true;
+      }
+      return new Pair<>(selectivity, guess);
+    }
+    // Cap selectivity to be between 0.0 and 1.0
+    selectivity = Math.min(1.0, selectivity);
+    selectivity = Math.max(0.0, selectivity);
+    logger.debug("Statistics: computeSelectivity: Cache MISS: Computed {} -> {}", conditionAsStr, selectivity);
+    return new Pair<>(selectivity, guess);
+  }
+
+  /*
+   * Filters out indexes from the given collection based on the row key of indexes i.e. after filtering
+   * the given collection would contain only one index for each distinct row key in the collection
+   */
+  private IndexCollection distinctFKeyIndexes(IndexCollection indexes, RelNode scanRel) {
+    IndexCollection distinctIdxCollection = new DrillIndexCollection(scanRel, new HashSet<DrillIndexDescriptor>());
+    Iterator<IndexDescriptor> iterator = indexes.iterator();
+    Map<String, List<IndexDescriptor>> firstColIndexMap = new HashMap<>();
+    while (iterator.hasNext()) {
+      IndexDescriptor index = iterator.next();
+      // If index has columns - the first column is the leading column for the index
+      if (index.getIndexColumns() != null) {
+        List<IndexDescriptor> idxList;
+        String firstCol = convertLExToStr(index.getIndexColumns().get(0));
+        if (firstColIndexMap.get(firstCol) != null) {
+          idxList = firstColIndexMap.get(firstCol);
+        } else {
+          idxList = new ArrayList<>();
+        }
+        idxList.add(index);
+        firstColIndexMap.put(firstCol, idxList);
+      }
+    }
+    for (String firstCol : firstColIndexMap.keySet()) {
+      List<IndexDescriptor> indexesSameFirstCol = firstColIndexMap.get(firstCol);
+      double maxAvgRowSize = -1.0;
+      IndexDescriptor selectedIdx = null;
+      for (IndexDescriptor idx : indexesSameFirstCol) {
+        String tabIdxIdentifier = buildUniqueIndexIdentifier(idx);
+        double idxRowSize = fIStatsCache.get(tabIdxIdentifier).getAvgRowSize();
+        // Prefer index with largest average row-size, breaking ties lexicographically
+        if (idxRowSize > maxAvgRowSize
+            || (idxRowSize == maxAvgRowSize
+                && (selectedIdx == null || idx.getIndexName().compareTo(selectedIdx.getIndexName()) < 0))) {
+          maxAvgRowSize = idxRowSize;
+          selectedIdx = idx;
+        }
+      }
+      assert (selectedIdx != null);
+      distinctIdxCollection.addIndex(selectedIdx);
+    }
+    return distinctIdxCollection;
+  }
+
+  /*
+   * Returns the String representation for the given Logical Expression
+   */
+  private String convertLExToStr(LogicalExpression lex) {
+    StringBuilder sb = new StringBuilder();
+    ExpressionStringBuilder esb = new ExpressionStringBuilder();
+    lex.accept(esb, sb);
+    return sb.toString();
+  }
+
+  /*
+   * Converts the given RexNode condition into a Drill logical expression.
+   */
+  private LogicalExpression convertToLogicalExpression(RexNode condition,
+      RelDataType type, PlannerSettings settings, RexBuilder builder) {
+    LogicalExpression conditionExp;
+    try {
+      conditionExp = DrillOptiq.toDrill(new DrillParseContext(settings), type, builder, condition);
+    } catch (ClassCastException e) {
+      return null;
+    }
+    return conditionExp;
+  }
 }
diff --git a/contrib/format-maprdb/src/main/java/org/apache/drill/exec/store/mapr/db/MapRDBFormatMatcher.java b/contrib/format-maprdb/src/main/java/org/apache/drill/exec/store/mapr/db/MapRDBFormatMatcher.java
index ee35a68..f982278 100644
--- a/contrib/format-maprdb/src/main/java/org/apache/drill/exec/store/mapr/db/MapRDBFormatMatcher.java
+++ b/contrib/format-maprdb/src/main/java/org/apache/drill/exec/store/mapr/db/MapRDBFormatMatcher.java
@@ -20,7 +20,10 @@ package org.apache.drill.exec.store.mapr.db;
 import java.io.IOException;
 
 import com.mapr.fs.MapRFileStatus;
+import com.mapr.db.index.IndexDesc;
 import com.mapr.fs.tables.TableProperties;
+import org.apache.drill.exec.planner.index.IndexDescriptor;
+import org.apache.drill.exec.planner.index.MapRDBIndexDescriptor;
 import org.apache.drill.exec.planner.logical.DrillTable;
 import org.apache.drill.exec.planner.logical.DynamicDrillTable;
 import org.apache.drill.exec.store.SchemaConfig;
@@ -33,6 +36,7 @@ import org.apache.drill.exec.store.mapr.TableFormatMatcher;
 import org.apache.drill.exec.store.mapr.TableFormatPlugin;
 
 import org.apache.drill.exec.store.mapr.db.binary.MapRDBBinaryTable;
+import org.apache.hadoop.fs.FileStatus;
 import org.apache.hadoop.fs.Path;
 
 public class MapRDBFormatMatcher extends TableFormatMatcher {
@@ -49,6 +53,44 @@ public class MapRDBFormatMatcher extends TableFormatMatcher {
         .getIsMarlinTable();
   }
 
+
+  /**
+   * Get an instance of DrillTable for a particular native secondary index
+   * @param fs
+   * @param selection
+   * @param fsPlugin
+   * @param storageEngineName
+   * @param userName
+   * @param secondaryIndexDesc
+   * @return
+   * @throws IOException
+   */
+  public DrillTable isReadableIndex(DrillFileSystem fs,
+                                    FileSelection selection, FileSystemPlugin fsPlugin,
+                                    String storageEngineName, String userName,
+                                    IndexDescriptor secondaryIndexDesc) throws IOException {
+    FileStatus status = selection.getFirstPath(fs);
+
+    if (!isFileReadable(fs, status)) {
+      return null;
+    }
+
+    MapRDBFormatPlugin fp = (MapRDBFormatPlugin) getFormatPlugin();
+    DrillTable dt = new DynamicDrillTable(fsPlugin,
+        storageEngineName,
+        userName,
+        new FormatSelection(fp.getConfig(),
+            selection));
+
+    // TODO:  Create groupScan using index descriptor
+    dt.setGroupScan(fp.getGroupScan(userName,
+        selection,
+        null /* columns */,
+        (IndexDesc) ((MapRDBIndexDescriptor) secondaryIndexDesc).getOriginalDesc()));
+
+    return dt;
+  }
+
   @Override
   public DrillTable isReadable(DrillFileSystem fs,
                                FileSelection selection, FileSystemPlugin fsPlugin,
diff --git a/contrib/format-maprdb/src/main/java/org/apache/drill/exec/store/mapr/db/MapRDBFormatPlugin.java b/contrib/format-maprdb/src/main/java/org/apache/drill/exec/store/mapr/db/MapRDBFormatPlugin.java
index da4829f..0d1bf04 100644
--- a/contrib/format-maprdb/src/main/java/org/apache/drill/exec/store/mapr/db/MapRDBFormatPlugin.java
+++ b/contrib/format-maprdb/src/main/java/org/apache/drill/exec/store/mapr/db/MapRDBFormatPlugin.java
@@ -108,7 +108,7 @@ public class MapRDBFormatPlugin extends TableFormatPlugin {
   public Set<StoragePluginOptimizerRule> getOptimizerRules() {
     return ImmutableSet.of(MapRDBPushFilterIntoScan.FILTER_ON_SCAN, MapRDBPushFilterIntoScan.FILTER_ON_PROJECT,
         MapRDBPushProjectIntoScan.PROJECT_ON_SCAN, MapRDBPushLimitIntoScan.LIMIT_ON_PROJECT,
-        MapRDBPushLimitIntoScan.LIMIT_ON_SCAN);
+        MapRDBPushLimitIntoScan.LIMIT_ON_SCAN, MapRDBPushLimitIntoScan.LIMIT_ON_RKJOIN);
   }
 
 
diff --git a/contrib/format-maprdb/src/main/java/org/apache/drill/exec/store/mapr/db/MapRDBGroupScan.java b/contrib/format-maprdb/src/main/java/org/apache/drill/exec/store/mapr/db/MapRDBGroupScan.java
index 1e6bcec..422a269 100644
--- a/contrib/format-maprdb/src/main/java/org/apache/drill/exec/store/mapr/db/MapRDBGroupScan.java
+++ b/contrib/format-maprdb/src/main/java/org/apache/drill/exec/store/mapr/db/MapRDBGroupScan.java
@@ -39,6 +39,9 @@ import org.apache.calcite.rel.RelNode;
 import org.apache.drill.exec.planner.index.IndexCollection;
 
 import org.apache.drill.exec.planner.cost.PluginCost;
+import org.apache.drill.exec.planner.index.IndexDiscover;
+import org.apache.drill.exec.planner.index.IndexDiscoverFactory;
+import org.apache.drill.exec.planner.index.MapRDBIndexDiscover;
 import org.apache.drill.exec.proto.CoordinationProtos.DrillbitEndpoint;
 import org.apache.drill.exec.store.AbstractStoragePlugin;
 
@@ -307,8 +310,13 @@ public abstract class MapRDBGroupScan extends AbstractDbGroupScan {
 
   @Override
   public IndexCollection getSecondaryIndexCollection(RelNode scanRel) {
-    //XXX to implement for complete secondary index framework
-    return null;
+    IndexDiscover discover = IndexDiscoverFactory.getIndexDiscover(
+        getStorageConfig(), this, scanRel, MapRDBIndexDiscover.class);
+
+    if (discover == null) {
+      logger.error("Null IndexDiscover was found for {}!", scanRel);
+    }
+    return discover.getTableIndex(getTableName());
   }
 
   @JsonIgnore
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 1f4b8c9..a26bc80 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
@@ -26,11 +26,13 @@ import org.apache.drill.exec.planner.common.DrillRelOptUtil;
 import org.apache.drill.exec.planner.logical.RelOptHelper;
 import org.apache.drill.exec.planner.physical.LimitPrel;
 import org.apache.drill.exec.planner.physical.ProjectPrel;
+import org.apache.drill.exec.planner.physical.RowKeyJoinPrel;
 import org.apache.drill.exec.planner.physical.ScanPrel;
 import org.apache.drill.exec.store.StoragePluginOptimizerRule;
 import org.apache.drill.exec.store.hbase.HBaseScanSpec;
 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.store.mapr.db.json.RestrictedJsonTableGroupScan;
 
 public abstract class MapRDBPushLimitIntoScan extends StoragePluginOptimizerRule {
   static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(MapRDBPushLimitIntoScan.class);
@@ -59,8 +61,8 @@ public abstract class MapRDBPushLimitIntoScan extends StoragePluginOptimizerRule
       if (scan.getGroupScan().supportsLimitPushdown()
             && !limit.isPushDown() && limit.getFetch() != null) {
         if ((scan.getGroupScan() instanceof JsonTableGroupScan
-              && ((JsonTableGroupScan) scan.getGroupScan()).isIndexScan()) ) {
-            //|| (scan.getGroupScan() instanceof RestrictedJsonTableGroupScan)) {
+              && ((JsonTableGroupScan) scan.getGroupScan()).isIndexScan())
+            || (scan.getGroupScan() instanceof RestrictedJsonTableGroupScan)) {
           return true;
         }
       }
@@ -111,6 +113,26 @@ public abstract class MapRDBPushLimitIntoScan extends StoragePluginOptimizerRule
     }
   };
 
+  public static final StoragePluginOptimizerRule LIMIT_ON_RKJOIN =
+      new MapRDBPushLimitIntoScan(RelOptHelper.some(LimitPrel.class, RelOptHelper.any(RowKeyJoinPrel.class)),
+          "MapRDBPushLimitIntoScan:Limit_On_RKJoin") {
+
+    @Override
+    public void onMatch(RelOptRuleCall call) {
+      final RowKeyJoinPrel join = call.rel(1);
+      final LimitPrel limit = call.rel(0);
+      doPushLimitIntoRowKeyJoin(call, limit, null, join);
+    }
+
+    @Override
+    public boolean matches(RelOptRuleCall call) {
+      final LimitPrel limit = call.rel(0);
+      // We do not fire this rule if fetch() is null (indicating we have to fetch all the
+      // remaining rows starting from offset.
+      return !limit.isPushDown() && limit.getFetch() != null;
+    }
+  };
+
   protected void doPushLimitIntoGroupScan(RelOptRuleCall call,
       LimitPrel limit, final ProjectPrel project, ScanPrel scan, GroupScan groupScan) {
     try {
@@ -153,5 +175,29 @@ public abstract class MapRDBPushLimitIntoScan extends StoragePluginOptimizerRule
     }
     return null;
   }
+
+  protected void doPushLimitIntoRowKeyJoin(RelOptRuleCall call,
+    LimitPrel limit, final ProjectPrel project, RowKeyJoinPrel join) {
+    final RelNode newChild;
+    try {
+      RelNode left = join.getLeft();
+      RelNode right = join.getRight();
+      final RelNode limitOnLeft = new LimitPrel(left.getCluster(), left.getTraitSet(), left,
+          limit.getOffset(), limit.getFetch());
+      RowKeyJoinPrel newJoin = new RowKeyJoinPrel(join.getCluster(), join.getTraitSet(), limitOnLeft, right,
+          join.getCondition(), join.getJoinType());
+      if (project != null) {
+        final ProjectPrel newProject = new ProjectPrel(project.getCluster(), project.getTraitSet(), newJoin,
+            project.getProjects(), project.getRowType());
+        newChild = newProject;
+      } else {
+        newChild = newJoin;
+      }
+      call.transformTo(newChild);
+      logger.debug("pushLimitIntoRowKeyJoin: Pushed limit on left side of Join " + join.toString());
+    } catch (Exception e) {
+      logger.warn("pushLimitIntoRowKeyJoin: Exception while trying limit pushdown!", e);
+    }
+  }
 }
 
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 2eb84e7..5215868 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
@@ -30,8 +30,9 @@ 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.common.DrillRelOptUtil;
+import org.apache.drill.exec.planner.common.DrillRelOptUtil.ProjectPushInfo;
 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.ScanPrel;
 import org.apache.drill.exec.store.StoragePluginOptimizerRule;
diff --git a/contrib/format-maprdb/src/main/java/org/apache/drill/exec/store/mapr/db/json/JsonTableGroupScan.java b/contrib/format-maprdb/src/main/java/org/apache/drill/exec/store/mapr/db/json/JsonTableGroupScan.java
index a269256..b545262 100644
--- a/contrib/format-maprdb/src/main/java/org/apache/drill/exec/store/mapr/db/json/JsonTableGroupScan.java
+++ b/contrib/format-maprdb/src/main/java/org/apache/drill/exec/store/mapr/db/json/JsonTableGroupScan.java
@@ -44,10 +44,14 @@ import org.apache.drill.exec.physical.base.PhysicalOperator;
 import org.apache.drill.exec.physical.base.ScanStats;
 import org.apache.drill.exec.physical.base.ScanStats.GroupScanProperty;
 import org.apache.drill.exec.store.AbstractStoragePlugin;
+import org.apache.drill.exec.planner.index.IndexDescriptor;
+import org.apache.drill.exec.planner.index.MapRDBIndexDescriptor;
+import org.apache.drill.exec.planner.index.MapRDBStatisticsPayload;
 import org.apache.drill.exec.planner.index.Statistics;
 import org.apache.drill.exec.planner.index.MapRDBStatistics;
 import org.apache.drill.exec.planner.cost.PluginCost;
 import org.apache.drill.exec.planner.physical.PartitionFunction;
+import org.apache.drill.exec.planner.physical.PrelUtil;
 import org.apache.drill.exec.store.StoragePluginRegistry;
 import org.apache.drill.exec.store.dfs.FileSystemConfig;
 import org.apache.drill.exec.store.dfs.FileSystemPlugin;
@@ -296,6 +300,9 @@ public class JsonTableGroupScan extends MapRDBGroupScan implements IndexGroupSca
 
   @Override
   public ScanStats getScanStats() {
+    if (isIndexScan()) {
+      return indexScanStats();
+    }
     return fullTableScanStats();
   }
 
@@ -359,6 +366,57 @@ public class JsonTableGroupScan extends MapRDBGroupScan implements IndexGroupSca
     }
   }
 
+  private ScanStats indexScanStats() {
+    if (!this.getIndexHint().equals("") &&
+        this.getIndexHint().equals(getIndexDesc().getIndexName())) {
+      logger.debug("JsonIndexGroupScan:{} forcing index {} by making tiny cost", this, this.getIndexHint());
+      return new ScanStats(GroupScanProperty.NO_EXACT_ROW_COUNT, 1,1, 0);
+    }
+
+    int totalColNum = STAR_COLS;
+    PluginCost pluginCostModel = formatPlugin.getPluginCostModel();
+    final int avgColumnSize = pluginCostModel.getAverageColumnSize(this);
+    boolean filterPushed = (scanSpec.getSerializedFilter() != null);
+    if(scanSpec != null && scanSpec.getIndexDesc() != null) {
+      totalColNum = scanSpec.getIndexDesc().getIncludedFields().size()
+          + scanSpec.getIndexDesc().getIndexedFields().size() + 1;
+    }
+    int numColumns = (columns == null || columns.isEmpty()) ?  totalColNum: columns.size();
+    String idxIdentifier = stats.buildUniqueIndexIdentifier(scanSpec.getIndexDesc().getPrimaryTablePath(),
+        scanSpec.getIndexDesc().getIndexName());
+    double rowCount = stats.getRowCount(scanSpec.getCondition(), idxIdentifier);
+    // rowcount based on index leading columns predicate.
+    double leadingRowCount = stats.getLeadingRowCount(scanSpec.getCondition(), idxIdentifier);
+    double avgRowSize = stats.getAvgRowSize(idxIdentifier, false);
+    // If UNKNOWN, use defaults
+    if (rowCount == ROWCOUNT_UNKNOWN || rowCount == 0) {
+      rowCount = (filterPushed ? 0.0005f : 0.001f) * fullTableRowCount / scanSpec.getIndexDesc().getIndexedFields().size();
+    }
+    // If limit pushdown has occurred - factor it in the rowcount
+    if (this.maxRecordsToRead > 0) {
+      rowCount = Math.min(rowCount, this.maxRecordsToRead);
+    }
+    if (leadingRowCount == ROWCOUNT_UNKNOWN || leadingRowCount == 0) {
+      leadingRowCount = rowCount;
+    }
+    if (avgRowSize == AVG_ROWSIZE_UNKNOWN || avgRowSize == 0) {
+      avgRowSize = avgColumnSize * numColumns;
+    }
+    double rowsFromDisk = leadingRowCount;
+    if (!filterPushed) {
+      // both start and stop rows are empty, indicating this is a full scan so
+      // use the total rows for calculating disk i/o
+      rowsFromDisk = fullTableRowCount;
+    }
+    double totalBlocks = Math.ceil((avgRowSize * fullTableRowCount)/pluginCostModel.getBlockSize(this));
+    double numBlocks = Math.ceil(((avgRowSize * rowsFromDisk)/pluginCostModel.getBlockSize(this)));
+    numBlocks = Math.min(totalBlocks, numBlocks);
+    double diskCost = numBlocks * pluginCostModel.getSequentialBlockReadCost(this);
+    logger.debug("index_plan_info: JsonIndexGroupScan:{} - indexName:{}: rowCount:{}, avgRowSize:{}, blocks:{}, totalBlocks:{}, rowsFromDisk {}, diskCost:{}",
+        System.identityHashCode(this), scanSpec.getIndexDesc().getIndexName(), rowCount, avgRowSize, numBlocks, totalBlocks, rowsFromDisk, diskCost);
+    return new ScanStats(GroupScanProperty.NO_EXACT_ROW_COUNT, rowCount, 1, diskCost);
+  }
+
   @Override
   @JsonIgnore
   public PhysicalOperator getNewWithChildren(List<PhysicalOperator> children) {
@@ -412,6 +470,142 @@ public class JsonTableGroupScan extends MapRDBGroupScan implements IndexGroupSca
     return true;
   }
 
+
+  @Override
+  public RestrictedJsonTableGroupScan getRestrictedScan(List<SchemaPath> columns) {
+    RestrictedJsonTableGroupScan newScan =
+        new RestrictedJsonTableGroupScan(this.getUserName(),
+            (FileSystemPlugin) this.getStoragePlugin(),
+            this.getFormatPlugin(),
+            this.getScanSpec(),
+            this.getColumns(),
+            this.getStatistics());
+    newScan.columns = columns;
+    return newScan;
+  }
+
+  /**
+   * Get the estimated average rowsize. DO NOT call this API directly.
+   * Call the stats API instead which modifies the counts based on preference options.
+   * @param index, to use for generating the estimate
+   * @return row count post filtering
+   */
+  public MapRDBStatisticsPayload getAverageRowSizeStats(IndexDescriptor index) {
+    IndexDesc indexDesc = null;
+    double avgRowSize = AVG_ROWSIZE_UNKNOWN;
+
+    if (index != null) {
+      indexDesc = (IndexDesc)((MapRDBIndexDescriptor)index).getOriginalDesc();
+    }
+    // If no index is specified, get it from the primary table
+    if(indexDesc == null && scanSpec.isSecondaryIndex()) {
+      throw new UnsupportedOperationException("getAverageRowSizeStats should be invoked on primary table");
+    }
+
+    // Get the index table or primary table and use the DB API to get the estimated number of rows. For size estimates,
+    // we assume that all the columns would be read from the disk.
+    final Table table = this.formatPlugin.getJsonTableCache().getTable(scanSpec.getTableName(), indexDesc, getUserName());
+
+    if (table != null) {
+      final MetaTable metaTable = table.getMetaTable();
+      if (metaTable != null) {
+        avgRowSize = metaTable.getAverageRowSize();
+      }
+    }
+    logger.debug("index_plan_info: getEstimatedRowCount obtained from DB Client for {}: indexName: {}, indexInfo: {}, " +
+            "avgRowSize: {}, estimatedSize {}", this, (indexDesc == null ? "null" : indexDesc.getIndexName()),
+        (indexDesc == null ? "null" : indexDesc.getIndexInfo()), avgRowSize);
+    return new MapRDBStatisticsPayload(ROWCOUNT_UNKNOWN, ROWCOUNT_UNKNOWN, avgRowSize);
+  }
+
+  /**
+   * Get the estimated statistics after applying the {@link RexNode} condition. DO NOT call this API directly.
+   * Call the stats API instead which modifies the counts based on preference options.
+   * @param condition, filter to apply
+   * @param index, to use for generating the estimate
+   * @return row count post filtering
+   */
+  public MapRDBStatisticsPayload getFirstKeyEstimatedStats(QueryCondition condition, IndexDescriptor index, RelNode scanRel) {
+    IndexDesc indexDesc = null;
+    if (index != null) {
+      indexDesc = (IndexDesc)((MapRDBIndexDescriptor)index).getOriginalDesc();
+    }
+    return getFirstKeyEstimatedStatsInternal(condition, indexDesc, scanRel);
+  }
+
+  /**
+   * Get the estimated statistics after applying the {@link QueryCondition} condition
+   * @param condition, filter to apply
+   * @param index, to use for generating the estimate
+   * @return {@link MapRDBStatisticsPayload} statistics
+   */
+  private MapRDBStatisticsPayload getFirstKeyEstimatedStatsInternal(QueryCondition condition, IndexDesc index, RelNode scanRel) {
+    // double totalRows = getRowCount(null, scanPrel);
+
+    // If no index is specified, get it from the primary table
+    if(index == null && scanSpec.isSecondaryIndex()) {
+      // If stats not cached get it from the table.
+      //table = MapRDB.getTable(scanSpec.getPrimaryTablePath());
+      throw new UnsupportedOperationException("getFirstKeyEstimatedStats should be invoked on primary table");
+    }
+
+    // Get the index table or primary table and use the DB API to get the estimated number of rows. For size estimates,
+    // we assume that all the columns would be read from the disk.
+    final Table table = this.formatPlugin.getJsonTableCache().getTable(scanSpec.getTableName(), index, getUserName());
+
+    if (table != null) {
+      // Factor reflecting confidence in the DB estimates. If a table has few tablets, the tablet-level stats
+      // might be off. The decay scalingFactor will reduce estimates when one tablet represents a significant percentage
+      // of the entire table.
+      double scalingFactor = 1.0;
+      boolean isFullScan = false;
+      final MetaTable metaTable = table.getMetaTable();
+      com.mapr.db.scan.ScanStats stats = (condition == null)
+          ? metaTable.getScanStats() : metaTable.getScanStats(condition);
+      if (index == null && condition != null) {
+        // Given table condition might not be on leading column. Check if the rowcount matches full table rows.
+        // In that case no leading key present or does not prune enough. Treat it like so.
+        com.mapr.db.scan.ScanStats noConditionPTabStats = metaTable.getScanStats();
+        if (stats.getEstimatedNumRows() == noConditionPTabStats.getEstimatedNumRows()) {
+          isFullScan = true;
+        }
+      }
+      // Use the scalingFactor only when a condition filters out rows from the table. If no condition is present, all rows
+      // should be selected. So the scalingFactor should not reduce the returned rows
+      if (condition != null && !isFullScan) {
+        double forcedScalingFactor = PrelUtil.getSettings(scanRel.getCluster()).getIndexStatsRowCountScalingFactor();
+        // Accuracy is defined as 1 - Error where Error = # Boundary Tablets (2) / # Total Matching Tablets.
+        // For 2 or less matching tablets, the error is assumed to be 50%. The Sqrt gives the decaying scalingFactor
+        if (stats.getTabletCount() > 2) {
+          double accuracy = 1.0 - (2.0/stats.getTabletCount());
+          scalingFactor = Math.min(1.0, 1.0 / Math.sqrt(1.0 / accuracy));
+        } else {
+          scalingFactor = 0.5;
+        }
+        if (forcedScalingFactor < 1.0
+            && metaTable.getScanStats().getTabletCount() < PluginConstants.JSON_TABLE_NUM_TABLETS_PER_INDEX_DEFAULT) {
+          // User forced confidence scalingFactor for small tables (assumed as less than 32 tablets (~512 MB))
+          scalingFactor = forcedScalingFactor;
+        }
+      }
+      logger.info("index_plan_info: getEstimatedRowCount obtained from DB Client for {}: indexName: {}, indexInfo: {}, " +
+              "condition: {} rowCount: {}, avgRowSize: {}, estimatedSize {}, tabletCount {}, totalTabletCount {}, " +
+              "scalingFactor {}",
+          this, (index == null ? "null" : index.getIndexName()), (index == null ? "null" : index.getIndexInfo()),
+          (condition == null ? "null" : condition.toString()), stats.getEstimatedNumRows(),
+          (stats.getEstimatedNumRows() == 0 ? 0 : stats.getEstimatedSize()/stats.getEstimatedNumRows()),
+          stats.getEstimatedSize(), stats.getTabletCount(), metaTable.getScanStats().getTabletCount(), scalingFactor);
+      return new MapRDBStatisticsPayload(scalingFactor * stats.getEstimatedNumRows(), scalingFactor * stats.getEstimatedNumRows(),
+          ((stats.getEstimatedNumRows() == 0 ? 0 : (double)stats.getEstimatedSize()/stats.getEstimatedNumRows())));
+    } else {
+      logger.info("index_plan_info: getEstimatedRowCount: {} indexName: {}, indexInfo: {}, " +
+              "condition: {} rowCount: UNKNOWN, avgRowSize: UNKNOWN", this, (index == null ? "null" : index.getIndexName()),
+          (index == null ? "null" : index.getIndexInfo()), (condition == null ? "null" : condition.toString()));
+      return new MapRDBStatisticsPayload(ROWCOUNT_UNKNOWN, ROWCOUNT_UNKNOWN, AVG_ROWSIZE_UNKNOWN);
+    }
+  }
+
+
   /**
    * Set the row count resulting from applying the {@link RexNode} condition. Forced row counts will take
    * precedence over stats row counts
@@ -518,9 +712,7 @@ public class JsonTableGroupScan extends MapRDBGroupScan implements IndexGroupSca
   @Override
   @JsonIgnore
   public PartitionFunction getRangePartitionFunction(List<FieldReference> refList) {
-
-    return null;
-    //new JsonTableRangePartitionFunction(refList, scanSpec.getTableName(), this.getUserName(), this.getFormatPlugin());
+    return new JsonTableRangePartitionFunction(refList, scanSpec.getTableName(), this.getUserName(), this.getFormatPlugin());
   }
 
   /**
diff --git a/contrib/format-maprdb/src/main/java/org/apache/drill/exec/store/mapr/db/json/JsonTableRangePartitionFunction.java b/contrib/format-maprdb/src/main/java/org/apache/drill/exec/store/mapr/db/json/JsonTableRangePartitionFunction.java
new file mode 100644
index 0000000..acaa6ca
--- /dev/null
+++ b/contrib/format-maprdb/src/main/java/org/apache/drill/exec/store/mapr/db/json/JsonTableRangePartitionFunction.java
@@ -0,0 +1,237 @@
+/*
+ * 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.store.mapr.db.json;
+
+import java.util.List;
+
+import org.apache.drill.common.expression.FieldReference;
+import org.apache.drill.exec.planner.physical.AbstractRangePartitionFunction;
+import org.apache.drill.exec.record.VectorWrapper;
+import org.apache.drill.exec.store.mapr.db.MapRDBFormatPlugin;
+import org.apache.drill.exec.vector.ValueVector;
+import org.ojai.store.QueryCondition;
+
+import com.fasterxml.jackson.annotation.JsonCreator;
+import com.fasterxml.jackson.annotation.JsonIgnore;
+import com.fasterxml.jackson.annotation.JsonProperty;
+import com.fasterxml.jackson.annotation.JsonTypeName;
+import com.google.common.base.Preconditions;
+import com.google.common.collect.Lists;
+import com.mapr.db.Table;
+import com.mapr.db.impl.ConditionImpl;
+import com.mapr.db.impl.IdCodec;
+import com.mapr.db.impl.ConditionNode.RowkeyRange;
+import com.mapr.db.scan.ScanRange;
+import com.mapr.fs.jni.MapRConstants;
+import com.mapr.org.apache.hadoop.hbase.util.Bytes;
+
+@JsonTypeName("jsontable-range-partition-function")
+public class JsonTableRangePartitionFunction extends AbstractRangePartitionFunction {
+
+  static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(JsonTableRangePartitionFunction.class);
+
+  @JsonProperty("refList")
+  protected List<FieldReference> refList;
+
+  @JsonProperty("tableName")
+  protected String tableName;
+
+  @JsonIgnore
+  protected String userName;
+
+  @JsonIgnore
+  protected ValueVector partitionKeyVector = null;
+
+  // List of start keys of the scan ranges for the table.
+  @JsonProperty
+  protected List<byte[]> startKeys = null;
+
+  // List of stop keys of the scan ranges for the table.
+  @JsonProperty
+  protected List<byte[]> stopKeys = null;
+
+  @JsonCreator
+  public JsonTableRangePartitionFunction(
+      @JsonProperty("refList") List<FieldReference> refList,
+      @JsonProperty("tableName") String tableName,
+      @JsonProperty("startKeys") List<byte[]> startKeys,
+      @JsonProperty("stopKeys") List<byte[]> stopKeys) {
+    this.refList = refList;
+    this.tableName = tableName;
+    this.startKeys = startKeys;
+    this.stopKeys = stopKeys;
+  }
+
+  public JsonTableRangePartitionFunction(List<FieldReference> refList,
+      String tableName, String userName, MapRDBFormatPlugin formatPlugin) {
+    this.refList = refList;
+    this.tableName = tableName;
+    this.userName = userName;
+    initialize(formatPlugin);
+  }
+
+  @JsonProperty("refList")
+  @Override
+  public List<FieldReference> getPartitionRefList() {
+    return refList;
+  }
+
+  @Override
+  public void setup(List<VectorWrapper<?>> partitionKeys) {
+    if (partitionKeys.size() != 1) {
+      throw new UnsupportedOperationException(
+          "Range partitioning function supports exactly one partition column; encountered " + partitionKeys.size());
+    }
+
+    VectorWrapper<?> v = partitionKeys.get(0);
+
+    partitionKeyVector = v.getValueVector();
+
+    Preconditions.checkArgument(partitionKeyVector != null, "Found null partitionKeVector.") ;
+  }
+
+  @Override
+  public boolean equals(Object obj) {
+    if (this == obj) {
+      return true;
+    }
+    if (obj instanceof JsonTableRangePartitionFunction) {
+      JsonTableRangePartitionFunction rpf = (JsonTableRangePartitionFunction) obj;
+      List<FieldReference> thisPartRefList = this.getPartitionRefList();
+      List<FieldReference> otherPartRefList = rpf.getPartitionRefList();
+      if (thisPartRefList.size() != otherPartRefList.size()) {
+        return false;
+      }
+      for (int refIdx=0; refIdx<thisPartRefList.size(); refIdx++) {
+        if (!thisPartRefList.get(refIdx).equals(otherPartRefList.get(refIdx))) {
+          return false;
+        }
+      }
+      return true;
+    }
+    return false;
+  }
+
+  @Override
+  public int eval(int index, int numPartitions) {
+
+	  String key = partitionKeyVector.getAccessor().getObject(index).toString();
+	  byte[] encodedKey = IdCodec.encodeAsBytes(key);
+
+    int tabletId = -1;
+
+    // Do a 'range' binary search through the list of start-stop keys to find nearest range.  Assumption is
+    // that the list of start keys is ordered (this is ensured by MetaTable.getScanRanges())
+    // TODO: This search should ideally be delegated to MapR-DB once an appropriate API is available
+    // to optimize this
+    int low = 0, high = startKeys.size() - 1;
+    while (low <= high) {
+      int mid = low + (high-low)/2;
+
+      byte[] start = startKeys.get(mid);
+      byte[] stop  = stopKeys.get(mid);
+
+      // Check if key is present in the mid interval of [start, stop].
+      // Account for empty byte array start/stop
+      if ( (Bytes.compareTo(encodedKey, start) >= 0 ||
+             Bytes.equals(start, MapRConstants.EMPTY_BYTE_ARRAY)
+           ) &&
+           (Bytes.compareTo(encodedKey, stop) < 0 ||
+             Bytes.equals(stop, MapRConstants.EMPTY_BYTE_ARRAY)
+           )
+         ) {
+        tabletId = mid;
+        break;
+      }
+
+      if (Bytes.compareTo(encodedKey, start) >= 0) {
+        // key is greater, ignore left side
+        low = mid + 1;
+      } else {
+        // key is smaller, ignore right side
+        high = mid - 1;
+      }
+    }
+
+    if (tabletId < 0) {
+      tabletId = 0;
+      logger.warn("Key {} was not found in any of the start-stop ranges. Using default tabletId {}", key, tabletId);
+    }
+
+    int partitionId = tabletId % numPartitions;
+
+    logger.trace("Key = {}, tablet id = {}, partition id = {}", key, tabletId, partitionId);
+
+    return partitionId;
+  }
+
+
+  public void initialize(MapRDBFormatPlugin plugin) {
+
+    // get the table handle from the table cache
+    Table table = plugin.getJsonTableCache().getTable(tableName, userName);
+
+    // Set the condition to null such that all scan ranges are retrieved for the primary table.
+    // The reason is the row keys could typically belong to any one of the tablets of the table, so
+    // there is no use trying to get only limited set of scan ranges.
+    // NOTE: here we use the restrictedScanRangeSizeMB because the range partitioning should be parallelized
+    // based on the number of scan ranges on the RestrictedJsonTableGroupScan.
+    List<ScanRange> ranges = table.getMetaTable().getScanRanges(plugin.getRestrictedScanRangeSizeMB());
+
+    this.startKeys = Lists.newArrayList();
+    this.stopKeys = Lists.newArrayList();
+
+    logger.debug("Num scan ranges for table {} = {}", table.getName(), ranges.size());
+
+    int count = 0;
+    for (ScanRange r : ranges) {
+      QueryCondition condition = r.getCondition();
+      List<RowkeyRange> rowkeyRanges =  ((ConditionImpl)condition).getRowkeyRanges();
+      byte[] start = rowkeyRanges.get(0).getStartRow();
+      byte[] stop  = rowkeyRanges.get(rowkeyRanges.size() - 1).getStopRow();
+
+      Preconditions.checkNotNull(start, String.format("Encountered a null start key at position %d for scan range condition %s.", count, condition.toString()));
+      Preconditions.checkNotNull(stop, String.format("Encountered a null stop key at position %d for scan range condition %s.", count, condition.toString()));
+
+      if (count > 0) {
+        // after the first start key, rest should be non-empty
+        Preconditions.checkState( !(Bytes.equals(start, MapRConstants.EMPTY_BYTE_ARRAY)), String.format("Encountered an empty start key at position %d", count));
+      }
+
+      if (count < ranges.size() - 1) {
+        // except for the last stop key, rest should be non-empty
+        Preconditions.checkState( !(Bytes.equals(stop, MapRConstants.EMPTY_BYTE_ARRAY)), String.format("Encountered an empty stop key at position %d", count));
+      }
+
+      startKeys.add(start);
+      stopKeys.add(stop);
+      count++;
+    }
+
+    // check validity; only need to check one of the lists since they are populated together
+    Preconditions.checkArgument(startKeys.size() > 0, "Found empty list of start/stopKeys.");
+
+    Preconditions.checkState(startKeys.size() == ranges.size(),
+        String.format("Mismatch between the lengths: num start keys = %d, num scan ranges = %d", startKeys.size(), ranges.size()));
+
+    Preconditions.checkState(stopKeys.size() == ranges.size(),
+        String.format("Mismatch between the lengths: num stop keys = %d, num scan ranges = %d", stopKeys.size(), ranges.size()));
+
+  }
+
+}
diff --git a/contrib/format-maprdb/src/main/java/org/apache/drill/exec/store/mapr/db/json/RestrictedJsonTableGroupScan.java b/contrib/format-maprdb/src/main/java/org/apache/drill/exec/store/mapr/db/json/RestrictedJsonTableGroupScan.java
new file mode 100644
index 0000000..48ad96d
--- /dev/null
+++ b/contrib/format-maprdb/src/main/java/org/apache/drill/exec/store/mapr/db/json/RestrictedJsonTableGroupScan.java
@@ -0,0 +1,184 @@
+/*
+ * 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.store.mapr.db.json;
+
+import java.util.List;
+import java.util.NavigableMap;
+
+import com.fasterxml.jackson.annotation.JsonCreator;
+import com.fasterxml.jackson.annotation.JsonIgnore;
+import com.fasterxml.jackson.annotation.JsonProperty;
+import com.fasterxml.jackson.annotation.JsonTypeName;
+import com.google.common.base.Preconditions;
+import com.google.common.collect.Lists;
+
+import org.apache.drill.common.expression.SchemaPath;
+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.physical.base.ScanStats.GroupScanProperty;
+import org.apache.drill.exec.planner.index.MapRDBStatistics;
+import org.apache.drill.exec.planner.cost.PluginCost;
+import org.apache.drill.exec.planner.index.Statistics;
+import org.apache.drill.exec.store.dfs.FileSystemPlugin;
+import org.apache.drill.exec.store.mapr.db.MapRDBFormatPlugin;
+import org.apache.drill.exec.store.mapr.db.MapRDBSubScan;
+import org.apache.drill.exec.store.mapr.db.MapRDBSubScanSpec;
+import org.apache.drill.exec.store.mapr.db.RestrictedMapRDBSubScan;
+import org.apache.drill.exec.store.mapr.db.RestrictedMapRDBSubScanSpec;
+import org.apache.drill.exec.store.mapr.db.TabletFragmentInfo;
+
+/**
+ * A RestrictedJsonTableGroupScan encapsulates (along with a subscan) the functionality
+ * for doing restricted (i.e skip) scan rather than sequential scan.  The skipping is based
+ * on a supplied set of row keys (primary keys) from a join operator.
+ */
+@JsonTypeName("restricted-json-scan")
+public class RestrictedJsonTableGroupScan extends JsonTableGroupScan {
+
+  @JsonCreator
+  public RestrictedJsonTableGroupScan(@JsonProperty("userName") String userName,
+                            @JsonProperty("storage") FileSystemPlugin storagePlugin,
+                            @JsonProperty("format") MapRDBFormatPlugin formatPlugin,
+                            @JsonProperty("scanSpec") JsonScanSpec scanSpec, /* scan spec of the original table */
+                            @JsonProperty("columns") List<SchemaPath> columns,
+                            @JsonProperty("")MapRDBStatistics statistics) {
+    super(userName, storagePlugin, formatPlugin, scanSpec, columns, statistics);
+  }
+
+  // TODO:  this method needs to be fully implemented
+  protected RestrictedMapRDBSubScanSpec getSubScanSpec(TabletFragmentInfo tfi) {
+    JsonScanSpec spec = scanSpec;
+    RestrictedMapRDBSubScanSpec subScanSpec =
+        new RestrictedMapRDBSubScanSpec(
+        spec.getTableName(),
+        getRegionsToScan().get(tfi), spec.getSerializedFilter(), getUserName());
+    return subScanSpec;
+  }
+
+  protected NavigableMap<TabletFragmentInfo, String> getRegionsToScan() {
+    return getRegionsToScan(formatPlugin.getRestrictedScanRangeSizeMB());
+  }
+
+  @Override
+  public MapRDBSubScan getSpecificScan(int minorFragmentId) {
+    assert minorFragmentId < endpointFragmentMapping.size() : String.format(
+        "Mappings length [%d] should be greater than minor fragment id [%d] but it isn't.", endpointFragmentMapping.size(),
+        minorFragmentId);
+    RestrictedMapRDBSubScan subscan =
+        new RestrictedMapRDBSubScan(getUserName(), formatPlugin,
+        getEndPointFragmentMapping(minorFragmentId), columns, maxRecordsToRead, TABLE_JSON);
+
+    return subscan;
+  }
+
+  private List<RestrictedMapRDBSubScanSpec> getEndPointFragmentMapping(int minorFragmentId) {
+    List<RestrictedMapRDBSubScanSpec> restrictedSubScanSpecList = Lists.newArrayList();
+    List<MapRDBSubScanSpec> subScanSpecList = endpointFragmentMapping.get(minorFragmentId);
+    for(MapRDBSubScanSpec s : subScanSpecList) {
+      restrictedSubScanSpecList.add((RestrictedMapRDBSubScanSpec) s);
+    }
+    return restrictedSubScanSpecList;
+  }
+
+  /**
+   * Private constructor, used for cloning.
+   * @param that The RestrictedJsonTableGroupScan to clone
+   */
+  private RestrictedJsonTableGroupScan(RestrictedJsonTableGroupScan that) {
+    super(that);
+  }
+
+  @Override
+  public GroupScan clone(JsonScanSpec scanSpec) {
+    RestrictedJsonTableGroupScan newScan = new RestrictedJsonTableGroupScan(this);
+    newScan.scanSpec = scanSpec;
+    newScan.resetRegionsToScan(); // resetting will force recalculation
+    return newScan;
+  }
+
+  @Override
+  public GroupScan clone(List<SchemaPath> columns) {
+    RestrictedJsonTableGroupScan newScan = new RestrictedJsonTableGroupScan(this);
+    newScan.columns = columns;
+    return newScan;
+  }
+
+  @Override
+  @JsonIgnore
+  public PhysicalOperator getNewWithChildren(List<PhysicalOperator> children) {
+    Preconditions.checkArgument(children.isEmpty());
+    return new RestrictedJsonTableGroupScan(this);
+  }
+
+  @Override
+  public ScanStats getScanStats() {
+    //TODO: ideally here we should use the rowcount from index scan, and multiply a factor of restricted scan
+    double rowCount;
+    PluginCost pluginCostModel = formatPlugin.getPluginCostModel();
+    final int avgColumnSize = pluginCostModel.getAverageColumnSize(this);
+    int numColumns = (columns == null || columns.isEmpty()) ?  STAR_COLS: columns.size();
+    // Get the restricted group scan row count - same as the right side index rows
+    rowCount = computeRestrictedScanRowcount();
+    // Get the average row size of the primary table
+    double avgRowSize = stats.getAvgRowSize(null, true);
+    if (avgRowSize == Statistics.AVG_ROWSIZE_UNKNOWN || avgRowSize == 0) {
+      avgRowSize = avgColumnSize * numColumns;
+    }
+    // restricted scan does random lookups and each row may belong to a different block, with the number
+    // of blocks upper bounded by the total num blocks in the primary table
+    double totalBlocksPrimary = Math.ceil((avgRowSize * fullTableRowCount)/pluginCostModel.getBlockSize(this));
+    double numBlocks = Math.min(totalBlocksPrimary, rowCount);
+    double diskCost = numBlocks * pluginCostModel.getRandomBlockReadCost(this);
+    // For non-covering plans, the dominating cost would be of the join back. Reduce it using the factor
+    // for biasing towards non-covering plans.
+    diskCost *= stats.getRowKeyJoinBackIOFactor();
+    logger.debug("RestrictedJsonGroupScan:{} rowCount:{}, avgRowSize:{}, blocks:{}, totalBlocks:{}, diskCost:{}",
+        System.identityHashCode(this), rowCount, avgRowSize, numBlocks, totalBlocksPrimary, diskCost);
+    return new ScanStats(GroupScanProperty.NO_EXACT_ROW_COUNT, rowCount, 1, diskCost);
+  }
+
+  private double computeRestrictedScanRowcount() {
+    double rowCount = Statistics.ROWCOUNT_UNKNOWN;
+    // The rowcount should be the same as the build side which was FORCED by putting it in forcedRowCountMap
+    if (forcedRowCountMap.get(null) != null) {
+      rowCount = forcedRowCountMap.get(null);
+    }
+    // If limit pushdown has occurred - factor it in the rowcount
+    if (rowCount == Statistics.ROWCOUNT_UNKNOWN || rowCount == 0) {
+      rowCount = (0.001f * fullTableRowCount);
+    }
+    if (this.maxRecordsToRead > 0) {
+      rowCount = Math.min(rowCount, this.maxRecordsToRead);
+    }
+    return rowCount;
+  }
+
+  @Override
+  public boolean isRestrictedScan() {
+    return true;
+  }
+
+  @Override
+  public String toString() {
+    return "RestrictedJsonTableGroupScan [ScanSpec=" + scanSpec + ", columns=" + columns
+        + ", rowcount=" + computeRestrictedScanRowcount()
+        + (maxRecordsToRead>0? ", limit=" + maxRecordsToRead : "")
+        + (getMaxParallelizationWidth()>0? ", maxwidth=" + getMaxParallelizationWidth() : "") + "]";
+  }
+}
diff --git a/contrib/format-maprdb/src/test/java/com/mapr/drill/maprdb/tests/index/IndexHintPlanTest.java b/contrib/format-maprdb/src/test/java/com/mapr/drill/maprdb/tests/index/IndexHintPlanTest.java
new file mode 100644
index 0000000..9ac27b4
--- /dev/null
+++ b/contrib/format-maprdb/src/test/java/com/mapr/drill/maprdb/tests/index/IndexHintPlanTest.java
@@ -0,0 +1,171 @@
+package com.mapr.drill.maprdb.tests.index;
+
+/*
+ * 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.
+ */
+
+import com.mapr.drill.maprdb.tests.json.BaseJsonTest;
+import com.mapr.tests.annotations.ClusterTest;
+import org.apache.drill.PlanTestBase;
+import org.junit.experimental.categories.Category;
+import org.junit.runners.MethodSorters;
+import org.junit.FixMethodOrder;
+import org.junit.Test;
+
+@FixMethodOrder(MethodSorters.NAME_ASCENDING)
+@Category(ClusterTest.class)
+public class IndexHintPlanTest extends IndexPlanTest {
+
+    private static final String defaultHavingIndexPlan = "alter session reset `planner.enable_index_planning`";
+
+    @Test
+    // A simple testcase with index hint on a table which has only one index for a column t.id.ssn;
+    // This should pick i_ssn index for the query
+    public void testSimpleIndexHint() throws Exception {
+        String hintquery = "SELECT  t.id.ssn as ssn FROM table(hbase.`index_test_primary`(type => 'maprdb', index => 'i_ssn')) as t " +
+                " where t.id.ssn = '100007423'";
+
+        String query = "SELECT t.id.ssn as ssn FROM hbase.`index_test_primary` as t where t.id.ssn = '100007423'";
+        test(defaultHavingIndexPlan);
+        PlanTestBase.testPlanMatchingPatterns(hintquery,
+                new String[] {".*JsonTableGroupScan.*tableName=.*index_test_primary.*indexName=i_ssn"},
+                new String[]{"RowKeyJoin"}
+        );
+
+        //default plan picked by optimizer.
+        PlanTestBase.testPlanMatchingPatterns(query,
+                new String[] {".*JsonTableGroupScan.*tableName=.*index_test_primary.*indexName=i_ssn"},
+                new String[]{"RowKeyJoin"}
+        );
+        testBuilder()
+                .sqlQuery(hintquery)
+                .ordered()
+                .baselineColumns("ssn").baselineValues("100007423")
+                .go();
+
+    }
+
+
+    @Test
+    // A testcase where there are multiple index to pick from but only picks the index provided as hint.
+    // A valid index is provided as hint and it is useful during the index selection process, hence it will be selected.
+    public void testHintCaseWithMultipleIndexes_1() throws Exception {
+
+        String hintquery = "SELECT t.`address`.`state` AS `state` FROM table(hbase.`index_test_primary`(type => 'maprdb', index => 'i_state_city')) as t " +
+                " where t.address.state = 'pc'";
+
+        String query = "SELECT t.`address`.`state` AS `state` FROM hbase.`index_test_primary` as t where t.address.state = 'pc'";
+        test(defaultHavingIndexPlan);
+        PlanTestBase.testPlanMatchingPatterns(hintquery,
+                new String[] {".*JsonTableGroupScan.*tableName=.*index_test_primary.*indexName=i_state_city"},
+                new String[]{"RowKeyJoin"}
+        );
+
+        //default plan picked by optimizer
+        PlanTestBase.testPlanMatchingPatterns(query,
+                new String[] {".*JsonTableGroupScan.*tableName=.*index_test_primary.*indexName=(i_state_city|i_state_age_phone)"},
+                new String[]{"RowKeyJoin"}
+        );
+
+        return;
+    }
+
+    @Test
+    // A testcase where there are multiple index to pick from but only picks the index provided as hint.
+    // A valid index is provided as hint and it is useful during the index selection process, hence it will be selected.
+    // Difference between this testcase and the one before this is that index name is switched. This shows that index hint makes sure to select only one
+    // valid index specified as hint.
+    public void testHintCaseWithMultipleIndexes_2() throws Exception {
+
+        String hintquery = "SELECT t.`address`.`state` AS `state` FROM table(hbase.`index_test_primary`(type => 'maprdb', index => 'i_state_age_phone')) as t " +
+                " where t.address.state = 'pc'";
+
+        String query = "SELECT t.`address`.`state` AS `state` FROM hbase.`index_test_primary` as t where t.address.state = 'pc'";
+        test(defaultHavingIndexPlan);
+        PlanTestBase.testPlanMatchingPatterns(hintquery,
+                new String[] {".*JsonTableGroupScan.*tableName=.*index_test_primary.*indexName=i_state_age_phone"},
+                new String[]{"RowKeyJoin"}
+        );
+
+        //default plan picked by query optimizer.
+        PlanTestBase.testPlanMatchingPatterns(query,
+                new String[] {".*JsonTableGroupScan.*tableName=.*index_test_primary.*indexName=(i_state_city|i_state_age_phone)"},
+                new String[]{"RowKeyJoin"}
+        );
+
+        return;
+    }
+
+    //Negative cases
+
+    @Test
+    // A testcase where there are multiple index to pick from but none of them equals to the index provided as hint (index hint is wrong).
+    //In this index is not at all present in the table hence it falls back to the case where the index itself is not given.
+    //Hence here one of the i_state_city or i_state_age_lic will be selected depending upon the cost.
+    public void testWithMultipleIndexesButNoIndexWithHint() throws Exception {
+
+        String hintquery = "SELECT t.`address`.`state` AS `state` FROM table(hbase.`index_test_primary`(type => 'maprdb', index => 'i_state_and_city')) as t " +
+                " where t.address.state = 'pc'";
+        test(defaultHavingIndexPlan);
+        PlanTestBase.testPlanMatchingPatterns(hintquery,
+                new String[] {".*JsonTableGroupScan.*tableName=.*index_test_primary.*indexName=(i_state_city|i_state_age_phone)"},
+                new String[]{"RowKeyJoin"}
+        );
+
+        return;
+    }
+
+    @Test
+    // A testcase where there are multiple index to pick from but none of them equals to the index provided as hint and the hint index is valid.
+    // Here the index name given is valid (i.e it is present in the table) but it is not useful.
+    // This case falls back to full table scan.
+    public void testWithMultipleIndexesButNoIndexWithValidHint() throws Exception {
+
+        String hintquery = "SELECT t.`address`.`state` AS `state` FROM table(hbase.`index_test_primary`(type => 'maprdb', index => 'i_ssn')) as t " +
+                " where t.address.state = 'pc'";
+
+        String query = "SELECT t.`address`.`state` AS `state` FROM hbase.`index_test_primary` as t where t.address.state = 'pc'";
+        test(defaultHavingIndexPlan);
+        PlanTestBase.testPlanMatchingPatterns(hintquery,
+                new String[] {".*JsonTableGroupScan.*tableName=.*index_test_primary"},
+                new String[]{"RowKeyJoin", "indexName="}
+        );
+
+        PlanTestBase.testPlanMatchingPatterns(query,
+                new String[] {".*JsonTableGroupScan.*tableName=.*index_test_primary.*indexName=(i_state_city|i_state_age_phone)"},
+                new String[]{"RowKeyJoin"}
+        );
+
+        return;
+    }
+
+
+    @Test
+    //Covering index should be generated for a simple query instead of a RowKeyJoin.
+    public void testSimpleNoRowKeyJoin() throws Exception {
+        String query = "SELECT `reverseid` from table(hbase.`index_test_primary`(type => 'maprdb', index => 'hash_i_reverseid'))  " +
+                "where `reverseid` = 1234";
+
+        test(defaultHavingIndexPlan);
+        PlanTestBase.testPlanMatchingPatterns(query,
+                new String[] {".*JsonTableGroupScan.*tableName=.*index_test_primary.*indexName=hash_i_reverseid"},
+                new String[]{"RowKeyJoin"}
+        );
+
+        return;
+   }
+}
diff --git a/contrib/format-maprdb/src/test/java/com/mapr/drill/maprdb/tests/index/IndexPlanTest.java b/contrib/format-maprdb/src/test/java/com/mapr/drill/maprdb/tests/index/IndexPlanTest.java
new file mode 100644
index 0000000..c0ea2a0
--- /dev/null
+++ b/contrib/format-maprdb/src/test/java/com/mapr/drill/maprdb/tests/index/IndexPlanTest.java
@@ -0,0 +1,1715 @@
+/*
+ * 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 com.mapr.drill.maprdb.tests.index;
+
+import com.mapr.db.Admin;
+import com.mapr.drill.maprdb.tests.MaprDBTestsSuite;
+import com.mapr.drill.maprdb.tests.json.BaseJsonTest;
+import com.mapr.tests.annotations.ClusterTest;
+import org.apache.drill.PlanTestBase;
+import org.joda.time.DateTime;
+import org.joda.time.format.DateTimeFormat;
+import org.apache.drill.common.config.DrillConfig;
+import org.junit.AfterClass;
+import org.junit.BeforeClass;
+import org.junit.FixMethodOrder;
+import org.junit.Ignore;
+import org.junit.Test;
+import org.junit.experimental.categories.Category;
+import org.junit.runners.MethodSorters;
+import java.util.Properties;
+
+
+@FixMethodOrder(MethodSorters.NAME_ASCENDING)
+@Category(ClusterTest.class)
+public class IndexPlanTest extends BaseJsonTest {
+
+  final static String PRIMARY_TABLE_NAME = "/tmp/index_test_primary";
+
+  final static int PRIMARY_TABLE_SIZE = 10000;
+  private static final String sliceTargetSmall = "alter session set `planner.slice_target` = 1";
+  private static final String sliceTargetDefault = "alter session reset `planner.slice_target`";
+  private static final String noIndexPlan = "alter session set `planner.enable_index_planning` = false";
+  private static final String defaultHavingIndexPlan = "alter session reset `planner.enable_index_planning`";
+  private static final String disableHashAgg = "alter session set `planner.enable_hashagg` = false";
+  private static final String enableHashAgg =  "alter session set `planner.enable_hashagg` = true";
+  private static final String defaultnonCoveringSelectivityThreshold = "alter session set `planner.index.noncovering_selectivity_threshold` = 0.025";
+  private static final String incrnonCoveringSelectivityThreshold = "alter session set `planner.index.noncovering_selectivity_threshold` = 0.25";
+  private static final String disableFTS = "alter session set `planner.disable_full_table_scan` = true";
+  private static final String enableFTS = "alter session reset `planner.disable_full_table_scan`";
+  private static final String preferIntersectPlans = "alter session set `planner.index.prefer_intersect_plans` = true";
+  private static final String defaultIntersectPlans = "alter session reset `planner.index.prefer_intersect_plans`";
+  private static final String lowRowKeyJoinBackIOFactor
+      = "alter session set `planner.index.rowkeyjoin_cost_factor` = 0.01";
+  private static final String defaultRowKeyJoinBackIOFactor
+      = "alter session reset `planner.index.rowkeyjoin_cost_factor`";
+
+  /**
+   *  A sample row of this 10K table:
+   ------------------+-----------------------------+--------+
+   | 1012  | {"city":"pfrrs","state":"pc"}  | {"email":"KfFzKUZwNk@gmail.com","phone":"6500005471"}  |
+   {"ssn":"100007423"}  | {"fname":"KfFzK","lname":"UZwNk"}  | {"age":53.0,"income":45.0}  | 1012   |
+   *
+   * This test suite generate random content to fill all the rows, since the random function always start from
+   * the same seed for different runs, when the row count is not changed, the data in table will always be the same,
+   * thus the query result could be predicted and verified.
+   */
+
+  @BeforeClass
+  public static void setupTableIndexes() throws Exception {
+
+    Properties overrideProps = new Properties();
+    overrideProps.setProperty("format-maprdb.json.useNumRegionsForDistribution", "true");
+    updateTestCluster(1, DrillConfig.create(overrideProps));
+
+    MaprDBTestsSuite.setupTests();
+    MaprDBTestsSuite.createPluginAndGetConf(getDrillbitContext());
+
+    test(incrnonCoveringSelectivityThreshold);
+
+    System.out.print("setupTableIndexes begins");
+    Admin admin = MaprDBTestsSuite.getAdmin();
+    if (admin != null) {
+      if (admin.tableExists(PRIMARY_TABLE_NAME)) {
+        admin.deleteTable(PRIMARY_TABLE_NAME);
+      }
+    }
+
+    LargeTableGen gen = new LargeTableGen(MaprDBTestsSuite.getAdmin());
+    /**
+     * indexDef is an array of string, LargeTableGen.generateTableWithIndex will take it as parameter to generate indexes
+     * for primary table.
+     * indexDef[3*i] defines i-th index's indexName, NOTE: IF the name begins with "hash", it is a hash index
+     * indexDef[3*i+1] indexed field,
+     * and indexDef[3*i+2] defines i-th index's non-indexed fields
+     */
+    final String[] indexDef = //null;
+        {"i_ssn", "id.ssn", "contact.phone",
+            "i_state_city", "address.state,address.city", "name.fname,name.lname",//mainly for composite key test
+            "i_age", "personal.age", "",
+            "i_income", "personal.income", "",
+            "i_lic", "driverlicense", "reverseid",
+            "i_state_city_dl", "address.state,address.city", "driverlicense",
+            "i_cast_int_ssn", "$CAST(id.ssn@INT)", "contact.phone",
+            "i_cast_vchar_lic", "$CAST(driverlicense@STRING)","contact.email",
+            "i_state_age_phone", "address.state,personal.age,contact.phone", "name.fname",
+            "i_cast_age_income_phone", "$CAST(personal.age@INT),$CAST(personal.income@INT),contact.phone", "name.fname",
+            "i_age_with_fname", "personal.age", "name.fname",
+            "hash_i_reverseid", "reverseid", "",
+            "hash_i_cast_timestamp_firstlogin", "$CAST(activity.irs.firstlogin@TIMESTAMP)", "id.ssn"
+        };
+    gen.generateTableWithIndex(PRIMARY_TABLE_NAME, PRIMARY_TABLE_SIZE, indexDef);
+  }
+
+  @AfterClass
+  public static void cleanupTableIndexes() throws Exception {
+    Admin admin = MaprDBTestsSuite.getAdmin();
+    if (admin != null) {
+      if (admin.tableExists(PRIMARY_TABLE_NAME)) {
+   //     admin.deleteTable(PRIMARY_TABLE_NAME);
+      }
+    }
+    test(defaultnonCoveringSelectivityThreshold);
+  }
+
+  @Test
+  public void CTASTestTable() throws Exception {
+    String ctasQuery = "CREATE TABLE hbase.tmp.`backup_index_test_primary` " +
+        "AS SELECT * FROM hbase.`index_test_primary` as t ";
+    test(ctasQuery);
+    test("DROP TABLE IF EXISTS hbase.tmp.`backup_index_test_primary`");
+  }
+
+  @Test
+  public void CoveringPlanWithNonIndexedField() throws Exception {
+
+    String query = "SELECT t.`contact`.`phone` AS `phone` FROM hbase.`index_test_primary` as t " +
+        " where t.id.ssn = '100007423'";
+    test(defaultHavingIndexPlan);
+    PlanTestBase.testPlanMatchingPatterns(query,
+        new String[] {".*JsonTableGroupScan.*tableName=.*index_test_primary.*indexName=i_ssn"},
+        new String[]{"RowKeyJoin"}
+    );
+
+    System.out.println("Covering Plan Verified!");
+
+    testBuilder()
+        .sqlQuery(query)
+        .ordered()
+        .baselineColumns("phone").baselineValues("6500005471")
+        .go();
+    return;
+
+  }
+
+  @Test
+  public void CoveringPlanWithOnlyIndexedField() throws Exception {
+    String query = "SELECT t.`id`.`ssn` AS `ssn` FROM hbase.`index_test_primary` as t " +
+        " where t.id.ssn = '100007423'";
+    test(defaultHavingIndexPlan);
+    PlanTestBase.testPlanMatchingPatterns(query,
+        new String[] {".*JsonTableGroupScan.*tableName=.*index_test_primary.*indexName=i_ssn"},
+        new String[]{"RowKeyJoin"}
+    );
+
+    System.out.println("Covering Plan Verified!");
+
+    testBuilder()
+        .optionSettingQueriesForTestQuery(defaultHavingIndexPlan)
+        .sqlQuery(query)
+        .ordered()
+        .baselineColumns("ssn").baselineValues("100007423")
+        .go();
+
+    return;
+  }
+
+  @Test
+  public void NoIndexPlanForNonIndexField() throws Exception {
+
+    String query = "SELECT t.`id`.`ssn` AS `ssn` FROM hbase.`index_test_primary` as t " +
+        " where t.contact.phone = '6500005471'";
+    test(defaultHavingIndexPlan);
+    PlanTestBase.testPlanMatchingPatterns(query,
+        new String[] {".*JsonTableGroupScan.*tableName=.*index_test_primary"},
+        new String[]{"RowKeyJoin", "indexName="}
+    );
+
+    System.out.println("No Index Plan Verified!");
+
+    testBuilder()
+        .sqlQuery(query)
+        .unOrdered()
+        .baselineColumns("ssn").baselineValues("100007423")
+        .baselineColumns("ssn").baselineValues("100007632")
+        .go();
+
+    return;
+  }
+
+  @Test
+  public void NonCoveringPlan() throws Exception {
+
+    String query = "SELECT t.`name`.`fname` AS `fname` FROM hbase.`index_test_primary` as t " +
+        " where t.id.ssn = '100007423'";
+    test(defaultHavingIndexPlan);
+    PlanTestBase.testPlanMatchingPatterns(query,
+        new String[] {"RowKeyJoin", ".*RestrictedJsonTableGroupScan.*tableName=.*index_test_primary,",
+           ".*JsonTableGroupScan.*tableName=.*index_test_primary,.*indexName=i_ssn"},
+        new String[]{}
+    );
+
+    System.out.println("Non-Covering Plan Verified!");
+
+    testBuilder()
+        .sqlQuery(query)
+        .ordered()
+        .baselineColumns("fname").baselineValues("KfFzK")
+        .go();
+
+    return;
+  }
+
+  @Test
+  public void RangeConditionIndexPlan() throws Exception {
+    String query = "SELECT t.`name`.`lname` AS `lname` FROM hbase.`index_test_primary` as t " +
+        " where t.personal.age > 52 AND t.name.fname='KfFzK'";
+    try {
+      test(defaultHavingIndexPlan + ";" + lowRowKeyJoinBackIOFactor + ";");
+      PlanTestBase.testPlanMatchingPatterns(query,
+              new String[]{"RowKeyJoin", ".*RestrictedJsonTableGroupScan.*tableName=.*index_test_primary,",
+                      ".*JsonTableGroupScan.*tableName=.*index_test_primary,.*indexName=(i_age|i_age_with_fname)"},
+              new String[]{}
+      );
+      testBuilder()
+              .optionSettingQueriesForTestQuery(defaultHavingIndexPlan)
+              .optionSettingQueriesForTestQuery(lowRowKeyJoinBackIOFactor)
+              .optionSettingQueriesForBaseline(noIndexPlan)
+              .unOrdered()
+              .sqlQuery(query)
+              .sqlBaselineQuery(query)
+              .build()
+              .run();
+
+      testBuilder()
+              .optionSettingQueriesForTestQuery(sliceTargetSmall)
+              .optionSettingQueriesForBaseline(sliceTargetDefault)
+              .unOrdered()
+              .sqlQuery(query)
+              .sqlBaselineQuery(query)
+              .build()
+              .run();
+    } finally {
+      test(defaultRowKeyJoinBackIOFactor);
+    }
+  }
+
+  @Test
+  public void CoveringWithSimpleFieldsOnly() throws Exception {
+
+    String query = "SELECT t._id AS `tid` FROM hbase.`index_test_primary` as t " +
+        " where t.driverlicense = 100007423";
+    test(defaultHavingIndexPlan);
+    PlanTestBase.testPlanMatchingPatterns(query,
+        new String[] {"JsonTableGroupScan.*tableName=.*index_test_primary,.*indexName=i_lic"},
+        new String[]{"RowKeyJoin"}
+    );
+
+    testBuilder()
+        .sqlQuery(query)
+        .ordered()
+        .baselineColumns("tid").baselineValues("1012")
+        .go();
+
+    return;
+  }
+
+  @Test
+  public void NonCoveringWithSimpleFieldsOnly() throws Exception {
+
+    String query = "SELECT t.rowid AS `rowid` FROM hbase.`index_test_primary` as t " +
+        " where t.driverlicense = 100007423";
+    test(defaultHavingIndexPlan);
+    PlanTestBase.testPlanMatchingPatterns(query,
+        new String[] {"RowKeyJoin(.*[\n\r])+.*" +
+            "RestrictedJsonTableGroupScan.*tableName=.*index_test_primary(.*[\n\r])+.*" +
+            "JsonTableGroupScan.*tableName=.*index_test_primary,.*indexName=i_lic"},
+        new String[]{}
+    );
+
+    testBuilder()
+        .sqlQuery(query)
+        .ordered()
+        .baselineColumns("rowid").baselineValues("1012")
+        .go();
+
+    return;
+  }
+
+  @Test
+  public void NonCoveringWithExtraConditonOnPrimary() throws Exception {
+
+    String query = "SELECT t.`name`.`fname` AS `fname` FROM hbase.`index_test_primary` as t " +
+        " where t.personal.age = 53 AND t.name.lname='UZwNk'";
+    try {
+      test(defaultHavingIndexPlan + ";" + lowRowKeyJoinBackIOFactor + ";");
+      PlanTestBase.testPlanMatchingPatterns(query,
+              new String[]{"RowKeyJoin", ".*RestrictedJsonTableGroupScan",
+                      ".*JsonTableGroupScan.*indexName=i_age",},
+              new String[]{}
+      );
+
+      testBuilder()
+              .sqlQuery(query)
+              .ordered()
+              .baselineColumns("fname").baselineValues("KfFzK")
+              .go();
+    } finally {
+      test(defaultRowKeyJoinBackIOFactor);
+    }
+    return;
+  }
+
+  @Test
+  public void Intersect2indexesPlan() throws Exception {
+
+    String query = "SELECT t.`name`.`lname` AS `lname` FROM hbase.`index_test_primary` as t " +
+        " where t.personal.age = 53 AND t.personal.income=45";
+    try {
+      test(defaultHavingIndexPlan);
+      test(preferIntersectPlans + ";" + disableFTS);
+      PlanTestBase.testPlanMatchingPatterns(query,
+              new String[]{"RowKeyJoin(.*[\n\r])+.*RestrictedJsonTableGroupScan(.*[\n\r])+.*HashJoin(.*[\n\r])+.*JsonTableGroupScan.*indexName=(i_age|i_income)(.*[\n\r])+.*JsonTableGroupScan.*indexName=(i_age|i_income)"},
+              new String[]{}
+      );
+
+      testBuilder()
+              .sqlQuery(query)
+              .unOrdered()
+              .baselineColumns("lname").baselineValues("UZwNk")
+              .baselineColumns("lname").baselineValues("foNwtze")
+              .baselineColumns("lname").baselineValues("qGZVfY")
+              .go();
+      testBuilder()
+              .optionSettingQueriesForTestQuery(sliceTargetSmall)
+              .optionSettingQueriesForBaseline(sliceTargetDefault)
+              .unOrdered()
+              .sqlQuery(query)
+              .sqlBaselineQuery(query)
+              .build()
+              .run();
+    } finally {
+      test(defaultIntersectPlans + ";" + enableFTS);
+    }
+    return;
+  }
+
+  @Test
+  public void CompositeIndexNonCoveringPlan() throws Exception {
+
+    String query = "SELECT t.`id`.`ssn` AS `ssn` FROM hbase.`index_test_primary` as t " +
+        " where t.address.state = 'pc' AND t.address.city='pfrrs'";
+    try {
+      test(defaultHavingIndexPlan + ";" + lowRowKeyJoinBackIOFactor + ";");
+
+      //either i_state_city or i_state_age_phone will be picked depends on cost model, both is fine for testing composite index nonCovering plan
+      PlanTestBase.testPlanMatchingPatterns(query,
+              new String[]{"RowKeyJoin(.*[\n\r])+.*RestrictedJsonTableGroupScan(.*[\n\r])+.*JsonTableGroupScan.*indexName=i_state_"},
+              new String[]{}
+      );
+
+      testBuilder()
+              .sqlQuery(query)
+              .unOrdered()
+              .baselineColumns("ssn").baselineValues("100007423")
+              .baselineColumns("ssn").baselineValues("100008861")
+              .go();
+
+      testBuilder()
+              .optionSettingQueriesForTestQuery(sliceTargetSmall)
+              .optionSettingQueriesForBaseline(sliceTargetDefault)
+              .unOrdered()
+              .sqlQuery(query)
+              .sqlBaselineQuery(query)
+              .build()
+              .run();
+    } finally {
+      test(defaultRowKeyJoinBackIOFactor);
+    }
+    return;
+  }
+
+  @Test//filter cover indexed, included and not in index at all filter
+  public void CompositeIndexNonCoveringFilterWithAllFieldsPlan() throws Exception {
+
+    String query = "SELECT t.`id`.`ssn` AS `ssn` FROM hbase.`index_test_primary` as t " +
+        " where t.address.state = 'pc' AND t.address.city='pfrrs' AND t.driverlicense IN (100007423, 100007424)";
+    test(defaultHavingIndexPlan+";"+lowRowKeyJoinBackIOFactor+";");
+    PlanTestBase.testPlanMatchingPatterns(query,
+        new String[] {"RowKeyJoin(.*[\n\r])+.*RestrictedJsonTableGroupScan.*condition=.*state.*city.*driverlicense.*or.*driverlicense.*(.*[\n\r])+.*JsonTableGroupScan.*indexName="},
+        new String[]{}
+    );
+
+    testBuilder()
+        .sqlQuery(query)
+        .unOrdered()
+        .baselineColumns("ssn").baselineValues("100007423")
+        .go();
+
+    testBuilder()
+        .optionSettingQueriesForTestQuery(sliceTargetSmall)
+        .optionSettingQueriesForBaseline(sliceTargetDefault)
+        .unOrdered()
+        .sqlQuery(query)
+        .sqlBaselineQuery(query)
+        .build()
+        .run();
+
+    return;
+  }
+  @Test
+  public void CompositeIndexCoveringPlan() throws Exception {
+
+    String query = "SELECT t.`address`.`city` AS `city` FROM hbase.`index_test_primary` as t " +
+        " where t.address.state = 'pc' AND t.address.city='pfrrs'";
+    test(defaultHavingIndexPlan);
+    PlanTestBase.testPlanMatchingPatterns(query,
+        new String[] {".*JsonTableGroupScan.*indexName=i_state_city"},
+        new String[]{"RowKeyJoin", "Filter"}
+    );
+
+    testBuilder()
+        .sqlQuery(query)
+        .unOrdered()
+        .baselineColumns("city").baselineValues("pfrrs")
+        .baselineColumns("city").baselineValues("pfrrs")
+        .go();
+
+    testBuilder()
+        .optionSettingQueriesForTestQuery(sliceTargetSmall)
+        .optionSettingQueriesForBaseline(sliceTargetDefault)
+        .unOrdered()
+        .sqlQuery(query)
+        .sqlBaselineQuery(query)
+        .build()
+        .run();
+    return;
+  }
+
+  @Test
+  public void TestNonCoveringRangePartition_1() throws Exception {
+
+    String query = "SELECT t.`name`.`lname` AS `lname` FROM hbase.`index_test_primary` as t " +
+        " where t.personal.age = 53";
+    String[] expectedPlan = new String[] {"RowKeyJoin(.*[\n\r])+.*" +
+        "RestrictedJsonTableGroupScan.*tableName=.*index_test_primary(.*[\n\r])+.*" +
+        "RangePartitionExchange(.*[\n\r])+.*" +
+    "JsonTableGroupScan.*tableName=.*index_test_primary,.*indexName=(i_age|i_age_with_fname)"};
+    test(defaultHavingIndexPlan+";"+sliceTargetSmall+";");
+    PlanTestBase.testPlanMatchingPatterns(query,
+        expectedPlan, new String[]{});
+
+    try {
+      testBuilder()
+          .optionSettingQueriesForTestQuery(defaultHavingIndexPlan)
+          .optionSettingQueriesForBaseline(noIndexPlan)
+          .unOrdered()
+          .sqlQuery(query)
+          .sqlBaselineQuery(query)
+          .build()
+          .run();
+    } finally {
+      test(defaultHavingIndexPlan);
+      test(sliceTargetDefault);
+    }
+    return;
+  }
+
+  @Test
+  public void TestCastVarCharCoveringPlan() throws Exception {
+    //length 255 is to exact match the casted indexed field's length
+    String query = "SELECT t._id as tid, cast(t.driverlicense as varchar(255)) as driverlicense FROM hbase.`index_test_primary` as t " +
+        " where cast(t.driverlicense as varchar(255))='100007423'";
+    test(defaultHavingIndexPlan);
+    PlanTestBase.testPlanMatchingPatterns(query,
+        new String[] {".*JsonTableGroupScan.*tableName=.*index_test_primary.*indexName=i_cast_vchar_lic"},
+        new String[]{"RowKeyJoin"}
+    );
+
+    System.out.println("TestCastCoveringPlan Plan Verified!");
+
+    testBuilder()
+        .optionSettingQueriesForTestQuery(defaultHavingIndexPlan)
+        .sqlQuery(query)
+        .ordered()
+        .baselineColumns("tid", "driverlicense").baselineValues("1012", "100007423")
+        .go();
+
+    return;
+  }
+
+  @Test
+  public void TestCastINTCoveringPlan() throws Exception {
+    String query = "SELECT t._id as tid, CAST(t.id.ssn as INT) as ssn, t.contact.phone AS `phone` FROM hbase.`index_test_primary` as t " +
+        " where CAST(t.id.ssn as INT) = 100007423";
+    test(defaultHavingIndexPlan);
+    PlanTestBase.testPlanMatchingPatterns(query,
+        new String[] {".*JsonTableGroupScan.*tableName=.*index_test_primary.*indexName=i_cast_int_ssn"},
+        new String[]{"RowKeyJoin"}
+    );
+
+    System.out.println("TestCastCoveringPlan Plan Verified!");
+
+    testBuilder()
+        .optionSettingQueriesForTestQuery(defaultHavingIndexPlan)
+        .sqlQuery(query)
+        .ordered()
+        .baselineColumns("tid", "ssn", "phone").baselineValues("1012", 100007423, "6500005471")
+        .go();
+
+    return;
+  }
+
+  @Test
+  public void TestCastNonCoveringPlan() throws Exception {
+    String query = "SELECT t.id.ssn AS `ssn` FROM hbase.`index_test_primary` as t " +
+        " where CAST(t.id.ssn as INT) = 100007423";
+    test(defaultHavingIndexPlan);
+    PlanTestBase.testPlanMatchingPatterns(query,
+        new String[] {"RowKeyJoin(.*[\n\r])+.*RestrictedJsonTableGroupScan(.*[\n\r])+.*JsonTableGroupScan.*indexName=i_cast_int_ssn"},
+        new String[]{}
+    );
+
+    System.out.println("TestCastNonCoveringPlan Plan Verified!");
+
+    testBuilder()
+        .sqlQuery(query)
+        .ordered()
+        .baselineColumns("ssn").baselineValues("100007423")
+        .go();
+    return;
+  }
+
+  @Test
+  public void TestCastVarchar_ConvertToRangePlan() throws Exception {
+    String query = "SELECT t.id.ssn AS `ssn` FROM hbase.`index_test_primary` as t " +
+        " where CAST(driverlicense as VARCHAR(10)) = '100007423'";
+    test(defaultHavingIndexPlan);
+    PlanTestBase.testPlanMatchingPatterns(query,
+        new String[] {"RowKeyJoin(.*[\n\r])+.*RestrictedJsonTableGroupScan(.*[\n\r])+.*JsonTableGroupScan.*MATCHES \"\\^.*100007423.*E.*\\$\".*indexName=i_cast_vchar_lic"},
+        new String[]{}
+    );
+
+    System.out.println("TestCastVarchar_ConvertToRangePlan Verified!");
+
+    testBuilder()
+        .sqlQuery(query)
+        .ordered()
+        .baselineColumns("ssn").baselineValues("100007423")
+        .go();
+
+    return;
+  }
+
+  @Test // cast expression in filter is not indexed, but the same field casted to different type was indexed (CAST id.ssn as INT)
+  public void TestCastNoIndexPlan() throws Exception {
+    String query = "select t.id.ssn from hbase.`index_test_primary` t where cast(t.id.ssn as varchar(10)) = '100007423'";
+
+    PlanTestBase.testPlanMatchingPatterns(query,
+        new String[]{},
+        new String[]{"indexName"}
+    );
+
+  }
+
+  @Test
+  public void TestLongerCastVarCharNoIndex() throws Exception {
+    //length 256 is to exact match the casted indexed field's length
+    String query = "SELECT t._id as tid, cast(t.driverlicense as varchar(500)) as driverlicense FROM hbase.`index_test_primary` as t " +
+        " where cast(t.driverlicense as varchar(500))='100007423'";
+    test(defaultHavingIndexPlan);
+    PlanTestBase.testPlanMatchingPatterns(query,
+        new String[] {},
+        new String[]{"RowKeyJoin", "indexName="}
+    );
+
+    System.out.println("TestLongerCastVarCharNoIndex Plan Verified!");
+
+    return;
+  }
+
+  @Test
+  public void TestCoveringPlanSortRemoved() throws Exception {
+    String query = "SELECT t.`contact`.`phone` as phone FROM hbase.`index_test_primary` as t " +
+        " where t.id.ssn <'100000003' order by t.id.ssn";
+    test(defaultHavingIndexPlan);
+    PlanTestBase.testPlanMatchingPatterns(query,
+        new String[] {".*JsonTableGroupScan.*tableName=.*index_test_primary.*indexName=i_ssn"},
+        new String[]{"Sort"}
+    );
+
+    testBuilder()
+        .sqlQuery(query)
+        .ordered()
+        .baselineColumns("phone").baselineValues("6500008069")
+        .baselineColumns("phone").baselineValues("6500001411")
+        .baselineColumns("phone").baselineValues("6500001595")
+        .go();
+  }
+
+  @Test
+  public void TestCoveringPlanSortNotRemoved() throws Exception {
+    String query = "SELECT t.`contact`.`phone` as phone FROM hbase.`index_test_primary` as t " +
+        " where t.id.ssn <'100000003' order by t.contact.phone";
+    test(defaultHavingIndexPlan);
+    PlanTestBase.testPlanMatchingPatterns(query,
+        new String[] {"Sort", ".*JsonTableGroupScan.*tableName=.*index_test_primary.*indexName=i_ssn"},
+        new String[]{"RowkeyJoin"}
+    );
+
+    testBuilder()
+        .sqlQuery(query)
+        .ordered()
+        .baselineColumns("phone").baselineValues("6500001411")
+        .baselineColumns("phone").baselineValues("6500001595")
+        .baselineColumns("phone").baselineValues("6500008069")
+        .go();
+  }
+
+  @Test
+  public void TestCoveringPlanSortRemovedWithSimpleFields() throws Exception {
+    String query = "SELECT t.driverlicense as l FROM hbase.`index_test_primary` as t " +
+        " where t.driverlicense < 100000003 order by t.driverlicense";
+    test(defaultHavingIndexPlan);
+    PlanTestBase.testPlanMatchingPatterns(query,
+        new String[] {".*JsonTableGroupScan.*tableName=.*index_test_primary.*indexName=i_lic"},
+        new String[]{"Sort"}
+    );
+
+    testBuilder()
+        .sqlQuery(query)
+        .ordered()
+        .baselineColumns("l").baselineValues(100000000l)
+        .baselineColumns("l").baselineValues(100000001l)
+        .baselineColumns("l").baselineValues(100000002l)
+        .go();
+  }
+
+  @Test
+  public void TestNonCoveringPlanSortRemoved() throws Exception {
+    String query = "SELECT t.contact.phone as phone FROM hbase.`index_test_primary` as t " +
+        " where t.driverlicense < 100000003 order by t.driverlicense";
+    test(defaultHavingIndexPlan);
+    PlanTestBase.testPlanMatchingPatterns(query,
+        new String[] {"RowKeyJoin(.*[\n\r])+.*RestrictedJsonTableGroupScan(.*[\n\r])+.*JsonTableGroupScan.*indexName=i_lic"},
+        new String[]{"Sort"}
+    );
+    String query2 = "SELECT t.name.fname as fname FROM hbase.`index_test_primary` as t " +
+        " where t.id.ssn < '100000003' order by t.id.ssn";
+    PlanTestBase.testPlanMatchingPatterns(query2,
+        new String[] {"RowKeyJoin(.*[\n\r])+.*RestrictedJsonTableGroupScan(.*[\n\r])+.*JsonTableGroupScan.*indexName="},
+        new String[]{"Sort"}
+    );
+
+    //simple field, driverlicense
+    testBuilder()
+        .sqlQuery(query)
+        .ordered()
+        .baselineColumns("phone").baselineValues("6500008069")
+        .baselineColumns("phone").baselineValues("6500001411")
+        .baselineColumns("phone").baselineValues("6500001595")
+        .go();
+
+    //query on field of item expression(having capProject), non-simple field t.id.ssn
+    testBuilder()
+        .sqlQuery(query2)
+        .ordered()
+        .baselineColumns("fname").baselineValues("VcFahj")
+        .baselineColumns("fname").baselineValues("WbKVK")
+        .baselineColumns("fname").baselineValues("vSAEsyFN")
+        .go();
+
+    test(sliceTargetSmall);
+    try {
+      PlanTestBase.testPlanMatchingPatterns(query2,
+          new String[]{"SingleMergeExchange(.*[\n\r])+.*"
+              + "RowKeyJoin(.*[\n\r])+.*RestrictedJsonTableGroupScan(.*[\n\r])+.*JsonTableGroupScan.*indexName=i_ssn"},
+          new String[]{"Sort"}
+      );
+    } finally {
+      test(sliceTargetDefault);
+    }
+  }
+
+  //test cases are from TestNonCoveringPlanSortRemoved. Sort was removed when force_sort_noncovering was default(false)
+  @Test
+  public void TestNonCoveringPlanWithNoRemoveSortOption() throws Exception {
+    try {
+      test("alter session set `planner.index.force_sort_noncovering`=true");
+      test(defaultHavingIndexPlan);
+
+      String query = "SELECT t.contact.phone as phone FROM hbase.`index_test_primary` as t " +
+          " where t.driverlicense < 100000003 order by t.driverlicense";
+      PlanTestBase.testPlanMatchingPatterns(query,
+          new String[]{"Sort", "RowKeyJoin(.*[\n\r])+.*RestrictedJsonTableGroupScan(.*[\n\r])+.*JsonTableGroupScan.*indexName=i_lic"},
+          new String[]{}
+      );
+
+      String query2 = "SELECT t.name.fname as fname FROM hbase.`index_test_primary` as t " +
+          " where t.id.ssn < '100000003' order by t.id.ssn";
+      PlanTestBase.testPlanMatchingPatterns(query2,
+          new String[]{"Sort", "RowKeyJoin(.*[\n\r])+.*RestrictedJsonTableGroupScan(.*[\n\r])+.*JsonTableGroupScan.*indexName="},
+          new String[]{}
+      );
+
+      //simple field, driverlicense
+      testBuilder()
+          .sqlQuery(query)
+          .ordered()
+          .baselineColumns("phone").baselineValues("6500008069")
+          .baselineColumns("phone").baselineValues("6500001411")
+          .baselineColumns("phone").baselineValues("6500001595")
+          .go();
+
+      //query on field of item expression(having capProject), non-simple field t.id.ssn
+      testBuilder()
+          .sqlQuery(query2)
+          .ordered()
+          .baselineColumns("fname").baselineValues("VcFahj")
+          .baselineColumns("fname").baselineValues("WbKVK")
+          .baselineColumns("fname").baselineValues("vSAEsyFN")
+          .go();
+
+      test(sliceTargetSmall);
+      try {
+        PlanTestBase.testPlanMatchingPatterns(query2,
+            new String[]{"Sort", "SingleMergeExchange(.*[\n\r])+.*"
+                + "RowKeyJoin(.*[\n\r])+.*RestrictedJsonTableGroupScan(.*[\n\r])+.*JsonTableGroupScan.*indexName=i_ssn"},
+            new String[]{}
+        );
+      } finally {
+        test(sliceTargetDefault);
+      }
+    }
+    finally {
+      test("alter session reset `planner.index.force_sort_noncovering`");
+    }
+  }
+
+  @Test  // 2 table join, each table has local predicate on top-level column
+  public void TestCoveringPlanJoin_1() throws Exception {
+    String query = "SELECT count(*) as cnt FROM hbase.`index_test_primary` as t1 " +
+        " inner join hbase.`index_test_primary` as t2 on t1.driverlicense = t2.driverlicense " +
+        " where t1.driverlicense < 100000003 and t2.driverlicense < 100000003" ;
+    test(defaultHavingIndexPlan);
+    PlanTestBase.testPlanMatchingPatterns(query,
+        new String[] {".*JsonTableGroupScan.*tableName=.*index_test_primary.*indexName=",
+                      ".*JsonTableGroupScan.*tableName=.*index_test_primary.*indexName="},
+        new String[]{}
+    );
+
+    testBuilder()
+        .sqlQuery(query)
+        .ordered()
+        .baselineColumns("cnt").baselineValues(3L)
+        .go();
+  }
+
+  @Test  // 2 table join, each table has local predicate on nested column
+  public void TestCoveringPlanJoin_2() throws Exception {
+    String query = "SELECT count(*) as cnt FROM hbase.`index_test_primary` as t1 " +
+        " inner join hbase.`index_test_primary` as t2 on t1.contact.phone = t2.contact.phone " +
+        " where t1.id.ssn < '100000003' and t2.id.ssn < '100000003' ";
+    test(defaultHavingIndexPlan);
+    PlanTestBase.testPlanMatchingPatterns(query,
+        new String[] {".*JsonTableGroupScan.*tableName=.*index_test_primary.*indexName=",
+                      ".*JsonTableGroupScan.*tableName=.*index_test_primary.*indexName="},
+        new String[]{}
+    );
+
+    testBuilder()
+       .sqlQuery(query)
+       .ordered()
+       .baselineColumns("cnt").baselineValues(3L)
+       .go();
+  }
+
+  @Test  // leading prefix of index has Equality conditions and ORDER BY last column; Sort SHOULD be dropped
+  public void TestCoveringPlanSortPrefix_1() throws Exception {
+    String query = "SELECT t.contact.phone FROM hbase.`index_test_primary` as t " +
+        " where t.address.state = 'wo' and t.personal.age = 35 and t.contact.phone < '6500003000' order by t.contact.phone";
+    test(defaultHavingIndexPlan);
+
+    //we should glue to index i_state_age_phone to make sure we are testing the targeted prefix construction code path
+    PlanTestBase.testPlanMatchingPatterns(query,
+        new String[] {".*JsonTableGroupScan.*tableName=.*index_test_primary.*indexName=i_state_age_phone"},
+        new String[]{"Sort"}
+    );
+
+    // compare the results of index plan with the no-index plan
+    testBuilder()
+      .optionSettingQueriesForTestQuery(defaultHavingIndexPlan)
+      .optionSettingQueriesForBaseline(noIndexPlan)
+      .unOrdered()
+      .sqlQuery(query)
+      .sqlBaselineQuery(query)
+      .build()
+      .run();
+  }
+
+  @Test  // leading prefix of index has Non-Equality conditions and ORDER BY last column; Sort SHOULD NOT be dropped
+  public void TestCoveringPlanSortPrefix_2() throws Exception {
+    String query = "SELECT t.contact.phone FROM hbase.`index_test_primary` as t " +
+        " where t.address.state = 'wo' and t.personal.age < 35 and t.contact.phone < '6500003000' order by t.contact.phone";
+    test(defaultHavingIndexPlan);
+    PlanTestBase.testPlanMatchingPatterns(query,
+        new String[] {"Sort", ".*JsonTableGroupScan.*tableName=.*index_test_primary.*indexName=i_state_age_phone"},
+        new String[]{}
+    );
+
+    // compare the results of index plan with the no-index plan
+    testBuilder()
+      .optionSettingQueriesForTestQuery(defaultHavingIndexPlan)
+      .optionSettingQueriesForBaseline(noIndexPlan)
+      .unOrdered()
+      .sqlQuery(query)
+      .sqlBaselineQuery(query)
+      .build()
+      .run();
+  }
+
+  @Test  //ORDER BY last two columns not in the indexed order; Sort SHOULD NOT be dropped
+  public void TestCoveringPlanSortPrefix_3() throws Exception {
+    String query = "SELECT CAST(t.personal.age as VARCHAR) as age, t.contact.phone FROM hbase.`index_test_primary` as t " +
+        " where t.address.state = 'wo' and t.personal.age < 35 and t.contact.phone < '6500003000' order by t.contact.phone, t.personal.age";
+    test(defaultHavingIndexPlan);
+    PlanTestBase.testPlanMatchingPatterns(query,
+        new String[] {"Sort", ".*JsonTableGroupScan.*tableName=.*index_test_primary.*indexName=i_state_age_phone"},
+        new String[]{}
+    );
+
+    // compare the results of index plan with the no-index plan
+    testBuilder()
+        .optionSettingQueriesForTestQuery(defaultHavingIndexPlan)
+        .optionSettingQueriesForBaseline(noIndexPlan)
+        .unOrdered()
+        .sqlQuery(query)
+        .sqlBaselineQuery(query)
+        .build()
+        .run();
+  }
+
+  @Test  // last two index fields in non-Equality conditions, ORDER BY last two fields; Sort SHOULD be dropped
+  public void TestCoveringPlanSortPrefix_4() throws Exception {
+    String query = "SELECT t._id as tid, t.contact.phone, CAST(t.personal.age as VARCHAR) as age FROM hbase.`index_test_primary` as t " +
+        " where t.address.state = 'wo' and t.personal.age < 35 and t.contact.phone < '6500003000' order by t.personal.age, t.contact.phone";
+    test(defaultHavingIndexPlan);
+    PlanTestBase.testPlanMatchingPatterns(query,
+        new String[] {".*JsonTableGroupScan.*tableName=.*index_test_primary.*indexName=i_state_age_phone"},
+        new String[]{"Sort"}
+    );
+
+    // compare the results of index plan with the no-index plan
+    testBuilder()
+        .optionSettingQueriesForTestQuery(defaultHavingIndexPlan)
+        .optionSettingQueriesForBaseline(noIndexPlan)
+        .unOrdered()
+        .sqlQuery(query)
+        .sqlBaselineQuery(query)
+        .build()
+        .run();
+  }
+
+  @Test  // index field in two or more equality conditions, it is not leading prefix, Sort SHOULD NOT be dropped
+  public void TestCoveringPlanSortPrefix_5() throws Exception {
+    String query = "SELECT t._id as tid, t.contact.phone, CAST(t.personal.age as VARCHAR) as age FROM hbase.`index_test_primary` as t " +
+        " where t.address.state = 'wo' and t.personal.age IN (31, 32, 33, 34) and t.contact.phone < '6500003000' order by t.contact.phone";
+    test(defaultHavingIndexPlan);
+    PlanTestBase.testPlanMatchingPatterns(query,
+        new String[] {"Sort", ".*JsonTableGroupScan.*tableName=.*index_test_primary.*indexName=i_state_age_phone"},
+        new String[]{}
+    );
+
+    // compare the results of index plan with the no-index plan
+    testBuilder()
+        .optionSettingQueriesForTestQuery(defaultHavingIndexPlan)
+        .optionSettingQueriesForBaseline(noIndexPlan)
+        .unOrdered()
+        .sqlQuery(query)
+        .sqlBaselineQuery(query)
+        .build()
+        .run();
+  }
+
+  @Test  // last two index fields in non-Equality conditions, ORDER BY last two fields NULLS FIRST; Sort SHOULD NOT be dropped
+  public void TestCoveringPlanSortPrefix_6() throws Exception {
+    String query = "SELECT t._id as tid, t.contact.phone, CAST(t.personal.age as VARCHAR) as age FROM hbase.`index_test_primary` as t " +
+        " where t.address.state = 'wo' and t.personal.age < 35 and t.contact.phone < '6500003000' order by t.personal.age, t.contact.phone NULLS FIRST";
+    test(defaultHavingIndexPlan);
+    PlanTestBase.testPlanMatchingPatterns(query,
+        new String[] {"Sort", ".*JsonTableGroupScan.*tableName=.*index_test_primary.*indexName=i_state_age_phone"},
+        new String[]{}
+    );
+
+    // compare the results of index plan with the no-index plan
+    testBuilder()
+        .optionSettingQueriesForTestQuery(defaultHavingIndexPlan)
+        .optionSettingQueriesForBaseline(noIndexPlan)
+        .unOrdered()
+        .sqlQuery(query)
+        .sqlBaselineQuery(query)
+        .build()
+        .run();
+  }
+
+  @Test  // last two index fields in non-Equality conditions, ORDER BY last two fields NULLS LAST; Sort SHOULD be dropped
+  public void TestCoveringPlanSortPrefix_7() throws Exception {
+    String query = "SELECT t._id as tid, t.contact.phone, CAST(t.personal.age as VARCHAR) as age FROM hbase.`index_test_primary` as t " +
+        " where t.address.state = 'wo' and t.personal.age < 35 and t.contact.phone < '6500003000' order by t.personal.age, t.contact.phone NULLS LAST";
+    test(defaultHavingIndexPlan);
+    PlanTestBase.testPlanMatchingPatterns(query,
+        new String[] {".*JsonTableGroupScan.*tableName=.*index_test_primary.*indexName=i_state_age_phone"},
+        new String[]{"Sort"}
+    );
+
+    // compare the results of index plan with the no-index plan
+    testBuilder()
+        .optionSettingQueriesForTestQuery(defaultHavingIndexPlan)
+        .optionSettingQueriesForBaseline(noIndexPlan)
+        .unOrdered()
+        .sqlQuery(query)
+        .sqlBaselineQuery(query)
+        .build()
+        .run();
+  }
+
+  @Test
+  public void orderByCastCoveringPlan() throws Exception {
+    String query = "SELECT t.contact.phone as phone FROM hbase.`index_test_primary` as t " +
+        " where CAST(t.id.ssn as INT) < 100000003 order by CAST(t.id.ssn as INT)";
+    test(defaultHavingIndexPlan);
+    PlanTestBase.testPlanMatchingPatterns(query,
+        new String[] {".*JsonTableGroupScan.*tableName=.*index_test_primary.*indexName="},
+        new String[]{"Sort"}
+    );
+
+    testBuilder()
+        .sqlQuery(query)
+        .ordered()
+        .baselineColumns("phone").baselineValues("6500008069")
+        .baselineColumns("phone").baselineValues("6500001411")
+        .baselineColumns("phone").baselineValues("6500001595")
+        .go();
+  }
+
+  @Test // non-covering plan. sort by the only indexed field, sort SHOULD be removed
+  public void orderByNonCoveringPlan() throws Exception {
+    String query = "SELECT t.name.lname as lname FROM hbase.`index_test_primary` as t " +
+        " where t.id.ssn < '100000003' order by t.id.ssn";
+    test(defaultHavingIndexPlan);
+    PlanTestBase.testPlanMatchingPatterns(query,
+        new String[] {"RowKeyJoin(.*[\n\r])+.*RestrictedJsonTableGroupScan(.*[\n\r])+.*JsonTableGroupScan.*indexName="},
+        new String[]{"Sort"}
+    );
+
+    testBuilder()
+        .sqlQuery(query)
+        .ordered()
+        .baselineColumns("lname").baselineValues("iuMG")
+        .baselineColumns("lname").baselineValues("KpFq")
+        .baselineColumns("lname").baselineValues("bkkAvz")
+        .go();
+  }
+
+  @Test //non-covering plan. order by cast indexed field, sort SHOULD be removed
+  public void orderByCastNonCoveringPlan() throws Exception {
+    String query = "SELECT t.name.lname as lname FROM hbase.`index_test_primary` as t " +
+        " where CAST(t.id.ssn as INT) < 100000003 order by CAST(t.id.ssn as INT)";
+    test(defaultHavingIndexPlan);
+    PlanTestBase.testPlanMatchingPatterns(query,
+        new String[] {"RowKeyJoin(.*[\n\r])+.*RestrictedJsonTableGroupScan(.*[\n\r])+.*JsonTableGroupScan.*indexName="},
+        new String[]{"Sort"}
+    );
+
+    testBuilder()
+        .sqlQuery(query)
+        .ordered()
+        .baselineColumns("lname").baselineValues("iuMG")
+        .baselineColumns("lname").baselineValues("KpFq")
+        .baselineColumns("lname").baselineValues("bkkAvz")
+        .go();
+  }
+
+
+  @Ignore //in statsCache, condition state+city has rowcount 1250, but state only has 1000. so it is picking i_state_age_phone
+  @Test //non-covering, order by non leading field, and leading fields are not in equality condition, Sort SHOULD NOT be removed
+  public void NonCoveringPlan_SortPrefix_1() throws Exception {
+
+    String query = "SELECT t.`id`.`ssn` AS `ssn` FROM hbase.`index_test_primary` as t " +
+        " where t.address.state > 'pc' AND t.address.city>'pfrrr' AND t.address.city<'pfrrt' order by t.adddress.city";
+    test(defaultHavingIndexPlan);
+    PlanTestBase.testPlanMatchingPatterns(query,
+        new String[] {"Sort",
+            "RowKeyJoin(.*[\n\r])+.*RestrictedJsonTableGroupScan(.*[\n\r])+.*JsonTableGroupScan.*indexName=i_state_city"},
+        new String[]{}
+    );
+    return;
+  }
+
+  @Test //non-covering, order by non leading field, and leading fields are in equality condition, Sort SHOULD be removed
+  public void NonCoveringPlan_SortPrefix_2() throws Exception {
+
+    String query = "SELECT t.`id`.`ssn` AS `ssn` FROM hbase.`index_test_primary` as t " +
+        " where t.address.state = 'pc' AND t.address.city>'pfrrr' AND t.address.city<'pfrrt' order by t.address.city";
+    test(defaultHavingIndexPlan);
+    PlanTestBase.testPlanMatchingPatterns(query,
+        new String[] {
+            "RowKeyJoin(.*[\n\r])+.*RestrictedJsonTableGroupScan(.*[\n\r])+.*JsonTableGroupScan.*indexName=i_state_city"},
+        new String[]{"Sort"}
+    );
+    return;
+  }
+
+  @Ignore ("Should be modified to get an index plan; not very useful since most covering plan filters get pushed")
+  @Test //Correct projection and results when filter on non-indexed column in covering plan.
+  public void nonIndexedColumnFilterCoveringPlan() throws Exception {
+    String query = "SELECT t.name.fname as fname FROM hbase.`index_test_primary` as t " +
+        " where t.personal.age > 68 and t.name.fname IN ('CnGobfR', 'THOHP')";
+    test(defaultHavingIndexPlan);
+    PlanTestBase.testPlanMatchingPatterns(query,
+        new String[] {".*Filter.*CnGobfR.*THOHP.*",
+            ".*JsonTableGroupScan.*tableName=.*index_test_primary.*indexName="},
+        new String[] {".*Filter.*ITEM*CnGobfR.*THOHP.*"});
+
+    testBuilder()
+        .sqlQuery(query)
+        .ordered()
+        .baselineColumns("fname").baselineValues("CnGobfR")
+        .baselineColumns("fname").baselineValues("THOHP")
+        .baselineColumns("fname").baselineValues("CnGobfR")
+        .go();
+  }
+
+  @Test
+  @Ignore ("Fix after MEP 5.0")
+  public void orderByLimitNonCoveringPlan() throws Exception {
+    String query = "SELECT t.name.lname as lname FROM hbase.`index_test_primary` as t " +
+        " where t.id.ssn < '100000003' order by t.id.ssn limit 2";
+    try {
+      test(defaultHavingIndexPlan);
+      test(sliceTargetSmall);
+      PlanTestBase.testPlanMatchingPatterns(query,
+          new String[]{"Limit(.*[\n\r])+.*SingleMergeExchange(.*[\n\r])+.*Limit(.*[\n\r])+.*indexName="},
+          new String[]{"Sort"}
+      );
+
+      testBuilder()
+          .sqlQuery(query)
+          .ordered()
+          .baselineColumns("lname").baselineValues("iuMG")
+          .baselineColumns("lname").baselineValues("KpFq")
+          .go();
+    } finally {
+      test(sliceTargetDefault);
+    }
+  }
+
+  @Test
+  public void orderByLimitCoveringPlan() throws Exception {
+    String query = "SELECT t.contact.phone as phone FROM hbase.`index_test_primary` as t " +
+        " where t.id.ssn < '100000003' order by t.id.ssn limit 2";
+    test(defaultHavingIndexPlan);
+
+    //when index table has only one tablet, the SingleMergeExchange in the middle of two Limits will be removed.
+    //The lower limit gets pushed into the scan
+    PlanTestBase.testPlanMatchingPatterns(query,
+        new String[] {"Limit(.*[\n\r])+.*indexName=.*limit=2"},
+        new String[]{"Sort"}
+    );
+
+    testBuilder()
+        .sqlQuery(query)
+        .ordered()
+        .baselineColumns("phone").baselineValues("6500008069")
+        .baselineColumns("phone").baselineValues("6500001411")
+        .go();
+  }
+
+  @Test
+  public void pickAnyIndexWithFTSDisabledPlan() throws Exception {
+    String lowCoveringSel = "alter session set `planner.index.covering_selectivity_threshold` = 0.025";
+    String defaultCoveringSel = "alter session reset `planner.index.covering_selectivity_threshold`";
+    String query = "SELECT t.`contact`.`phone` AS `phone` FROM hbase.`index_test_primary` as t " +
+        " where t.id.ssn = '100007423'";
+    try {
+      test(defaultHavingIndexPlan + ";" + lowCoveringSel + ";");
+      PlanTestBase.testPlanMatchingPatterns(query,
+          new String[]{".*JsonTableGroupScan.*tableName=.*index_test_primary"},
+          new String[]{".*indexName=i_ssn"}
+      );
+      // Must not throw CANNOTPLANEXCEPTION
+      test(defaultHavingIndexPlan + ";" + lowCoveringSel + ";" + disableFTS + ";");
+      PlanTestBase.testPlanMatchingPatterns(query,
+          new String[]{".*JsonTableGroupScan.*tableName=.*index_test_primary.*indexName=i_ssn"},
+          new String[]{"RowKeyJoin"}
+      );
+    } finally {
+      test(defaultCoveringSel+";"+enableFTS+";");
+    }
+  }
+
+  @Test
+  public void testCaseSensitive() throws Exception {
+    String query = "SELECT t.contact.phone as phone FROM hbase.`index_test_primary` as t " +
+        " where t.id.SSN = '100000003' ";
+    test(defaultHavingIndexPlan);
+
+    PlanTestBase.testPlanMatchingPatterns(query,
+        new String[] {""},
+        new String[]{"indexName"}
+    );
+
+  }
+
+  @Test
+  public void testCaseSensitiveIncludedField() throws Exception {
+
+    String query = "SELECT t.`CONTACT`.`phone` AS `phone` FROM hbase.`index_test_primary` as t " +
+        " where t.id.ssn = '100007423'";
+    test(defaultHavingIndexPlan);
+
+    PlanTestBase.testPlanMatchingPatterns(query,
+        new String[]{"RowKeyJoin",
+            ".*JsonTableGroupScan.*tableName=.*index_test_primary.*indexName=i_ssn"},
+        new String[]{}
+    );
+  }
+
+
+  @Test
+  public void testHashIndexNoRemovingSort() throws Exception {
+    String query = "SELECT t.`contact`.`phone` as phone FROM hbase.`index_test_primary` as t " +
+        " where t.reverseid <'10' order by t.reverseid";
+    test(defaultHavingIndexPlan);
+    PlanTestBase.testPlanMatchingPatterns(query,
+        new String[] {"Sort", "indexName=hash_i_reverseid", "RowKeyJoin"},
+        new String[]{}
+    );
+  }
+
+  @Test
+  public void testCastTimestampPlan() throws Exception {
+    String query = "SELECT  t.id.ssn as ssn FROM hbase.`index_test_primary` as t " +
+        " where cast(t.activity.irs.firstlogin as timestamp)=to_timestamp('2013-02-04 22:34:38.0', 'YYYY-MM-dd HH:mm:ss.S')";
+    test(defaultHavingIndexPlan);
+    PlanTestBase.testPlanMatchingPatterns(query,
+        new String[] {"indexName=hash_i_cast_timestamp_firstlogin"},
+        new String[]{"RowKeyJoin"}
+    );
+    testBuilder()
+        .sqlQuery(query)
+        .ordered()
+        .baselineColumns("ssn").baselineValues("100007423")
+        .go();
+
+  }
+
+  @Test
+  public void testNotConditionNoIndexPlan() throws Exception {
+    String query = "SELECT t.`id`.`ssn` AS `ssn` FROM hbase.`index_test_primary` as t " +
+        " where NOT t.id.ssn = '100007423'";
+
+    test(defaultHavingIndexPlan);
+    PlanTestBase.testPlanMatchingPatterns(query,
+        new String[] {},
+        new String[]{"indexName="}
+    );
+
+
+    String notInQuery = "SELECT t.`id`.`ssn` AS `ssn` FROM hbase.`index_test_primary` as t " +
+        " where t.id.ssn NOT IN ('100007423', '100007424')";
+    PlanTestBase.testPlanMatchingPatterns(notInQuery,
+        new String[] {},
+        new String[]{"indexName="}
+    );
+
+    String notLikeQuery = "SELECT t.`id`.`ssn` AS `ssn` FROM hbase.`index_test_primary` as t " +
+        " where t.id.ssn NOT LIKE '100007423'";
+    PlanTestBase.testPlanMatchingPatterns(notLikeQuery,
+        new String[] {},
+        new String[]{"indexName="}
+    );
+
+  }
+
+  @Test
+  public void testNoFilterOrderByCoveringPlan() throws Exception {
+    String query = "SELECT t.`id`.`ssn` AS `ssn`, t.contact.phone as phone FROM hbase.`index_test_primary` as t " +
+        "order by t.id.ssn limit 2";
+    test(defaultHavingIndexPlan);
+    PlanTestBase.testPlanMatchingPatterns(query,
+        new String[] {"indexName=i_ssn"},
+        new String[]{"Sort", "TopN", "RowKeyJoin"}
+    );
+    testBuilder()
+        .ordered()
+        .sqlQuery(query)
+        .baselineColumns("ssn", "phone").baselineValues("100000000", "6500008069")
+        .baselineColumns("ssn", "phone").baselineValues("100000001", "6500001411")
+        .build()
+        .run();
+  }
+
+  @Test
+  public void testNoFilterAndLimitOrderByCoveringPlan() throws Exception {
+    String query = "SELECT t.`id`.`ssn` AS `ssn`, t.contact.phone as phone FROM hbase.`index_test_primary` as t " +
+            "order by t.id.ssn";
+    test(defaultHavingIndexPlan);
+    PlanTestBase.testPlanMatchingPatterns(query,
+            new String[] {"Sort"},
+            new String[]{"indexName=*", "RowKeyJoin", "TopN"}
+    );
+  }
+
+  @Test
+  public void testNoFilterOrderByCast() throws Exception {
+    String query = "SELECT CAST(t.id.ssn as INT) AS `ssn`, t.contact.phone as phone FROM hbase.`index_test_primary` as t " +
+        "order by CAST(t.id.ssn as INT) limit 2";
+    test(defaultHavingIndexPlan);
+    PlanTestBase.testPlanMatchingPatterns(query,
+        new String[] {"indexName=i_cast_int_ssn"},
+        new String[]{"TopN", "Sort", "RowKeyJoin"}
+    );
+    testBuilder()
+        .ordered()
+        .sqlQuery(query)
+        .baselineColumns("ssn", "phone").baselineValues(100000000, "6500008069")
+        .baselineColumns("ssn", "phone").baselineValues(100000001, "6500001411")
+        .build()
+        .run();
+  }
+
+  @Test
+  public void testNoFilterAndLimitOrderByCast() throws Exception {
+    String query = "SELECT CAST(t.id.ssn as INT) AS `ssn`, t.contact.phone as phone FROM hbase.`index_test_primary` as t " +
+            "order by CAST(t.id.ssn as INT)";
+    test(defaultHavingIndexPlan);
+    PlanTestBase.testPlanMatchingPatterns(query,
+            new String[] { "Sort"},
+            new String[]{"indexName=*","TopN", "RowKeyJoin"}
+    );
+  }
+
+  @Test
+  public void testNoFilterOrderByHashIndex() throws Exception {
+    String query = "SELECT cast(t.activity.irs.firstlogin as timestamp) AS `firstlogin`, t.id.ssn as ssn FROM hbase.`index_test_primary` as t " +
+        "order by cast(t.activity.irs.firstlogin as timestamp), t.id.ssn limit 2";
+    test(defaultHavingIndexPlan);
+    //no collation for hash index so Sort or TopN must have been preserved
+    PlanTestBase.testPlanMatchingPatterns(query,
+        new String[] {"(Sort|TopN)"},
+        new String[]{"indexName="}
+    );
+    DateTime date = DateTimeFormat.forPattern("yyyy-MM-dd HH:mm:ss")
+        .parseDateTime("2010-01-21 00:12:24");
+
+    testBuilder()
+        .ordered()
+        .sqlQuery(query)
+        .baselineColumns("firstlogin", "ssn").baselineValues(date, "100005592")
+        .baselineColumns("firstlogin", "ssn").baselineValues(date, "100005844")
+        .build()
+        .run();
+  }
+
+  @Test
+  public void testNoFilterOrderBySimpleField() throws Exception {
+    String query = "SELECT t.reverseid as rid, t.driverlicense as lic FROM hbase.`index_test_primary` as t " +
+        "order by t.driverlicense limit 2";
+    test(defaultHavingIndexPlan);
+    PlanTestBase.testPlanMatchingPatterns(query,
+        new String[] {"indexName=i_lic"},
+        new String[]{"Sort", "TopN"}
+    );
+    testBuilder()
+        .ordered()
+        .sqlQuery(query)
+        .baselineColumns("rid", "lic").baselineValues("4539", 100000000L)
+        .baselineColumns("rid", "lic").baselineValues("943", 100000001L)
+        .build()
+        .run();
+  }
+
+  @Test //negative case for no filter plan
+  public void testNoFilterOrderByNoIndexMatch() throws Exception {
+    String query = "SELECT t.`id`.`ssn` AS `ssn`, t.contact.phone as phone FROM hbase.`index_test_primary` as t " +
+        "order by t.name.fname limit 2";
+    test(defaultHavingIndexPlan);
+    PlanTestBase.testPlanMatchingPatterns(query,
+        new String[] {"(Sort|TopN)"},
+        new String[]{"indexName="}
+    );
+  }
+
+// Enable this testcase once MD-2848 is fixed.
+//  @Test
+//  public void IntersectPlanWithOneSideNoRows() throws Exception {
+//    try {
+//      String query = "SELECT t.`name`.`lname` AS `lname` FROM hbase.`index_test_primary` as t " +
+//              " where t.personal.age = 53 AND t.personal.income=111145";
+//      test(defaultHavingIndexPlan);
+//      test(preferIntersectPlans + ";" + disableFTS);
+//      PlanTestBase.testPlanMatchingPatterns(query,
+//              new String[]{"RowKeyJoin(.*[\n\r])+.*RestrictedJsonTableGroupScan(.*[\n\r])+.*HashJoin(.*[\n\r])+.*JsonTableGroupScan.*indexName=(i_age|i_income)(.*[\n\r])+.*JsonTableGroupScan.*indexName=(i_age|i_income)"},
+//              new String[]{}
+//      );
+//
+//      testNoResult(query);
+//
+//    } finally {
+//      test(defaultIntersectPlans + ";" + enableFTS);
+//    }
+//  }
+
+  //"i_cast_age_state_phone", "$CAST(personal.age@STRING),address.state,contact.phone", "name.fname",
+  @Test
+  public void testTrailingFieldIndexCovering() throws Exception {
+    String query = "SELECT t.`name`.`fname` AS `fname` FROM hbase.`index_test_primary` as t " +
+        " where cast(t.personal.age as INT)=53 AND t.contact.phone='6500005471' ";
+
+    test(defaultHavingIndexPlan);
+    PlanTestBase.testPlanMatchingPatterns(query,
+        new String[] {"indexName=i_cast_age_income_phone"},
+        new String[]{"RowKeyJoin"}
+    );
+
+    testBuilder()
+        .ordered()
+        .sqlQuery(query)
+        .baselineColumns("fname").baselineValues("KfFzK")
+        .build()
+        .run();
+  }
+
+  @Test
+  public void testIncludedFieldCovering() throws Exception {
+    String query = "SELECT t.`contact`.`phone` AS `phone` FROM hbase.`index_test_primary` as t " +
+        " where cast(t.personal.age as INT)=53 AND t.name.fname='KfFzK' ";
+
+    test(defaultHavingIndexPlan);
+    PlanTestBase.testPlanMatchingPatterns(query,
+        new String[] {"indexName=i_cast_age_income_phone"},
+        new String[]{"RowKeyJoin"}
+    );
+
+    testBuilder()
+        .ordered()
+        .sqlQuery(query)
+        .baselineColumns("phone").baselineValues("6500005471")
+        .build()
+        .run();
+  }
+
+  @Test
+  public void testWithFilterGroupBy() throws Exception {
+    String query = " select t1.driverlicense from hbase.`index_test_primary` t1" +
+            " where t1.driverlicense > 100000001 group by t1.driverlicense limit 2";
+    try {
+      test(defaultHavingIndexPlan);
+      test(disableHashAgg);
+      //no collation for hash index so Sort or TopN must have been preserved
+      PlanTestBase.testPlanMatchingPatterns(query,
+              new String[]{"indexName=i_lic", "StreamAgg"},
+              new String[]{"(Sort|TopN)"}
+      );
+
+      testBuilder()
+              .ordered()
+              .sqlQuery(query)
+              .optionSettingQueriesForTestQuery(disableHashAgg)
+              .baselineColumns("driverlicense").baselineValues(100000002L)
+              .baselineColumns("driverlicense").baselineValues(100000003L)
+              .build()
+              .run();
+    } finally {
+      test(enableHashAgg);
+    }
+  }
+
+  @Test
+  public void testNoFilterOrderByDesc() throws Exception {
+    String query = " select t1.driverlicense from hbase.`index_test_primary` t1" +
+            " order by t1.driverlicense desc limit 2";
+    test(defaultHavingIndexPlan);
+    //no collation for hash index so Sort or TopN must have been preserved
+    PlanTestBase.testPlanMatchingPatterns(query,
+            new String[] {"(Sort|TopN)"},
+            new String[]{"indexName="}
+    );
+
+    testBuilder()
+            .unOrdered()
+            .sqlQuery(query)
+            .baselineColumns("driverlicense").baselineValues(100009999L)
+            .baselineColumns("driverlicense").baselineValues(100009998L)
+            .build()
+            .run();
+  }
+
+  @Test
+  public void testNoFilterGroupBy() throws Exception {
+    String query = " select t1.driverlicense from hbase.`index_test_primary` t1" +
+            " group by t1.driverlicense limit 2";
+    try {
+      test(defaultHavingIndexPlan);
+      test(disableHashAgg);
+      //no collation for hash index so Sort or TopN must have been preserved
+      PlanTestBase.testPlanMatchingPatterns(query,
+              new String[]{"indexName=i_lic", "StreamAgg"},
+              new String[]{"(Sort|TopN)"}
+      );
+
+      testBuilder()
+              .ordered()
+              .sqlQuery(query)
+              .optionSettingQueriesForTestQuery(disableHashAgg)
+              .baselineColumns("driverlicense").baselineValues(100000000L)
+              .baselineColumns("driverlicense").baselineValues(100000001L)
+              .build()
+              .run();
+    } finally {
+      test(enableHashAgg);
+    }
+  }
+
+  @Test
+  public void testNoFilterGroupByCoveringPlan() throws Exception {
+    String query = "SELECT t.`id`.`ssn` AS `ssn`, max(t.contact.phone) as phone FROM hbase.`index_test_primary` as t " +
+            "group by t.id.ssn limit 2";
+    try {
+      test(defaultHavingIndexPlan);
+      test(disableHashAgg);
+      PlanTestBase.testPlanMatchingPatterns(query,
+              new String[]{"indexName=i_ssn", "StreamAgg"},
+              new String[]{"Sort", "TopN", "RowKeyJoin"}
+      );
+      testBuilder()
+              .ordered()
+              .sqlQuery(query)
+              .optionSettingQueriesForTestQuery(disableHashAgg)
+              .baselineColumns("ssn", "phone").baselineValues("100000000", "6500008069")
+              .baselineColumns("ssn", "phone").baselineValues("100000001", "6500001411")
+              .build()
+              .run();
+    } finally {
+      test(enableHashAgg);
+    }
+  }
+
+  @Test
+  public void testNoFilterGroupByCast() throws Exception {
+    String query = "SELECT CAST(t.id.ssn as INT) AS `ssn`, max(t.contact.phone) as phone FROM hbase.`index_test_primary` as t " +
+            "group by CAST(t.id.ssn as INT) limit 2";
+    try {
+      test(defaultHavingIndexPlan);
+      test(disableHashAgg);
+      PlanTestBase.testPlanMatchingPatterns(query,
+              new String[]{"indexName=i_cast_int_ssn", "StreamAgg"},
+              new String[]{"TopN", "Sort", "RowKeyJoin"}
+      );
+      testBuilder()
+              .ordered()
+              .sqlQuery(query)
+              .optionSettingQueriesForTestQuery(disableHashAgg)
+              .baselineColumns("ssn", "phone").baselineValues(100000000, "6500008069")
+              .baselineColumns("ssn", "phone").baselineValues(100000001, "6500001411")
+              .build()
+              .run();
+    } finally {
+      test(enableHashAgg);
+    }
+  }
+
+  @Test
+  public void testNoFilterGroupByHashIndex() throws Exception {
+    String query = "SELECT cast(t.activity.irs.firstlogin as timestamp) AS `firstlogin`, max(t.id.ssn) as ssn FROM hbase.`index_test_primary` as t " +
+            "group by cast(t.activity.irs.firstlogin as timestamp) limit 2";
+    try {
+      test(defaultHavingIndexPlan);
+      test(disableHashAgg);
+      //no collation for hash index so Sort or TopN must have been preserved
+      PlanTestBase.testPlanMatchingPatterns(query,
+              new String[]{"(Sort|TopN)", "StreamAgg"},
+              new String[]{"indexName="}
+      );
+      DateTime date1 = DateTimeFormat.forPattern("yyyy-MM-dd HH:mm:ss")
+              .parseDateTime("2010-01-21 00:12:24");
+
+      DateTime date2 = DateTimeFormat.forPattern("yyyy-MM-dd HH:mm:ss")
+              .parseDateTime("2010-01-21 00:24:48");
+      testBuilder()
+              .unOrdered()
+              .sqlQuery(query)
+              .optionSettingQueriesForTestQuery(disableHashAgg)
+              .baselineColumns("firstlogin", "ssn").baselineValues(date1, "100006852")
+              .baselineColumns("firstlogin", "ssn").baselineValues(date2, "100003660")
+              .build()
+              .run();
+    } finally {
+      test(enableHashAgg);
+    }
+  }
+
+  @Test
+  public void testNoFilterGroupBySimpleField() throws Exception {
+    String query = "SELECT max(t.reverseid) as rid, t.driverlicense as lic FROM hbase.`index_test_primary` as t " +
+            "group by t.driverlicense limit 2";
+    try {
+      test(defaultHavingIndexPlan);
+      test(disableHashAgg);
+      PlanTestBase.testPlanMatchingPatterns(query,
+              new String[]{"indexName=i_lic", "StreamAgg"},
+              new String[]{"Sort", "TopN"}
+      );
+      testBuilder()
+              .ordered()
+              .sqlQuery(query)
+              .optionSettingQueriesForTestQuery(disableHashAgg)
+              .baselineColumns("rid", "lic").baselineValues("4539", 100000000L)
+              .baselineColumns("rid", "lic").baselineValues("943", 100000001L)
+              .build()
+              .run();
+    } finally {
+      test(enableHashAgg);
+    }
+  }
+
+  @Test //negative case for no filter plan
+  public void testNoFilterGroupByNoIndexMatch() throws Exception {
+    String query = "SELECT max(t.`id`.`ssn`) AS `ssn`, max(t.contact.phone) as phone FROM hbase.`index_test_primary` as t " +
+            "group by t.name.fname limit 2";
+    try {
+      test(defaultHavingIndexPlan);
+      test(disableHashAgg);
+      PlanTestBase.testPlanMatchingPatterns(query,
+              new String[]{"(Sort|TopN)", "StreamAgg"},
+              new String[]{"indexName="}
+      );
+    } finally {
+      test(enableHashAgg);
+    }
+  }
+
+  @Test
+  public void testNoFilterGroupBySimpleFieldParallel() throws Exception {
+    String query = "SELECT max(t.reverseid) as rid, t.driverlicense as lic FROM hbase.`index_test_primary` as t " +
+        "group by t.driverlicense order by t.driverlicense limit 2";
+    try {
+      test(defaultHavingIndexPlan);
+      test(disableHashAgg);
+      test(sliceTargetSmall);
+      PlanTestBase.testPlanMatchingPatterns(query,
+              new String[]{"indexName=i_lic", "StreamAgg", "HashToMergeExchange"},
+              new String[]{"Sort", "TopN"}
+      );
+      testBuilder()
+              .unOrdered()
+              .sqlQuery(query)
+              .optionSettingQueriesForTestQuery(defaultHavingIndexPlan)
+              .optionSettingQueriesForTestQuery(disableHashAgg)
+              .optionSettingQueriesForTestQuery(sliceTargetSmall)
+              .baselineColumns("rid", "lic").baselineValues("4539", 100000000L)
+              .baselineColumns("rid", "lic").baselineValues("943", 100000001L)
+              .build()
+              .run();
+    } finally {
+      test(enableHashAgg);
+      test(sliceTargetDefault);
+    }
+  }
+
+  @Test
+  public void testLimitPushdownCoveringPlan() throws Exception {
+    String query = "SELECT t.`name`.`fname` AS `fname` FROM hbase.`index_test_primary` as t " +
+        " where t.personal.age = 53 limit 3";
+    try {
+      test(defaultHavingIndexPlan + ";" + disableFTS + ";");
+      PlanTestBase.testPlanWithAttributesMatchingPatterns(query,
+              new String[]{".*JsonTableGroupScan.*indexName=i_age_with_fname.*rowcount = 3.0"},
+              new String[]{}
+      );
+    } finally {
+      test(enableFTS);
+    }
+  }
+
+  @Test
+  public void testLimitPushdownOrderByCoveringPlan() throws Exception {
+    String query = "SELECT t.`name`.`fname` AS `fname` FROM hbase.`index_test_primary` as t " +
+        " where t.personal.age = 53 order by t.personal.age limit 3";
+    try {
+      test(defaultHavingIndexPlan + ";" + disableFTS + ";");
+      PlanTestBase.testPlanWithAttributesMatchingPatterns(query,
+              new String[]{".*JsonTableGroupScan.*indexName=i_age_with_fname.*rowcount = 3.0"},
+              new String[]{}
+      );
+    } finally {
+      test(enableFTS);
+    }
+  }
+
+  @Test
+  public void testLimitPushdownNonCoveringPlan() throws Exception {
+    String query = "SELECT t.`name`.`lname` AS `lname` FROM hbase.`index_test_primary` as t " +
+        " where t.personal.age = 53 limit 7";
+    try {
+      test(defaultHavingIndexPlan+";"+disableFTS+";");
+      PlanTestBase.testPlanWithAttributesMatchingPatterns(query,
+              new String[]{"RowKeyJoin", ".*RestrictedJsonTableGroupScan.*tableName=.*index_test_primary.*rowcount = 7.0"},
+              new String[]{}
+      );
+    } finally {
+      test(enableFTS);
+    }
+  }
+
+  @Test
+  public void testLimitPushdownOrderByNonCoveringPlan() throws Exception {
+    // Limit pushdown should NOT happen past rowkey join when ordering is required
+    String query = "SELECT t.`name`.`lname` AS `lname` FROM hbase.`index_test_primary` as t " +
+        " where t.personal.age = 53 order by t.personal.age limit 7";
+    try {
+      test(defaultHavingIndexPlan + ";" + disableFTS + ";" + sliceTargetSmall + ";");
+      PlanTestBase.testPlanWithAttributesMatchingPatterns(query,
+              new String[]{"RowKeyJoin", ".*RestrictedJsonTableGroupScan.*"},
+              new String[]{".*tableName=.*index_test_primary.*rowcount = 7.*"}
+      );
+    } finally {
+      test(enableFTS);
+    }
+  }
+
+  @Test
+  public void testLimit0Pushdown() throws Exception {
+    // Limit pushdown should NOT happen past project with CONVERT_FROMJSON
+    String query = "select convert_from(convert_to(t.`name`.`lname`, 'JSON'), 'JSON') " +
+        "from hbase.`index_test_primary` as t limit 0";
+    try {
+      test(defaultHavingIndexPlan + ";");
+      PlanTestBase.testPlanWithAttributesMatchingPatterns(query,
+          new String[]{"Limit(.*[\n\r])+.*Project.*CONVERT_FROMJSON(.*[\n\r])+.*Scan"},
+          new String[]{}
+      );
+    } finally {
+    }
+  }
+
+  @Test
+  public void testRemovalOfReduntantHashToMergeExchange() throws Exception {
+    String query = "SELECT t.driverlicense as lic FROM hbase.`index_test_primary` as t " +
+            "order by t.driverlicense limit 2";
+    try {
+      test(defaultHavingIndexPlan);
+      test(sliceTargetSmall);
+      PlanTestBase.testPlanMatchingPatterns(query,
+              new String[]{"indexName=i_lic"},
+              new String[]{"HashToMergeExchange", "Sort", "TopN"});
+      testBuilder()
+              .ordered()
+              .sqlQuery(query)
+              .baselineColumns("lic").baselineValues(100000000L)
+              .baselineColumns("lic").baselineValues(100000001L)
+              .build()
+              .run();
+    } finally {
+      test(sliceTargetDefault);
+    }
+  }
+
+  @Test
+  public void testMultiPhaseAgg() throws Exception {
+    String query = "select count(t.reverseid) from hbase.`index_test_primary` as t " +
+            "group by t.driverlicense order by t.driverlicense";
+    try {
+      test(defaultHavingIndexPlan);
+      test(sliceTargetSmall);
+      PlanTestBase.testPlanMatchingPatterns(query,
+              new String[]{"indexName=i_lic", "HashToMergeExchange", "StreamAgg", "StreamAgg"},
+              new String[]{"Sort", "TopN"});
+    } finally {
+      test(sliceTargetDefault);
+    }
+  }
+
+  @Test
+  public void testHangForSimpleDistinct() throws Exception {
+    String query = "select distinct t.driverlicense from hbase.`index_test_primary` as t order by t.driverlicense limit 1";
+
+    try {
+      test(sliceTargetSmall);
+      testBuilder()
+              .ordered()
+              .sqlQuery(query)
+              .baselineColumns("driverlicense").baselineValues(100000000L)
+              .build()
+              .run();
+    } finally {
+      test(sliceTargetDefault);
+    }
+  }
+}
diff --git a/contrib/format-maprdb/src/test/java/com/mapr/drill/maprdb/tests/index/LargeTableGen.java b/contrib/format-maprdb/src/test/java/com/mapr/drill/maprdb/tests/index/LargeTableGen.java
new file mode 100644
index 0000000..bc857d1
--- /dev/null
+++ b/contrib/format-maprdb/src/test/java/com/mapr/drill/maprdb/tests/index/LargeTableGen.java
@@ -0,0 +1,176 @@
+/*
+ * 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 com.mapr.drill.maprdb.tests.index;
+
+import static com.mapr.drill.maprdb.tests.MaprDBTestsSuite.INDEX_FLUSH_TIMEOUT;
+
+import java.io.InputStream;
+import java.io.StringBufferInputStream;
+
+import org.apache.hadoop.fs.Path;
+import org.ojai.DocumentStream;
+import org.ojai.json.Json;
+
+import com.mapr.db.Admin;
+import com.mapr.db.Table;
+import com.mapr.db.TableDescriptor;
+import com.mapr.db.impl.MapRDBImpl;
+import com.mapr.db.impl.TableDescriptorImpl;
+import com.mapr.db.tests.utils.DBTests;
+import com.mapr.fs.utils.ssh.TestCluster;
+
+/**
+ * This class is to generate a MapR json table of this schema:
+ * {
+ *   "address" : {
+ *      "city":"wtj",
+ *      "state":"ho"
+ *   }
+ *   "contact" : {
+ *      "email":"VcFahjRfM@gmail.com",
+ *      "phone":"6500005583"
+ *   }
+ *   "id" : {
+ *      "ssn":"100005461"
+ *   }
+ *   "name" : {
+ *      "fname":"VcFahj",
+ *      "lname":"RfM"
+ *   }
+ * }
+ *
+ */
+public class LargeTableGen extends LargeTableGenBase {
+
+  static final int SPLIT_SIZE = 5000;
+  private Admin admin;
+
+  public LargeTableGen(Admin dbadmin) {
+    admin = dbadmin;
+  }
+
+  Table createOrGetTable(String tableName, int recordNum) {
+    if (admin.tableExists(tableName)) {
+      return MapRDBImpl.getTable(tableName);
+      //admin.deleteTable(tableName);
+    }
+    else {
+      TableDescriptor desc = new TableDescriptorImpl(new Path(tableName));
+
+      int splits = (recordNum / SPLIT_SIZE) - (((recordNum % SPLIT_SIZE) > 1)? 0 : 1);
+
+      String[] splitsStr = new String[splits];
+      StringBuilder strBuilder = new StringBuilder("Splits:");
+      for(int i=0; i<splits; ++i) {
+        splitsStr[i] = String.format("%d", (i+1)*SPLIT_SIZE);
+        strBuilder.append(splitsStr[i] + ", ");
+      }
+      System.out.print(strBuilder.toString());
+
+      return admin.createTable(desc, splitsStr);
+    }
+  }
+
+  private void createIndex(Table table, String[] indexDef) throws Exception {
+    if(indexDef == null) {
+      //don't create index here. indexes may have been created
+      return;
+    }
+    for(int i=0; i<indexDef.length / 3; ++i) {
+      String indexCmd = String.format("maprcli table index add"
+          + " -path " + table.getPath()
+          + " -index %s"
+          + " -indexedfields '%s'"
+          + ((indexDef[3 * i + 2].length()==0)?"":" -includedfields '%s'")
+          + ((indexDef[3 * i].startsWith("hash"))? " -hashed true" : ""),
+          indexDefInCommand(indexDef[3 * i]), //index name
+          indexDefInCommand(indexDef[3 * i + 1]), //indexedfields
+          indexDefInCommand(indexDef[3 * i + 2])); //includedfields
+      System.out.println(indexCmd);
+
+      TestCluster.runCommand(indexCmd);
+      DBTests.admin().getTableIndexes(table.getPath(), true);
+    }
+  }
+
+  private String indexDefInCommand(String def) {
+    String[] splitted = def.split(",");
+    StringBuffer ret = new StringBuffer();
+    for(String field: splitted) {
+      if(ret.length() == 0) {
+        ret.append(field);
+      }
+      else {
+        ret.append(",").append(field);
+      }
+    }
+    return ret.toString();
+  }
+  public void generateTableWithIndex(String tablePath, int recordNumber, String[] indexDef) throws Exception {
+    // create index
+
+    initRandVector(recordNumber);
+    initDictionary();
+    DBTests.setTableStatsSendInterval(1);
+
+    if (admin.tableExists(tablePath)) {
+      //admin.deleteTable(tablePath);
+    }
+
+    //create Json String
+    int batch, i;
+    int BATCH_SIZE=2000;
+    try (Table table = createOrGetTable(tablePath, recordNumber)) {
+      //create index
+      createIndex(table, indexDef);
+      for (batch = 0; batch < recordNumber; batch += BATCH_SIZE) {
+        int batchStop = Math.min(recordNumber, batch + BATCH_SIZE);
+        StringBuffer strBuf = new StringBuffer();
+        for (i = batch; i < batchStop; ++i) {
+
+          strBuf.append(String.format("{\"rowid\": \"%d\", \"reverseid\": \"%d\", \"id\": {\"ssn\": \"%s\"}, \"contact\": {\"phone\": \"%s\", \"email\": \"%s\"}," +
+                  "\"address\": {\"city\": \"%s\", \"state\": \"%s\"}, \"name\": { \"fname\": \"%s\", \"lname\": \"%s\" }," +
+                  "\"personal\": {\"age\" : %s, \"income\": %s, \"birthdate\": {\"$dateDay\": \"%s\"} }," +
+                  "\"activity\": {\"irs\" : { \"firstlogin\":  \"%s\" } }," +
+                  "\"driverlicense\":{\"$numberLong\": %s} } \n",
+              i + 1, recordNumber - i , getSSN(i), getPhone(i), getEmail(i),
+              getAddress(i)[2], getAddress(i)[1], getFirstName(i), getLastName(i),
+              getAge(i), getIncome(i), getBirthdate(i),
+              getFirstLogin(i),
+              getSSN(i)));
+        }
+        try (InputStream in = new StringBufferInputStream(strBuf.toString());
+             DocumentStream stream = Json.newDocumentStream(in)) {
+          //write by individual document
+          //for (Document document : stream) {
+          //  table.insert(document, "rowid");
+          //}
+          try {
+            table.insert(stream, "rowid"); //insert a batch  of document in stream
+          }catch(Exception e) {
+            System.out.println(stream.toString());
+            throw e;
+          }
+        }
+      }
+      table.flush();
+      DBTests.waitForIndexFlush(table.getPath(), INDEX_FLUSH_TIMEOUT);
+      Thread.sleep(200000);
+    }
+  }
+}
diff --git a/contrib/format-maprdb/src/test/java/com/mapr/drill/maprdb/tests/index/LargeTableGenBase.java b/contrib/format-maprdb/src/test/java/com/mapr/drill/maprdb/tests/index/LargeTableGenBase.java
new file mode 100644
index 0000000..917f42a
--- /dev/null
+++ b/contrib/format-maprdb/src/test/java/com/mapr/drill/maprdb/tests/index/LargeTableGenBase.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 com.mapr.drill.maprdb.tests.index;
+
+import org.apache.commons.lang3.RandomStringUtils;
+
+import java.util.ArrayList;
+import java.util.Iterator;
+import java.util.LinkedHashSet;
+import java.util.List;
+import java.util.Random;
+import java.util.Set;
+
+public class LargeTableGenBase {
+
+  private boolean dict_ready = false;
+
+  protected List<String> firstnames;
+  protected List<String> lastnames;
+  protected List<String[]> cities;
+  protected int[] randomized;
+
+  protected synchronized void  initDictionary() {
+    initDictionaryWithRand();
+  }
+
+  protected void initDictionaryWithRand() {
+    {
+      firstnames = new ArrayList<>();
+      lastnames = new ArrayList<>();
+      cities = new ArrayList<>();
+      List<String> states = new ArrayList<>();
+
+      int fnNum = 2000; //2k
+      int lnNum = 200000;//200k
+      int cityNum = 10000;//10k
+      int stateNum = 50;
+      Random rand = new Random(2017);
+      int i;
+      try {
+        Set<String> strSet = new LinkedHashSet<>();
+        while(strSet.size() < stateNum) {
+          strSet.add(RandomStringUtils.random(2, 0, 0, true, false, null, rand));
+        }
+        states.addAll(strSet);
+
+        strSet = new LinkedHashSet<>();
+        while(strSet.size() < cityNum) {
+          int len = 3 + strSet.size() % 6;
+          strSet.add(RandomStringUtils.random(len, 0, 0, true, false, null, rand));
+        }
+
+        Iterator<String> it = strSet.iterator();
+        for(i=0; i<cityNum; ++i) {
+          cities.add(new String[]{"10000", states.get(i%stateNum),  it.next()});
+        }
+
+        strSet = new LinkedHashSet<>();
+        while(strSet.size() < fnNum) {
+          int len = 3 + strSet.size() % 6;
+          strSet.add(RandomStringUtils.random(len, 0, 0, true, false, null, rand));
+        }
+        firstnames.addAll(strSet);
+
+        strSet = new LinkedHashSet<>();
+        while(strSet.size() < lnNum) {
+          int len = 3 + strSet.size() % 6;
+          strSet.add(RandomStringUtils.random(len, 0, 0, true, false, null, rand));
+        }
+        lastnames.addAll(strSet);
+      }
+      catch(Exception e) {
+        System.out.println("init data got exception");
+        e.printStackTrace();
+      }
+      dict_ready = true;
+    }
+  }
+
+  protected  String getFirstName(int i) {
+    return firstnames.get((randomized[ i%randomized.length ] + i )% firstnames.size());
+  }
+
+  protected String getLastName(int i) {
+    return lastnames.get((randomized[ (2*i + randomized[i%randomized.length])% randomized.length]) % lastnames.size());
+  }
+
+  protected String[] getAddress(int i) {
+    return cities.get((randomized[(i+ randomized[i%randomized.length])%randomized.length]) % cities.size());
+  }
+
+  protected String getSSN(int i){
+    return String.format("%d", 1000*1000*100 + randomized[ i % randomized.length]);
+  }
+
+  protected String getPhone(int i) {
+    //80% phones are unique,
+    return String.format("%d", 6500*1000*1000L + randomized[ (randomized.length - i) %((int) (randomized.length * 0.8)) ]);
+  }
+
+  protected String getEmail(int i){
+    return getFirstName(i) + getLastName(i) + "@" + "gmail.com";
+  }
+
+  protected String getAge(int i) {
+    return String.format("%d",randomized[i%randomized.length] % 60 + 10);
+  }
+
+  protected String getIncome(int i) {//unit should be $10k
+    return String.format("%d",randomized[i%randomized.length] % 47 + 1);
+  }
+
+  //date yyyy-mm-dd
+  protected String getBirthdate(int i) {
+    int thisseed = randomized[i%randomized.length];
+    return String.format("%d-%02d-%02d",
+        2016 - (thisseed % 60 + 10), thisseed % 12 + 1, (thisseed * 31) % 28 + 1 );
+  }
+
+  //timestamp, yyyy-mm-dd HH:mm:ss
+  protected String getFirstLogin(int i) {
+    int thisseed = randomized[i%randomized.length];
+    int nextseed = randomized[(i+1)%randomized.length];
+    return String.format("%d-%02d-%02d %02d:%02d:%02d.0",
+        2016 - (thisseed % 7), (thisseed * 31) % 12 + 1, thisseed % 28 + 1, nextseed % 24, nextseed % 60, (nextseed * 47) % 60 );
+  }
+
+
+  protected String getField(String field, int i) {
+    if(field.equals("ssn")) {
+      return getSSN(i);
+    }
+    else if (field.equals("phone")) {
+      return getPhone(i);
+    }
+    else if(field.equals("email")) {
+      return getEmail(i);
+    }
+    else if(field.equals("city")) {
+      return getAddress(i)[1];
+    }
+    else if(field.equals("state")) {
+      return getAddress(i)[0];
+    }
+    else if(field.equals("fname")) {
+      return getFirstName(i);
+    }
+    else if(field.equals("lname")) {
+      return getLastName(i);
+    }
+    return "";
+  }
+
+
+  protected void initRandVector(int recordNumber) {
+    int i;
+    Random rand = new Random(2016);
+    randomized = new int[recordNumber];
+    for(i = 0; i<recordNumber; ++i) {
+      randomized[i] = i;
+    }
+    for (i=0; i<recordNumber; ++i) {
+      int idx1 =  rand.nextInt(recordNumber);
+      int idx2 = rand.nextInt(recordNumber);
+      int temp = randomized[idx1];
+      randomized[idx1] = randomized[idx2];
+      randomized[idx2] = temp;
+    }
+  }
+
+}
diff --git a/contrib/format-maprdb/src/test/java/com/mapr/drill/maprdb/tests/index/StatisticsTest.java b/contrib/format-maprdb/src/test/java/com/mapr/drill/maprdb/tests/index/StatisticsTest.java
new file mode 100644
index 0000000..36e25ab
--- /dev/null
+++ b/contrib/format-maprdb/src/test/java/com/mapr/drill/maprdb/tests/index/StatisticsTest.java
@@ -0,0 +1,115 @@
+/*
+ * 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 com.mapr.drill.maprdb.tests.index;
+
+import com.google.common.collect.Lists;
+import com.mapr.db.Admin;
+import com.mapr.drill.maprdb.tests.MaprDBTestsSuite;
+import com.mapr.drill.maprdb.tests.json.BaseJsonTest;
+import com.mapr.tests.annotations.ClusterTest;
+import org.apache.drill.PlanTestBase;
+import org.apache.hadoop.hbase.TableName;
+import org.junit.AfterClass;
+import org.junit.BeforeClass;
+import org.junit.Ignore;
+import org.junit.Test;
+import org.junit.experimental.categories.Category;
+
+import java.util.List;
+
+@Category(ClusterTest.class)
+public class StatisticsTest extends IndexPlanTest {
+  /**
+   *  A sample row of this 10K table:
+   ------------------+-----------------------------+--------+
+   | 1012  | {"city":"pfrrs","state":"pc"}  | {"email":"KfFzKUZwNk@gmail.com","phone":"6500005471"}  |
+   {"ssn":"100007423"}  | {"fname":"KfFzK","lname":"UZwNk"}  | {"age":53.0,"income":45.0}  | 1012   |
+   *
+   * This test suite generate random content to fill all the rows, since the random function always start from
+   * the same seed for different runs, when the row count is not changed, the data in table will always be the same,
+   * thus the query result could be predicted and verified.
+   */
+
+  @Test
+  @Ignore("Currently untested; re-enable after stats/costing integration complete")
+  public void testFilters() throws Exception {
+    String query;
+    String explain = "explain plan including all attributes for ";
+
+    // Top-level ANDs - Leading columns (personal.age), (address.state)
+    query = "select * from hbase.`index_test_primary` t "
+        + " where (t.personal.age < 30 or t.personal.age > 100)"
+        + " and (t.address.state = 'mo' or t.address.state = 'ca')";
+    PlanTestBase.testPlanMatchingPatterns(explain+query,
+        new String[] {".*JsonTableGroupScan.*tableName=.*index_test_primary.*rows=10000"},
+        new String[] {}
+    );
+
+    // Top-level ORs - Cannot split top-level ORs so use defaults
+    query = "select * from hbase.`index_test_primary` t "
+        + " where (t.personal.age > 30 and t.personal.age < 100)"
+        + " or (t.address.state = 'mo')";
+    PlanTestBase.testPlanMatchingPatterns(explain+query,
+        new String[] {".*JsonTableGroupScan.*tableName=.*index_test_primary.*rows=10000"},
+        new String[] {}
+    );
+
+    // ANDed condition - Leading index column(personal.age) and non-leading column(address.city)
+    query = "select * from hbase.`index_test_primary` t "
+        + " where (t.personal.age < 30 or t.personal.age > 100)"
+        + " and `address.city` = 'sf'";
+    PlanTestBase.testPlanMatchingPatterns(explain+query,
+        new String[] {".*JsonTableGroupScan.*tableName=.*index_test_primary.*rows=10000"},
+        new String[] {}
+    );
+
+    // ANDed condition - Leading index columns (address.state) and (address.city)
+    query = "select * from hbase.`index_test_primary` t "
+        + " where (`address.state` = 'mo' or `address.state` = 'ca') " // Leading index column
+        + " and `address.city` = 'sf'";                                // Non leading index column
+    PlanTestBase.testPlanMatchingPatterns(explain+query,
+        new String[] {".*JsonTableGroupScan.*tableName=.*index_test_primary.*rows=10000"},
+        new String[] {}
+    );
+
+    // ANDed condition - Leading index columns (address.state) and non-index column (name.fname)
+    query = "select * from hbase.`index_test_primary` t "
+        + " where (`address.state` = 'mo' or `address.state` = 'ca') " // Leading index column
+        + " and `name.fname` = 'VcFahj'";                              // Non index column
+    PlanTestBase.testPlanMatchingPatterns(explain+query,
+        new String[] {".*JsonTableGroupScan.*tableName=.*index_test_primary.*rows=10000"},
+        new String[] {}
+    );
+
+    // Simple condition - LIKE predicate
+    query = "select t._id as rowid from hbase.`index_test_primary` as t "
+        + "where t.driverlicense like '100007423%'";
+    PlanTestBase.testPlanMatchingPatterns(explain+query,
+        new String[] {".*JsonTableGroupScan.*tableName=.*index_test_primary.*rows=10000"},
+        new String[] {}
+    );
+
+    // Simple condition - LIKE predicate with ESCAPE clause
+    query = "select t._id as rowid from hbase.`index_test_primary` as t "
+        + "where t.driverlicense like '100007423%' ESCAPE '/'";
+    PlanTestBase.testPlanMatchingPatterns(explain+query,
+        new String[] {".*JsonTableGroupScan.*tableName=.*index_test_primary.*rows=10000"},
+        new String[] {}
+    );
+  }
+}
diff --git a/contrib/format-maprdb/src/test/java/com/mapr/drill/maprdb/tests/index/TableIndexCmd.java b/contrib/format-maprdb/src/test/java/com/mapr/drill/maprdb/tests/index/TableIndexCmd.java
new file mode 100644
index 0000000..a501f8f
--- /dev/null
+++ b/contrib/format-maprdb/src/test/java/com/mapr/drill/maprdb/tests/index/TableIndexCmd.java
@@ -0,0 +1,127 @@
+/*
+ * 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 com.mapr.drill.maprdb.tests.index;
+
+
+import com.mapr.db.Admin;
+import com.mapr.db.MapRDB;
+import org.apache.drill.exec.util.GuavaPatcher;
+
+import java.util.HashMap;
+import java.util.Map;
+
+/**
+* Copy classes to a MapR cluster node, then run a command like this:
+* java -classpath /tmp/drill-cmd-1.9.0-SNAPSHOT.jar:/opt/mapr/drill/drill-1.9.0/jars/*:/opt/mapr/drill/drill-1.9.0/jars/3rdparty/*:/opt/mapr/drill/drill-1.9.0/jars/ext/*
+*                 org.apache.drill.hbase.index.TableIndexGen -host 10.10.88.128 -port 5181 [-table pop3] [-size 1000000]
+*/
+
+class TestBigTable {
+
+  Admin admin;
+  boolean initialized = false;
+
+  LargeTableGen gen;
+
+  /*
+    "hbase.zookeeper.quorum": "10.10.88.128",
+    "hbase.zookeeper.property.clientPort": "5181"
+   */
+  void init(String host, String port) {
+    try {
+      admin = MapRDB.newAdmin();
+      initialized = true;
+      gen = new LargeTableGen(admin);
+    } catch (Exception e) {
+      System.out.println("Connection to HBase threw" + e.getMessage());
+    }
+  }
+}
+
+
+public class TableIndexCmd {
+
+  public static Map<String,String> parseParameter(String[] params) {
+    HashMap<String,String> retParams = new HashMap<String, String>();
+    for (int i=0; i<params.length; ++i) {
+      if (params[i].startsWith("-") && i<params.length - 1) {
+        String paramName = params[i].replaceFirst("-*", "");
+        retParams.put(paramName, params[i+1]);
+        ++i;
+      }
+    }
+    return retParams;
+  }
+
+  public static void pressEnterKeyToContinue()
+  {
+    System.out.println("Press any key to continue...");
+    try
+    {
+      System.in.read();
+    }
+    catch(Exception e)
+    {}
+  }
+
+
+  public static void main(String[] args) {
+    GuavaPatcher.patch();
+
+    String inHost = new String("localhost");
+    String inPort = new String("5181");
+    String inTable = new String("/tmp/population");
+    String dictPath = "hbase";
+    boolean waitKeyPress = true;
+    long inSize = 10000;
+    Map<String, String> params = parseParameter(args);
+    if(args.length >= 2) {
+      if(params.get("host") != null) {
+        inHost = params.get("host");
+      }
+      if(params.get("port") != null) {
+        inPort = params.get("port");
+      }
+      if(params.get("table") != null) {
+        inTable = params.get("table");
+      }
+      if(params.get("size") != null) {
+        inSize = Long.parseLong(params.get("size"));
+      }
+      if(params.get("dict") != null) {
+        dictPath = params.get("dict");
+      }
+      if(params.get("wait") != null) {
+        String answer = params.get("wait");
+        waitKeyPress = answer.startsWith("y") || answer.startsWith("t")? true : false;
+      }
+    }
+    if(waitKeyPress == true) {
+      pressEnterKeyToContinue();
+    }
+    try {
+      TestBigTable tbt = new TestBigTable();
+      tbt.init(inHost, inPort);
+      tbt.gen.generateTableWithIndex(inTable, (int)(inSize & 0xFFFFFFFFL), null);
+    }
+    catch(Exception e) {
+      System.out.println("generate big table got exception:" + e.getMessage());
+      e.printStackTrace();
+    }
+  }
+}
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/common/OrderedRel.java b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/common/OrderedRel.java
new file mode 100644
index 0000000..5f4da74
--- /dev/null
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/common/OrderedRel.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;
+
+import org.apache.calcite.rel.RelCollation;
+import org.apache.calcite.rex.RexNode;
+
+/**
+ * Class implementing OrderedPrel interface guarantees to provide ordered
+ * output on certain columns. TopNPrel and SortPrel base classes which implement
+ * this interface.
+ */
+public interface OrderedRel extends DrillRelNode {
+
+  /**
+   * A method to return ordering columns of the result.
+   * @return Collation order of the output.
+   */
+  RelCollation getCollation();
+
+  /**
+   * Offset value represented in RexNode.
+   * @return offset.
+   */
+  RexNode getOffset();
+
+  /**
+   * Fetch value represented in RexNode.
+   * @return fetch
+   */
+  RexNode getFetch();
+
+  /**
+   * A method to return if this relational node can be dropped during optimization process.
+   * @return true if this node can be dropped, false otherwise.
+   */
+  boolean canBeDropped();
+}
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/index/IndexCallContext.java b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/index/IndexCallContext.java
index 65788cb..45251c6 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/index/IndexCallContext.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/index/IndexCallContext.java
@@ -20,7 +20,6 @@ package org.apache.drill.exec.planner.index;
 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;
@@ -28,6 +27,7 @@ import org.apache.drill.exec.physical.base.DbGroupScan;
 import org.apache.drill.exec.planner.physical.DrillDistributionTrait.DistributionField;
 import org.apache.drill.exec.planner.common.DrillScanRelBase;
 import org.apache.drill.exec.planner.common.DrillProjectRelBase;
+import org.apache.drill.exec.planner.common.OrderedRel;
 import java.util.List;
 import java.util.Set;
 
@@ -66,7 +66,7 @@ public interface IndexCallContext {
 
   RexNode getOrigCondition();
 
-  Sort getSort();
+  OrderedRel getSort();
 
   void createSortExprs();
 
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
index 27198bb..3a6ea83 100644
--- 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
@@ -20,7 +20,6 @@ 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;
@@ -31,6 +30,7 @@ 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.common.OrderedRel;
 import org.apache.drill.exec.planner.physical.DrillDistributionTrait.DistributionField;
 import org.apache.calcite.rel.RelNode;
 
@@ -164,7 +164,7 @@ public class IndexLogicalPlanCallContext implements IndexCallContext {
     return origPushedCondition;
   }
 
-  public Sort getSort() {
+  public OrderedRel getSort() {
     return sort;
   }
 
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
index 9c7b651..91ff02c 100644
--- 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
@@ -21,7 +21,6 @@ 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;
@@ -29,10 +28,10 @@ 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.common.OrderedRel;
 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;
@@ -42,8 +41,9 @@ import java.util.List;
 import java.util.Set;
 
 public class IndexPhysicalPlanCallContext implements IndexCallContext {
+
   final public RelOptRuleCall call;
-  final public SortPrel sort;
+  final public OrderedRel sort;
   final public ProjectPrel upperProject;
   final public FilterPrel filter;
   final public ProjectPrel lowerProject;
@@ -67,7 +67,7 @@ public class IndexPhysicalPlanCallContext implements IndexCallContext {
   }
 
   public IndexPhysicalPlanCallContext(RelOptRuleCall call,
-                                      SortPrel sort,
+                                      OrderedRel sort,
                                       ProjectPrel capProject,
                                       FilterPrel filter,
                                       ProjectPrel project,
@@ -83,7 +83,7 @@ public class IndexPhysicalPlanCallContext implements IndexCallContext {
   }
 
   public IndexPhysicalPlanCallContext(RelOptRuleCall call,
-                                      SortPrel sort,
+                                      OrderedRel sort,
                                       ProjectPrel project,
                                       ScanPrel scan, ExchangePrel exch) {
     this.call = call;
@@ -171,7 +171,7 @@ public class IndexPhysicalPlanCallContext implements IndexCallContext {
     return origPushedCondition;
   }
 
-  public Sort getSort() {
+  public OrderedRel getSort() {
     return sort;
   }
 
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
index 666e282..cdad63a 100644
--- 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
@@ -38,6 +38,7 @@ 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.rex.RexLiteral;
 import org.apache.calcite.sql.SqlKind;
 import org.apache.drill.common.expression.FieldReference;
 import org.apache.drill.common.expression.LogicalExpression;
@@ -56,6 +57,7 @@ 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.drill.exec.planner.common.OrderedRel;
 import org.apache.calcite.rel.type.RelDataType;
 import org.apache.calcite.rel.type.RelDataTypeField;
 import org.apache.calcite.rex.RexInputRef;
@@ -346,6 +348,21 @@ public class IndexPlanUtils {
     return proj.getProjects();
   }
 
+  public static boolean generateLimit(OrderedRel sort) {
+    RexNode fetchNode = sort.getFetch();
+    int fetchValue = (fetchNode == null) ? -1 : RexLiteral.intValue(fetchNode);
+    return fetchValue >=0;
+  }
+
+  public static RexNode getOffset(OrderedRel sort) {
+    return sort.getOffset();
+  }
+
+  public static RexNode getFetch(OrderedRel sort) {
+    return sort.getFetch();
+  }
+
+
   /**
    * generate logical expressions for sort rexNodes in SortRel, the result is store to IndexPlanCallContext
    * @param indexContext
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
index 36ff61f..456542b 100644
--- 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
@@ -30,7 +30,6 @@ 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;
@@ -43,16 +42,19 @@ 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.OrderedRel;
 import org.apache.drill.exec.planner.common.DrillScanRelBase;
+import org.apache.drill.exec.planner.physical.SubsetTransformer;
 import org.apache.drill.exec.planner.physical.PlannerSettings;
+import org.apache.drill.exec.planner.physical.Prule;
+import org.apache.drill.exec.planner.physical.DrillDistributionTrait;
 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.drill.exec.planner.physical.TopNPrel;
+import org.apache.drill.exec.planner.physical.SortPrel;
+import org.apache.drill.exec.planner.physical.LimitPrel;
 import org.apache.calcite.plan.RelOptCluster;
 import org.apache.calcite.rel.type.RelDataType;
 import org.apache.calcite.rel.type.RelDataTypeFactory;
@@ -67,7 +69,6 @@ public abstract class AbstractIndexPlanGenerator extends SubsetTransformer<RelNo
   final protected DrillProjectRelBase origProject;
   final protected DrillScanRelBase origScan;
   final protected DrillProjectRelBase upperProject;
-  final protected RelNode origSort;
 
   final protected RexNode indexCondition;
   final protected RexNode remainderCondition;
@@ -84,7 +85,6 @@ public abstract class AbstractIndexPlanGenerator extends SubsetTransformer<RelNo
     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;
@@ -168,8 +168,8 @@ public abstract class AbstractIndexPlanGenerator extends SubsetTransformer<RelNo
     return set;
   }
 
-  protected static boolean toRemoveSort(Sort sort, RelCollation inputCollation) {
-    if ( (inputCollation != null) && inputCollation.satisfies(IndexPlanUtils.getCollation(sort))) {
+  protected static boolean toRemoveSort(RelCollation sortCollation, RelCollation inputCollation) {
+    if ( (inputCollation != null) && inputCollation.satisfies(sortCollation)) {
       return true;
     }
     return false;
@@ -194,18 +194,34 @@ public abstract class AbstractIndexPlanGenerator extends SubsetTransformer<RelNo
     }
   }
 
+  private static RelNode getSortOrTopN(IndexCallContext indexContext,
+                                       RelNode sortNode, RelNode newRel, RelNode child) {
+    if (sortNode instanceof TopNPrel) {
+      return new TopNPrel(sortNode.getCluster(),
+                    newRel.getTraitSet().replace(Prel.DRILL_PHYSICAL).plus(indexContext.getCollation()),
+                    child, ((TopNPrel)sortNode).getLimit(), indexContext.getCollation());
+    }
+    return new SortPrel(sortNode.getCluster(),
+            newRel.getTraitSet().replace(Prel.DRILL_PHYSICAL).plus(indexContext.getCollation()),
+            child, indexContext.getCollation());
+  }
+
   public static RelNode getSortNode(IndexCallContext indexContext, RelNode newRel, boolean donotGenerateSort,
                                     boolean isSingleton, boolean isExchangeRequired) {
-    Sort rel = indexContext.getSort();
+    OrderedRel rel = indexContext.getSort();
     DrillDistributionTrait hashDistribution =
         new DrillDistributionTrait(DrillDistributionTrait.DistributionType.HASH_DISTRIBUTED,
             ImmutableList.copyOf(indexContext.getDistributionFields()));
 
-    if ( toRemoveSort(indexContext.getSort(), newRel.getTraitSet().getTrait(RelCollationTraitDef.INSTANCE))) {
+    if ( toRemoveSort(indexContext.getCollation(), 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);
+      if (IndexPlanUtils.generateLimit(rel)) {
+        newRel = new LimitPrel(newRel.getCluster(),
+                newRel.getTraitSet().plus(indexContext.getCollation()).plus(Prel.DRILL_PHYSICAL),
+                newRel, IndexPlanUtils.getOffset(rel), IndexPlanUtils.getFetch(rel));
+      }
+      RelTraitSet traits = newRel.getTraitSet().plus(indexContext.getCollation()).plus(Prel.DRILL_PHYSICAL);
       newRel = Prule.convert(newRel, traits);
       newRel = getExchange(newRel.getCluster(), isSingleton, isExchangeRequired,
                                  traits, hashDistribution, indexContext, newRel);
@@ -215,10 +231,9 @@ public abstract class AbstractIndexPlanGenerator extends SubsetTransformer<RelNo
         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));
+      RelTraitSet traits = newRel.getTraitSet().plus(indexContext.getCollation()).plus(Prel.DRILL_PHYSICAL);
+      newRel = getSortOrTopN(indexContext, rel, newRel,
+                  Prule.convert(newRel, newRel.getTraitSet().replace(Prel.DRILL_PHYSICAL)));
       newRel = getExchange(newRel.getCluster(), isSingleton, isExchangeRequired,
                                  traits, hashDistribution, indexContext, newRel);
     }
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
index 163aef9..e06ac8f 100644
--- 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
@@ -18,6 +18,8 @@
 package org.apache.drill.exec.planner.index.generators;
 
 import org.apache.drill.shaded.guava.com.google.common.collect.Lists;
+import org.apache.drill.shaded.guava.com.google.common.base.Preconditions;
+
 import org.apache.calcite.plan.RelTraitSet;
 import org.apache.calcite.rel.InvalidRelException;
 import org.apache.calcite.rel.RelCollation;
@@ -30,11 +32,11 @@ 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.ScanPrel;
 import org.apache.drill.exec.planner.physical.Prule;
+import org.apache.drill.exec.planner.physical.Prel;
 import org.apache.drill.exec.physical.base.DbGroupScan;
 import java.util.List;
 
@@ -62,7 +64,7 @@ public class CoveringPlanNoFilterGenerator extends AbstractIndexPlanGenerator {
   }
 
   public RelNode convertChild() throws InvalidRelException {
-
+    Preconditions.checkNotNull(indexContext.getSort());
     if (indexGroupScan == null) {
       logger.error("Null indexgroupScan in CoveringIndexPlanGenerator.convertChild");
       return null;
@@ -106,11 +108,9 @@ public class CoveringPlanNoFilterGenerator extends AbstractIndexPlanGenerator {
       }
     }
 
-    if (indexContext.getSort() != null) {
-      finalRel = getSortNode(indexContext, finalRel, true, isSingletonSortedStream, indexContext.getExchange() != null);
-      if (finalRel == null) {
-        return 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));
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
index db220fa..e1337bc 100644
--- 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
@@ -319,7 +319,7 @@ public class NonCoveringIndexPlanGenerator extends AbstractIndexPlanGenerator {
     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))) {
+      if (toRemoveSort(indexContext.getCollation(), newRel.getTraitSet().getTrait(RelCollationTraitDef.INSTANCE))) {
         ((IndexGroupScan)indexScanPrel.getGroupScan()).setParallelizationWidth(1);
       }
       newRel = getSortNode(indexContext, newRel, false,true, true);
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
index db09504..86ed430 100644
--- 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
@@ -18,6 +18,8 @@
 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.base.Preconditions;
+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;
@@ -39,7 +41,7 @@ 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.common.OrderedRel;
 import org.apache.drill.exec.planner.physical.Prel;
 import org.apache.drill.exec.planner.physical.HashToRandomExchangePrel;
 import org.apache.calcite.rel.RelNode;
@@ -52,21 +54,21 @@ 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,
+      new DbScanSortRemovalRule(RelOptHelper.some(OrderedRel.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,
+          new DbScanSortRemovalRule(RelOptHelper.some(OrderedRel.class,
                           RelOptHelper.any(ScanPrel.class)), "DbScanSortRemovalRule:Sort_Scan", new MatchSS());
 
   public static final RelOptRule INDEX_SORT_PROJ_SCAN =
-          new DbScanSortRemovalRule(RelOptHelper.some(SortPrel.class,
+          new DbScanSortRemovalRule(RelOptHelper.some(OrderedRel.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,
+      new DbScanSortRemovalRule(RelOptHelper.some(OrderedRel.class,
           RelOptHelper.some(HashToRandomExchangePrel.class,
               RelOptHelper.some(ProjectPrel.class,
                   RelOptHelper.any(ScanPrel.class)))), "DbScanSortRemovalRule:sort_exchange_proj_Scan", new MatchSEPS());
@@ -80,16 +82,21 @@ public class DbScanSortRemovalRule extends Prule {
     this.match = match;
   }
 
+  private static boolean isRemovableRel(OrderedRel node) {
+    return node.canBeDropped();
+  }
+
   private static class MatchSES extends AbstractMatchFunction<IndexPhysicalPlanCallContext> {
 
     public boolean match(RelOptRuleCall call) {
-      final ScanPrel scan = (ScanPrel)call.rel(2);
-      return checkScan(scan.getGroupScan());
+      final OrderedRel sort = call.rel(0);
+      final ScanPrel scan = call.rel(2);
+      return sort instanceof Prel && checkScan(scan.getGroupScan()) && isRemovableRel(sort);
     }
 
     public IndexPhysicalPlanCallContext onMatch(RelOptRuleCall call) {
       final ScanPrel scan = call.rel(2);
-      final SortPrel sort = call.rel(0);
+      final OrderedRel sort = call.rel(0);
       final ExchangePrel exch = call.rel(1);
       return new IndexPhysicalPlanCallContext(call, sort, null, scan, exch);
     }
@@ -98,13 +105,14 @@ public class DbScanSortRemovalRule extends Prule {
   private static class MatchSS extends AbstractMatchFunction<IndexPhysicalPlanCallContext> {
 
     public boolean match(RelOptRuleCall call) {
-      final ScanPrel scan = (ScanPrel)call.rel(1);
-      return checkScan(scan.getGroupScan());
+      final OrderedRel sort = call.rel(0);
+      final ScanPrel scan = call.rel(1);
+      return sort instanceof Prel && checkScan(scan.getGroupScan()) && isRemovableRel(sort);
     }
 
     public IndexPhysicalPlanCallContext onMatch(RelOptRuleCall call) {
       final ScanPrel scan = call.rel(1);
-      final SortPrel sort = call.rel(0);
+      final OrderedRel sort = call.rel(0);
       return new IndexPhysicalPlanCallContext(call, sort, null, scan, null);
     }
   }
@@ -112,14 +120,15 @@ public class DbScanSortRemovalRule extends Prule {
   private static class MatchSPS extends AbstractMatchFunction<IndexPhysicalPlanCallContext> {
 
     public boolean match(RelOptRuleCall call) {
-      final ScanPrel scan = (ScanPrel)call.rel(2);
-      return checkScan(scan.getGroupScan());
+      final OrderedRel sort = call.rel(0);
+      final ScanPrel scan = call.rel(2);
+      return sort instanceof Prel && checkScan(scan.getGroupScan()) && isRemovableRel(sort);
     }
 
     public IndexPhysicalPlanCallContext onMatch(RelOptRuleCall call) {
       final ScanPrel scan = call.rel(2);
       final ProjectPrel proj = call.rel(1);
-      final SortPrel sort = call.rel(0);
+      final OrderedRel sort = call.rel(0);
       return new IndexPhysicalPlanCallContext(call, sort, proj, scan, null);
     }
   }
@@ -127,13 +136,14 @@ public class DbScanSortRemovalRule extends Prule {
   private static class MatchSEPS extends AbstractMatchFunction<IndexPhysicalPlanCallContext> {
 
     public boolean match(RelOptRuleCall call) {
-      final ScanPrel scan = (ScanPrel)call.rel(3);
-      return checkScan(scan.getGroupScan());
+      final OrderedRel sort = call.rel(0);
+      final ScanPrel scan = call.rel(3);
+      return sort instanceof Prel && checkScan(scan.getGroupScan()) && isRemovableRel(sort);
     }
 
     public IndexPhysicalPlanCallContext onMatch(RelOptRuleCall call) {
       final ScanPrel scan = call.rel(3);
-      final SortPrel sort = call.rel(0);
+      final OrderedRel sort = call.rel(0);
       final ProjectPrel proj = call.rel(2);
       final ExchangePrel exch = call.rel(1);
       return new IndexPhysicalPlanCallContext(call,  sort, proj, scan, exch);
@@ -187,12 +197,15 @@ public class DbScanSortRemovalRule extends Prule {
                           false, settings);
           if (planGen.convertChild() != null) {
             indexContext.getCall().transformTo(planGen.convertChild());
+          } else {
+            logger.debug("Not able to generate index plan in ", this.getClass().toString());
           }
         } catch (Exception e) {
           logger.warn("Exception while trying to generate indexscan to remove sort", e);
         }
       }
     } else {
+      Preconditions.checkNotNull(indexContext.getSort());
       //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;
@@ -204,12 +217,12 @@ public class DbScanSortRemovalRule extends Prule {
           inputs.add(finalRel);
           finalRel = indexContext.lowerProject.copy(indexContext.lowerProject.getTraitSet(), inputs);
         }
-        if (indexContext.getSort() != null) {
-          finalRel = AbstractIndexPlanGenerator.getSortNode(indexContext, finalRel, true,false,
+
+        finalRel = AbstractIndexPlanGenerator.getSortNode(indexContext, finalRel, true,false,
                   indexContext.exch != null);
-        }
 
         if (finalRel == null) {
+          logger.debug("Not able to generate index plan in ", this.getClass().toString());
           return;
         }
 
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/logical/DrillSortRel.java b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/logical/DrillSortRel.java
index cfa0e26..1e380cf 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/logical/DrillSortRel.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/logical/DrillSortRel.java
@@ -26,6 +26,7 @@ import org.apache.drill.common.logical.data.LogicalOperator;
 import org.apache.drill.common.logical.data.Order;
 import org.apache.drill.common.logical.data.Order.Ordering;
 import org.apache.drill.exec.planner.torel.ConversionContext;
+import org.apache.drill.exec.planner.common.OrderedRel;
 import org.apache.calcite.rel.InvalidRelException;
 import org.apache.calcite.rel.RelCollation;
 import org.apache.calcite.rel.RelFieldCollation;
@@ -41,7 +42,7 @@ import org.apache.drill.shaded.guava.com.google.common.collect.Maps;
 /**
  * Sort implemented in Drill.
  */
-public class DrillSortRel extends Sort implements DrillRel {
+public class DrillSortRel extends Sort implements DrillRel,OrderedRel {
 
   /** Creates a DrillSortRel. */
   public DrillSortRel(RelOptCluster cluster, RelTraitSet traits, RelNode input, RelCollation collation) {
@@ -98,4 +99,18 @@ public class DrillSortRel extends Sort implements DrillRel {
     return new DrillSortRel(context.getCluster(), context.getLogicalTraits(), input, RelCollations.of(collations));
   }
 
+  @Override
+  public RexNode getOffset() {
+    return offset;
+  }
+
+  @Override
+  public RexNode getFetch() {
+    return fetch;
+  }
+
+  @Override
+  public boolean canBeDropped() {
+    return true;
+  }
 }
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/physical/SortPrel.java b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/physical/SortPrel.java
index 77fb4c8..8064c42 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/physical/SortPrel.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/physical/SortPrel.java
@@ -27,6 +27,7 @@ import org.apache.calcite.rel.RelFieldCollation;
 import org.apache.drill.exec.ExecConstants;
 import org.apache.drill.exec.physical.base.PhysicalOperator;
 import org.apache.drill.exec.physical.config.ExternalSort;
+import org.apache.drill.exec.planner.common.OrderedRel;
 import org.apache.drill.exec.planner.cost.DrillCostBase;
 import org.apache.drill.exec.planner.cost.DrillCostBase.DrillCostFactory;
 import org.apache.drill.exec.planner.physical.visitor.PrelVisitor;
@@ -40,16 +41,25 @@ import org.apache.calcite.plan.RelOptPlanner;
 import org.apache.calcite.plan.RelTraitSet;
 import org.apache.calcite.rex.RexNode;
 
-public class SortPrel extends org.apache.calcite.rel.core.Sort implements Prel {
+public class SortPrel extends org.apache.calcite.rel.core.Sort implements OrderedRel,Prel {
+  private final boolean isRemovable;
 
   /** Creates a DrillSortRel. */
   public SortPrel(RelOptCluster cluster, RelTraitSet traits, RelNode input, RelCollation collation) {
     super(cluster, traits, input, collation);
+    isRemovable = true;
   }
 
   /** Creates a DrillSortRel with offset and fetch. */
   public SortPrel(RelOptCluster cluster, RelTraitSet traits, RelNode input, RelCollation collation, RexNode offset, RexNode fetch) {
     super(cluster, traits, input, collation, offset, fetch);
+    isRemovable = true;
+  }
+
+  /** Creates a DrillSortRel. */
+  public SortPrel(RelOptCluster cluster, RelTraitSet traits, RelNode input, RelCollation collation, boolean isRemovable) {
+    super(cluster, traits, input, collation);
+    this.isRemovable = isRemovable;
   }
 
   @Override
@@ -141,4 +151,20 @@ public class SortPrel extends org.apache.calcite.rel.core.Sort implements Prel {
 
     return this.copy(traits, children.get(0), collationTrait, this.offset, this.fetch);
   }
+
+  @Override
+  public RexNode getOffset() {
+    return offset;
+  }
+
+  @Override
+  public RexNode getFetch() {
+    return fetch;
+  }
+
+  @Override
+  public boolean canBeDropped() {
+    return isRemovable;
+  }
+
 }
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/physical/SortPrule.java b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/physical/SortPrule.java
index 3fc86b3..bec1b6a 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/physical/SortPrule.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/physical/SortPrule.java
@@ -47,26 +47,25 @@ public class SortPrule extends Prule{
 
   @Override
   public void onMatch(RelOptRuleCall call) {
-    final DrillSortRel sort = (DrillSortRel) call.rel(0);
+    final DrillSortRel sort = call.rel(0);
     final RelNode input = sort.getInput();
 
     // Keep the collation in logical sort. Convert input into a RelNode with 1) this collation, 2) Physical, 3) hash distributed on
 
     DrillDistributionTrait hashDistribution =
-        new DrillDistributionTrait(DrillDistributionTrait.DistributionType.HASH_DISTRIBUTED, ImmutableList.copyOf(getDistributionField(sort)));
+            new DrillDistributionTrait(DrillDistributionTrait.DistributionType.HASH_DISTRIBUTED, ImmutableList.copyOf(getDistributionField(sort)));
 
-    final RelTraitSet traits = sort.getTraitSet().plus(Prel.DRILL_PHYSICAL).plus(hashDistribution);
-
-    final RelNode convertedInput = convert(input, traits);
+    final RelTraitSet traits = RelTraitSet.createEmpty().plus(Prel.DRILL_PHYSICAL).plus(hashDistribution);
+    SortPrel child = new SortPrel(sort.getCluster(), traits.plus(sort.getCollation()),
+            convert(sort.getInput(), traits), sort.getCollation(), false);
 
     if(isSingleMode(call)){
-      call.transformTo(convertedInput);
+      call.transformTo(child);
     }else{
-      RelNode exch = new SingleMergeExchangePrel(sort.getCluster(), sort.getTraitSet().plus(Prel.DRILL_PHYSICAL).plus(DrillDistributionTrait.SINGLETON), convertedInput, sort.getCollation());
+      RelNode exch = new SingleMergeExchangePrel(sort.getCluster(), sort.getTraitSet().plus(Prel.DRILL_PHYSICAL).plus(DrillDistributionTrait.SINGLETON), child, sort.getCollation());
       call.transformTo(exch);  // transform logical "sort" into "SingleMergeExchange".
 
     }
-
   }
 
   private List<DistributionField> getDistributionField(DrillSortRel rel) {
@@ -76,7 +75,6 @@ public class SortPrule extends Prule{
       DistributionField field = new DistributionField(relField.getFieldIndex());
       distFields.add(field);
     }
-
     return distFields;
   }
 
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/physical/TopNPrel.java b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/physical/TopNPrel.java
index e9414f1..f8f4b9d 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/physical/TopNPrel.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/physical/TopNPrel.java
@@ -26,8 +26,11 @@ import org.apache.calcite.rel.RelCollationImpl;
 import org.apache.calcite.rel.RelFieldCollation;
 import org.apache.calcite.rex.RexNode;
 import org.apache.calcite.sql.type.SqlTypeName;
+import org.apache.calcite.rex.RexNode;
+import org.apache.calcite.sql.type.SqlTypeName;
 import org.apache.drill.exec.physical.base.PhysicalOperator;
 import org.apache.drill.exec.physical.config.TopN;
+import org.apache.drill.exec.planner.common.OrderedRel;
 import org.apache.drill.exec.planner.cost.DrillCostBase;
 import org.apache.drill.exec.planner.cost.DrillCostBase.DrillCostFactory;
 import org.apache.drill.exec.record.BatchSchema.SelectionVectorMode;
@@ -40,7 +43,7 @@ import org.apache.calcite.plan.RelOptCost;
 import org.apache.calcite.plan.RelOptPlanner;
 import org.apache.calcite.plan.RelTraitSet;
 
-public class TopNPrel extends SinglePrel {
+public class TopNPrel extends SinglePrel implements OrderedRel,Prel {
 
   protected int limit;
   protected final RelCollation collation;
@@ -66,6 +69,28 @@ public class TopNPrel extends SinglePrel {
     return creator.addMetadata(this, topN);
   }
 
+  @Override
+  public RelCollation getCollation() {
+    return collation;
+  }
+
+  @Override
+  public RexNode getOffset() {
+    return getCluster().getRexBuilder().makeExactLiteral(BigDecimal.ZERO,
+                  getCluster().getTypeFactory().createSqlType(SqlTypeName.INTEGER));
+  }
+
+  @Override
+  public RexNode getFetch() {
+    return getCluster().getRexBuilder().makeExactLiteral(BigDecimal.valueOf(limit),
+                 getCluster().getTypeFactory().createSqlType(SqlTypeName.INTEGER));
+  }
+
+  @Override
+  public boolean canBeDropped() {
+    return true;
+  }
+
   /**
    * Cost of doing Top-N is proportional to M log N where M is the total number of
    * input rows and N is the limit for Top-N.  This makes Top-N preferable to Sort
@@ -93,6 +118,10 @@ public class TopNPrel extends SinglePrel {
         .item("limit", limit);
   }
 
+  public int getLimit() {
+    return limit;
+  }
+
   @Override
   public SelectionVectorMode[] getSupportedEncodings() {
     return SelectionVectorMode.NONE_AND_TWO;
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/physical/visitor/ExcessiveExchangeIdentifier.java b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/physical/visitor/ExcessiveExchangeIdentifier.java
index f77a437..fa8e69d 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/physical/visitor/ExcessiveExchangeIdentifier.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/physical/visitor/ExcessiveExchangeIdentifier.java
@@ -25,6 +25,11 @@ import org.apache.drill.exec.planner.physical.ExchangePrel;
 import org.apache.drill.exec.planner.physical.Prel;
 import org.apache.drill.exec.planner.physical.ScanPrel;
 import org.apache.drill.exec.planner.physical.ScreenPrel;
+import org.apache.drill.exec.planner.physical.LimitPrel;
+import org.apache.drill.exec.planner.physical.ProjectPrel;
+import org.apache.drill.exec.planner.physical.FilterPrel;
+import org.apache.drill.exec.planner.physical.SingleMergeExchangePrel;
+import org.apache.drill.exec.planner.physical.HashToMergeExchangePrel;
 import org.apache.calcite.rel.RelNode;
 
 import org.apache.drill.shaded.guava.com.google.common.collect.Lists;
@@ -48,7 +53,18 @@ public class ExcessiveExchangeIdentifier extends BasePrelVisitor<Prel, Excessive
     parent.add(prel);
     MajorFragmentStat newFrag = new MajorFragmentStat();
     newFrag.setRightSideOfLateral(parent.isRightSideOfLateral());
+
+    if (prel instanceof SingleMergeExchangePrel) {
+      newFrag.isSimpleRel = true;
+    }
+
     Prel newChild = ((Prel) prel.getInput()).accept(this, newFrag);
+
+    if (parent.isSimpleRel &&
+        prel instanceof HashToMergeExchangePrel) {
+      return newChild;
+    }
+
     if (canRemoveExchange(parent, newFrag)) {
       return newChild;
     } else {
@@ -129,6 +145,8 @@ public class ExcessiveExchangeIdentifier extends BasePrelVisitor<Prel, Excessive
       s.add(p);
     }
 
+    s.setHashDistribution(prel);
+
     for(Prel p : prel) {
       children.add(p.accept(this, s));
     }
@@ -145,6 +163,9 @@ public class ExcessiveExchangeIdentifier extends BasePrelVisitor<Prel, Excessive
     private int maxWidth = Integer.MAX_VALUE;
     private boolean isMultiSubScan = false;
     private boolean rightSideOfLateral = false;
+    //This flag if true signifies that all the Rels thus far
+    //are simple rels with no distribution requirement.
+    private boolean isSimpleRel = false;
 
     public void add(Prel prel) {
       maxRows = Math.max(prel.estimateRowCount(prel.getCluster().getMetadataQuery()), maxRows);
@@ -162,6 +183,13 @@ public class ExcessiveExchangeIdentifier extends BasePrelVisitor<Prel, Excessive
       add(prel);
     }
 
+    public void setHashDistribution(Prel prel) {
+      isSimpleRel = isSimpleRel &&
+                    (prel instanceof LimitPrel ||
+                     prel instanceof ProjectPrel ||
+                     prel instanceof FilterPrel);
+    }
+
     public boolean isSingular() {
       // do not remove exchanges when a scan has more than one subscans (e.g. SystemTableScan)
       if (isMultiSubScan) {


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

Posted by am...@apache.org.
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 ...


[drill] 05/08: DRILL-6381: (Part 5) Update Javadoc for a few interfaces.

Posted by am...@apache.org.
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 3c2f9ab6c0ed290bed3bc440d72a670bbe512b5a
Author: Aman Sinha <as...@maprtech.com>
AuthorDate: Thu Sep 13 03:55:47 2018 -0700

    DRILL-6381: (Part 5) Update Javadoc for a few interfaces.
---
 .../org/apache/drill/exec/physical/base/DbSubScan.java |  4 ++--
 .../drill/exec/planner/index/IndexCollection.java      |  4 +++-
 .../drill/exec/planner/index/IndexDefinition.java      | 18 ++++++++++++++++--
 .../drill/exec/planner/index/IndexDescriptor.java      | 15 +++++++++++++++
 4 files changed, 36 insertions(+), 5 deletions(-)

diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/base/DbSubScan.java b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/base/DbSubScan.java
index 874468d..1977257 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/base/DbSubScan.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/base/DbSubScan.java
@@ -32,8 +32,8 @@ public interface DbSubScan extends SubScan {
   boolean isRestrictedSubScan();
 
   /**
-   * For a restricted sub-scan, this method allows associating a (hash)join instance.  A subscan within a minor
-   * fragment must have a corresponding (hash)join batch instance from which it will retrieve its set of
+   * For a restricted sub-scan, this method allows associating a rowkey join instance.  A subscan within a minor
+   * fragment must have a corresponding rowkey join batch instance from which it will retrieve its set of
    * rowkeys to perform the restricted scan.
    * @param batch
    */
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/index/IndexCollection.java b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/index/IndexCollection.java
index 9b4d170..f5ab173 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/index/IndexCollection.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/index/IndexCollection.java
@@ -21,7 +21,9 @@ import org.apache.calcite.rex.RexNode;
 import org.apache.drill.common.expression.SchemaPath;
 import org.apache.drill.exec.physical.base.IndexGroupScan;
 
-// Interface used to describe an index collection
+/**
+ *  Interface used to describe an index collection
+ */
 public interface IndexCollection extends Iterable<IndexDescriptor> {
   /**
    * Types of an index collections: NATIVE_SECONDARY_INDEX_COLLECTION, EXTERNAL_SECONDARY_INDEX_COLLECTION
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/index/IndexDefinition.java b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/index/IndexDefinition.java
index 995d23c..bfbccc2 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/index/IndexDefinition.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/index/IndexDefinition.java
@@ -26,7 +26,9 @@ import java.util.Collection;
 import java.util.List;
 import java.util.Map;
 
-// Interface used to define an index,
+/**
+ * Top level interface used to define an index.
+ */
 public interface IndexDefinition {
   /**
    * Types of an index: PRIMARY_KEY_INDEX, NATIVE_SECONDARY_INDEX, EXTERNAL_SECONDARY_INDEX
@@ -87,13 +89,25 @@ public interface IndexDefinition {
    */
   IndexType getIndexType();
 
-
+  /**
+   * Get the list of columns in the index key.
+   */
   List<LogicalExpression> getIndexColumns();
 
+  /**
+   * Get the list of columns that are in the 'included' or 'covered' fields.
+   */
   List<LogicalExpression> getNonIndexColumns();
 
+  /**
+   * Get the collation property (physical ordering) of the index.
+   */
   RelCollation getCollation();
 
+  /**
+   * Get a mapping of the LogicalExpresion/SchemaPath to its field collation
+   *  - e.g Ascending/Descending, Nulls First/Nulls Last
+   */
   Map<LogicalExpression, RelFieldCollation> getCollationMap();
 
   /**
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/index/IndexDescriptor.java b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/index/IndexDescriptor.java
index f355285..d43ba81 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/index/IndexDescriptor.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/index/IndexDescriptor.java
@@ -58,11 +58,26 @@ public interface IndexDescriptor extends IndexDefinition {
    */
   boolean supportsFullTextSearch();
 
+  /**
+   * Get the functional index information associated with this index (Functional indexes are
+   * indexes involving expressions e.g CAST(a as INT).
+   */
   FunctionalIndexInfo getFunctionalInfo();
 
+  /**
+   * Get the total cost of index access (I/O, CPU) in the context of the current query
+   * @param indexProps properties (metrics) of a single index in the context of current query
+   * @param planner Planner instance
+   * @param numProjectedFields Number of projected fields
+   * @param primaryGroupScan Primary table's GroupScan instance
+   * @return a RelOptCost instance representing the total cost
+   */
   public RelOptCost getCost(IndexProperties indexProps, RelOptPlanner planner,
       int numProjectedFields, GroupScan primaryGroupScan);
 
+  /**
+   * Get the costing factors associated with the storage/format plugin
+   */
   public PluginCost getPluginCostModel();
 
 }


[drill] 07/08: DRILL-6381: Address review comments (part 2): fix formatting issues and add javadoc.

Posted by am...@apache.org.
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 387bc4fefc40b645685439fd1da43f7223e5933c
Author: Aman Sinha <as...@maprtech.com>
AuthorDate: Wed Oct 10 16:36:48 2018 -0700

    DRILL-6381: Address review comments (part 2): fix formatting issues and add javadoc.
---
 .../planner/index/MapRDBFunctionalIndexInfo.java   |  22 ++--
 .../exec/planner/index/MapRDBIndexDescriptor.java  |   4 +-
 .../exec/store/mapr/db/MapRDBScanBatchCreator.java |   4 +-
 .../exec/store/mapr/db/MapRDBSubScanSpec.java      |   2 +-
 .../store/mapr/db/json/JsonTableGroupScan.java     |  17 ++-
 .../db/json/JsonTableRangePartitionFunction.java   |   4 +-
 .../store/mapr/db/json/OjaiFunctionsProcessor.java |   2 +-
 .../mapr/db/json/RestrictedJsonTableGroupScan.java |   8 +-
 .../drill/exec/udf/mapr/db/DecodeFieldPath.java    |   4 +-
 .../drill/exec/udf/mapr/db/MatchesPlaceholder.java |   7 +-
 .../exec/udf/mapr/db/NotMatchesPlaceholder.java    |   7 +-
 .../exec/udf/mapr/db/NotTypeOfPlaceholder.java     |   7 +-
 .../drill/exec/udf/mapr/db/SizeOfPlaceholder.java  |   7 +-
 .../drill/exec/udf/mapr/db/TypeOfPlaceholder.java  |   7 +-
 .../maprdb/tests/index/IndexHintPlanTest.java      |  19 ++-
 .../drill/maprdb/tests/index/IndexPlanTest.java    | 135 +++++++++------------
 .../drill/maprdb/tests/index/LargeTableGen.java    |  34 +++---
 .../maprdb/tests/index/LargeTableGenBase.java      |  25 ++--
 .../drill/maprdb/tests/index/StatisticsTest.java   |  21 ++--
 .../drill/maprdb/tests/index/TableIndexCmd.java    |  26 ++--
 .../drill/exec/physical/base/IndexGroupScan.java   |   1 -
 .../exec/physical/impl/join/RowKeyJoinBatch.java   |  18 +--
 .../exec/planner/index/DrillIndexDefinition.java   |  22 ++--
 .../exec/planner/index/DrillIndexDescriptor.java   |  24 +++-
 .../exec/planner/index/IndexConditionInfo.java     |  44 ++++---
 .../drill/exec/planner/index/IndexPlanUtils.java   |  74 +++++------
 .../drill/exec/planner/index/IndexSelector.java    |   2 +-
 .../drill/exec/planner/index/SimpleRexRemap.java   |  18 +--
 .../planner/index/rules/AbstractMatchFunction.java |   8 +-
 .../planner/logical/DrillMergeProjectRule.java     |   2 +-
 30 files changed, 271 insertions(+), 304 deletions(-)

diff --git a/contrib/format-maprdb/src/main/java/org/apache/drill/exec/planner/index/MapRDBFunctionalIndexInfo.java b/contrib/format-maprdb/src/main/java/org/apache/drill/exec/planner/index/MapRDBFunctionalIndexInfo.java
index ec38636..67938f3 100644
--- a/contrib/format-maprdb/src/main/java/org/apache/drill/exec/planner/index/MapRDBFunctionalIndexInfo.java
+++ b/contrib/format-maprdb/src/main/java/org/apache/drill/exec/planner/index/MapRDBFunctionalIndexInfo.java
@@ -32,15 +32,15 @@ public class MapRDBFunctionalIndexInfo implements FunctionalIndexInfo {
 
   private boolean hasFunctionalField = false;
 
-  //when we scan schemaPath in groupscan's columns, we check if this column(schemaPath) should be rewritten to '$N',
-  //When there are more than two functions on the same column in index, CAST(a.b as INT), CAST(a.b as VARCHAR),
+  // When we scan schemaPath in groupscan's columns, we check if this column(schemaPath) should be rewritten to '$N',
+  // When there are more than two functions on the same column in index, CAST(a.b as INT), CAST(a.b as VARCHAR),
   // then we should map SchemaPath a.b to a set of SchemaPath, e.g. $1, $2
   private Map<SchemaPath, Set<SchemaPath>> columnToConvert;
 
   // map of functional index expression to destination SchemaPath e.g. $N
   private Map<LogicalExpression, LogicalExpression> exprToConvert;
 
-  //map of SchemaPath involved in a functional field
+  // map of SchemaPath involved in a functional field
   private Map<LogicalExpression, Set<SchemaPath>> pathsInExpr;
 
   private Set<SchemaPath> newPathsForIndexedFunction;
@@ -52,7 +52,7 @@ public class MapRDBFunctionalIndexInfo implements FunctionalIndexInfo {
     columnToConvert = Maps.newHashMap();
     exprToConvert = Maps.newHashMap();
     pathsInExpr = Maps.newHashMap();
-    //keep the order of new paths, it may be related to the naming policy
+    // keep the order of new paths, it may be related to the naming policy
     newPathsForIndexedFunction = Sets.newLinkedHashSet();
     allPathsInFunction = Sets.newHashSet();
     init();
@@ -60,15 +60,15 @@ public class MapRDBFunctionalIndexInfo implements FunctionalIndexInfo {
 
   private void init() {
     int count = 0;
-    for(LogicalExpression indexedExpr : indexDesc.getIndexColumns()) {
-      if( !(indexedExpr instanceof SchemaPath) ) {
+    for (LogicalExpression indexedExpr : indexDesc.getIndexColumns()) {
+      if (!(indexedExpr instanceof SchemaPath)) {
         hasFunctionalField = true;
         SchemaPath functionalFieldPath = SchemaPath.getSimplePath("$"+count);
         newPathsForIndexedFunction.add(functionalFieldPath);
 
-        //now we handle only cast expression
-        if(indexedExpr instanceof CastExpression) {
-          //We handle only CAST directly on SchemaPath for now.
+        // now we handle only cast expression
+        if (indexedExpr instanceof CastExpression) {
+          // We handle only CAST directly on SchemaPath for now.
           SchemaPath pathBeingCasted = (SchemaPath)((CastExpression) indexedExpr).getInput();
           addTargetPathForOriginalPath(pathBeingCasted, functionalFieldPath);
           addPathInExpr(indexedExpr, pathBeingCasted);
@@ -119,7 +119,7 @@ public class MapRDBFunctionalIndexInfo implements FunctionalIndexInfo {
    * @return
    */
   public SchemaPath getNewPath(SchemaPath path) {
-    if(columnToConvert.containsKey(path)) {
+    if (columnToConvert.containsKey(path)) {
       return columnToConvert.get(path).iterator().next();
     }
     return null;
@@ -131,7 +131,7 @@ public class MapRDBFunctionalIndexInfo implements FunctionalIndexInfo {
    * @return the renamed schemapath in index table for the indexed expression
    */
   public SchemaPath getNewPathFromExpr(LogicalExpression expr) {
-    if(exprToConvert.containsKey(expr)) {
+    if (exprToConvert.containsKey(expr)) {
       return (SchemaPath)exprToConvert.get(expr);
     }
     return null;
diff --git a/contrib/format-maprdb/src/main/java/org/apache/drill/exec/planner/index/MapRDBIndexDescriptor.java b/contrib/format-maprdb/src/main/java/org/apache/drill/exec/planner/index/MapRDBIndexDescriptor.java
index a57f5b5..75e6bc2 100644
--- a/contrib/format-maprdb/src/main/java/org/apache/drill/exec/planner/index/MapRDBIndexDescriptor.java
+++ b/contrib/format-maprdb/src/main/java/org/apache/drill/exec/planner/index/MapRDBIndexDescriptor.java
@@ -122,9 +122,9 @@ public class MapRDBIndexDescriptor extends DrillIndexDescriptor {
       List<LogicalExpression> allCols = Lists.newArrayList();
       Collection<SchemaPath> decoded;
 
-      for(LogicalExpression expr : expressions) {
+      for (LogicalExpression expr : expressions) {
         LogicalExpression nonDecoded = expr.accept(this, null);
-        if(nonDecoded != null) {
+        if (nonDecoded != null) {
           allCols.add(nonDecoded);
         }
       }
diff --git a/contrib/format-maprdb/src/main/java/org/apache/drill/exec/store/mapr/db/MapRDBScanBatchCreator.java b/contrib/format-maprdb/src/main/java/org/apache/drill/exec/store/mapr/db/MapRDBScanBatchCreator.java
index de2817e..2f53398 100644
--- a/contrib/format-maprdb/src/main/java/org/apache/drill/exec/store/mapr/db/MapRDBScanBatchCreator.java
+++ b/contrib/format-maprdb/src/main/java/org/apache/drill/exec/store/mapr/db/MapRDBScanBatchCreator.java
@@ -33,14 +33,14 @@ import org.apache.drill.exec.store.mapr.db.json.MaprDBJsonRecordReader;
 
 import org.apache.drill.shaded.guava.com.google.common.base.Preconditions;
 
-public class MapRDBScanBatchCreator implements BatchCreator<MapRDBSubScan>{
+public class MapRDBScanBatchCreator implements BatchCreator<MapRDBSubScan> {
   private static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(MapRDBScanBatchCreator.class);
 
   @Override
   public ScanBatch getBatch(ExecutorFragmentContext context, MapRDBSubScan subScan, List<RecordBatch> children) throws ExecutionSetupException {
     Preconditions.checkArgument(children.isEmpty());
     List<RecordReader> readers = new LinkedList<>();
-    for(MapRDBSubScanSpec scanSpec : subScan.getRegionScanSpecList()){
+    for (MapRDBSubScanSpec scanSpec : subScan.getRegionScanSpecList()) {
       try {
         if (BinaryTableGroupScan.TABLE_BINARY.equals(subScan.getTableType())) {
           readers.add(new HBaseRecordReader(
diff --git a/contrib/format-maprdb/src/main/java/org/apache/drill/exec/store/mapr/db/MapRDBSubScanSpec.java b/contrib/format-maprdb/src/main/java/org/apache/drill/exec/store/mapr/db/MapRDBSubScanSpec.java
index e24438e..7fc2e83 100644
--- a/contrib/format-maprdb/src/main/java/org/apache/drill/exec/store/mapr/db/MapRDBSubScanSpec.java
+++ b/contrib/format-maprdb/src/main/java/org/apache/drill/exec/store/mapr/db/MapRDBSubScanSpec.java
@@ -23,7 +23,7 @@ import com.mapr.db.index.IndexDesc;
 import com.mapr.fs.jni.MapRConstants;
 import com.mapr.org.apache.hadoop.hbase.util.Bytes;
 
-public class MapRDBSubScanSpec implements Comparable<MapRDBSubScanSpec>{
+public class MapRDBSubScanSpec implements Comparable<MapRDBSubScanSpec> {
 
   protected String tableName;
   protected IndexDesc indexDesc;
diff --git a/contrib/format-maprdb/src/main/java/org/apache/drill/exec/store/mapr/db/json/JsonTableGroupScan.java b/contrib/format-maprdb/src/main/java/org/apache/drill/exec/store/mapr/db/json/JsonTableGroupScan.java
index b545262..647fe82 100644
--- a/contrib/format-maprdb/src/main/java/org/apache/drill/exec/store/mapr/db/json/JsonTableGroupScan.java
+++ b/contrib/format-maprdb/src/main/java/org/apache/drill/exec/store/mapr/db/json/JsonTableGroupScan.java
@@ -320,7 +320,7 @@ public class JsonTableGroupScan extends MapRDBGroupScan implements IndexGroupSca
     double totalRowCount = stats.getRowCount(null, null);
     logger.debug("GroupScan {} with stats {}: rowCount={}, condition={}, totalRowCount={}, fullTableRowCount={}",
             System.identityHashCode(this), System.identityHashCode(stats), rowCount,
-            scanSpec.getCondition()==null?"null":scanSpec.getCondition(),
+            scanSpec.getCondition() == null ? "null" : scanSpec.getCondition(),
             totalRowCount, fullTableRowCount);
     // If UNKNOWN, or DB stats sync issues(manifests as 0 rows) use defaults.
     if (rowCount == ROWCOUNT_UNKNOWN || rowCount == 0) {
@@ -377,7 +377,7 @@ public class JsonTableGroupScan extends MapRDBGroupScan implements IndexGroupSca
     PluginCost pluginCostModel = formatPlugin.getPluginCostModel();
     final int avgColumnSize = pluginCostModel.getAverageColumnSize(this);
     boolean filterPushed = (scanSpec.getSerializedFilter() != null);
-    if(scanSpec != null && scanSpec.getIndexDesc() != null) {
+    if (scanSpec != null && scanSpec.getIndexDesc() != null) {
       totalColNum = scanSpec.getIndexDesc().getIncludedFields().size()
           + scanSpec.getIndexDesc().getIndexedFields().size() + 1;
     }
@@ -446,8 +446,8 @@ public class JsonTableGroupScan extends MapRDBGroupScan implements IndexGroupSca
   @Override
   public String toString() {
     return "JsonTableGroupScan [ScanSpec=" + scanSpec + ", columns=" + columns
-        + (maxRecordsToRead>0? ", limit=" + maxRecordsToRead : "")
-        + (getMaxParallelizationWidth()>0? ", maxwidth=" + getMaxParallelizationWidth() : "") + "]";
+        + (maxRecordsToRead > 0 ? ", limit=" + maxRecordsToRead : "")
+        + (getMaxParallelizationWidth() > 0 ? ", maxwidth=" + getMaxParallelizationWidth() : "") + "]";
   }
 
   public JsonScanSpec getScanSpec() {
@@ -498,7 +498,7 @@ public class JsonTableGroupScan extends MapRDBGroupScan implements IndexGroupSca
       indexDesc = (IndexDesc)((MapRDBIndexDescriptor)index).getOriginalDesc();
     }
     // If no index is specified, get it from the primary table
-    if(indexDesc == null && scanSpec.isSecondaryIndex()) {
+    if (indexDesc == null && scanSpec.isSecondaryIndex()) {
       throw new UnsupportedOperationException("getAverageRowSizeStats should be invoked on primary table");
     }
 
@@ -540,12 +540,11 @@ public class JsonTableGroupScan extends MapRDBGroupScan implements IndexGroupSca
    * @return {@link MapRDBStatisticsPayload} statistics
    */
   private MapRDBStatisticsPayload getFirstKeyEstimatedStatsInternal(QueryCondition condition, IndexDesc index, RelNode scanRel) {
-    // double totalRows = getRowCount(null, scanPrel);
 
     // If no index is specified, get it from the primary table
-    if(index == null && scanSpec.isSecondaryIndex()) {
+    if (index == null && scanSpec.isSecondaryIndex()) {
       // If stats not cached get it from the table.
-      //table = MapRDB.getTable(scanSpec.getPrimaryTablePath());
+      // table = MapRDB.getTable(scanSpec.getPrimaryTablePath());
       throw new UnsupportedOperationException("getFirstKeyEstimatedStats should be invoked on primary table");
     }
 
@@ -740,7 +739,7 @@ public class JsonTableGroupScan extends MapRDBGroupScan implements IndexGroupSca
     if (maxRecordsToRead < 0) {
       return true;
     }
-    return false;//limit is already pushed. No more pushdown of limit
+    return false; // limit is already pushed. No more pushdown of limit
   }
 
   @Override
diff --git a/contrib/format-maprdb/src/main/java/org/apache/drill/exec/store/mapr/db/json/JsonTableRangePartitionFunction.java b/contrib/format-maprdb/src/main/java/org/apache/drill/exec/store/mapr/db/json/JsonTableRangePartitionFunction.java
index 436347f..c0b73ee 100644
--- a/contrib/format-maprdb/src/main/java/org/apache/drill/exec/store/mapr/db/json/JsonTableRangePartitionFunction.java
+++ b/contrib/format-maprdb/src/main/java/org/apache/drill/exec/store/mapr/db/json/JsonTableRangePartitionFunction.java
@@ -117,7 +117,7 @@ public class JsonTableRangePartitionFunction extends AbstractRangePartitionFunct
       if (thisPartRefList.size() != otherPartRefList.size()) {
         return false;
       }
-      for (int refIdx=0; refIdx<thisPartRefList.size(); refIdx++) {
+      for (int refIdx = 0; refIdx < thisPartRefList.size(); refIdx++) {
         if (!thisPartRefList.get(refIdx).equals(otherPartRefList.get(refIdx))) {
           return false;
         }
@@ -148,7 +148,7 @@ public class JsonTableRangePartitionFunction extends AbstractRangePartitionFunct
 
       // Check if key is present in the mid interval of [start, stop].
       // Account for empty byte array start/stop
-      if ( (Bytes.compareTo(encodedKey, start) >= 0 ||
+      if ((Bytes.compareTo(encodedKey, start) >= 0 ||
              Bytes.equals(start, MapRConstants.EMPTY_BYTE_ARRAY)
            ) &&
            (Bytes.compareTo(encodedKey, stop) < 0 ||
diff --git a/contrib/format-maprdb/src/main/java/org/apache/drill/exec/store/mapr/db/json/OjaiFunctionsProcessor.java b/contrib/format-maprdb/src/main/java/org/apache/drill/exec/store/mapr/db/json/OjaiFunctionsProcessor.java
index 959e243..7e29c22 100644
--- a/contrib/format-maprdb/src/main/java/org/apache/drill/exec/store/mapr/db/json/OjaiFunctionsProcessor.java
+++ b/contrib/format-maprdb/src/main/java/org/apache/drill/exec/store/mapr/db/json/OjaiFunctionsProcessor.java
@@ -47,7 +47,7 @@ class OjaiFunctionsProcessor extends AbstractExprVisitor<Void, Void, RuntimeExce
     final Throwable throwable = new Throwable();
     final StackTraceElement[] ste = throwable.getStackTrace();
     final StringBuilder sb = new StringBuilder();
-    for(int i = 1; i < ste.length; ++i) {
+    for (int i = 1; i < ste.length; ++i) {
       sb.append(ste[i].toString());
       sb.append('\n');
     }
diff --git a/contrib/format-maprdb/src/main/java/org/apache/drill/exec/store/mapr/db/json/RestrictedJsonTableGroupScan.java b/contrib/format-maprdb/src/main/java/org/apache/drill/exec/store/mapr/db/json/RestrictedJsonTableGroupScan.java
index 48ad96d..2f06d00 100644
--- a/contrib/format-maprdb/src/main/java/org/apache/drill/exec/store/mapr/db/json/RestrictedJsonTableGroupScan.java
+++ b/contrib/format-maprdb/src/main/java/org/apache/drill/exec/store/mapr/db/json/RestrictedJsonTableGroupScan.java
@@ -90,7 +90,7 @@ public class RestrictedJsonTableGroupScan extends JsonTableGroupScan {
   private List<RestrictedMapRDBSubScanSpec> getEndPointFragmentMapping(int minorFragmentId) {
     List<RestrictedMapRDBSubScanSpec> restrictedSubScanSpecList = Lists.newArrayList();
     List<MapRDBSubScanSpec> subScanSpecList = endpointFragmentMapping.get(minorFragmentId);
-    for(MapRDBSubScanSpec s : subScanSpecList) {
+    for (MapRDBSubScanSpec s : subScanSpecList) {
       restrictedSubScanSpecList.add((RestrictedMapRDBSubScanSpec) s);
     }
     return restrictedSubScanSpecList;
@@ -128,7 +128,7 @@ public class RestrictedJsonTableGroupScan extends JsonTableGroupScan {
 
   @Override
   public ScanStats getScanStats() {
-    //TODO: ideally here we should use the rowcount from index scan, and multiply a factor of restricted scan
+    // TODO: ideally here we should use the rowcount from index scan, and multiply a factor of restricted scan
     double rowCount;
     PluginCost pluginCostModel = formatPlugin.getPluginCostModel();
     final int avgColumnSize = pluginCostModel.getAverageColumnSize(this);
@@ -178,7 +178,7 @@ public class RestrictedJsonTableGroupScan extends JsonTableGroupScan {
   public String toString() {
     return "RestrictedJsonTableGroupScan [ScanSpec=" + scanSpec + ", columns=" + columns
         + ", rowcount=" + computeRestrictedScanRowcount()
-        + (maxRecordsToRead>0? ", limit=" + maxRecordsToRead : "")
-        + (getMaxParallelizationWidth()>0? ", maxwidth=" + getMaxParallelizationWidth() : "") + "]";
+        + (maxRecordsToRead > 0 ? ", limit=" + maxRecordsToRead : "")
+        + (getMaxParallelizationWidth() > 0 ? ", maxwidth=" + getMaxParallelizationWidth() : "") + "]";
   }
 }
diff --git a/contrib/format-maprdb/src/main/java/org/apache/drill/exec/udf/mapr/db/DecodeFieldPath.java b/contrib/format-maprdb/src/main/java/org/apache/drill/exec/udf/mapr/db/DecodeFieldPath.java
index 6748c4f..3241be4 100644
--- a/contrib/format-maprdb/src/main/java/org/apache/drill/exec/udf/mapr/db/DecodeFieldPath.java
+++ b/contrib/format-maprdb/src/main/java/org/apache/drill/exec/udf/mapr/db/DecodeFieldPath.java
@@ -46,9 +46,9 @@ public class DecodeFieldPath implements DrillSimpleFunc {
         toStringFromUTF8(input.start, input.end, input.buffer).split(",");
     String[] decodedPaths = org.apache.drill.exec.util.EncodedSchemaPathSet.decode(encodedPaths);
     java.util.Arrays.sort(decodedPaths);
-    
+
     StringBuilder sb = new StringBuilder();
-    for(String decodedPath : decodedPaths) {
+    for (String decodedPath : decodedPaths) {
       sb.append(", ").append(org.ojai.FieldPath.parseFrom(decodedPath).asPathString());
     }
     String outputString = "[" + sb.substring(2) + "]";
diff --git a/contrib/format-maprdb/src/main/java/org/apache/drill/exec/udf/mapr/db/MatchesPlaceholder.java b/contrib/format-maprdb/src/main/java/org/apache/drill/exec/udf/mapr/db/MatchesPlaceholder.java
index 6aad44e..38f61c3 100644
--- a/contrib/format-maprdb/src/main/java/org/apache/drill/exec/udf/mapr/db/MatchesPlaceholder.java
+++ b/contrib/format-maprdb/src/main/java/org/apache/drill/exec/udf/mapr/db/MatchesPlaceholder.java
@@ -34,10 +34,9 @@ import org.apache.drill.exec.expr.holders.VarCharHolder;
  * which will replace this function with the real OJAI equivalent to be pushed down.
  * Therefore, there's no implementation here.
  */
-@FunctionTemplate(
-                name="ojai_matches",
-                scope=FunctionTemplate.FunctionScope.SIMPLE,
-                nulls=FunctionTemplate.NullHandling.INTERNAL)
+@FunctionTemplate(name = "ojai_matches",
+                scope = FunctionTemplate.FunctionScope.SIMPLE,
+                nulls = FunctionTemplate.NullHandling.INTERNAL)
 public class MatchesPlaceholder implements DrillSimpleFunc {
 
         @Param BigIntHolder /*FieldReader*/ field;
diff --git a/contrib/format-maprdb/src/main/java/org/apache/drill/exec/udf/mapr/db/NotMatchesPlaceholder.java b/contrib/format-maprdb/src/main/java/org/apache/drill/exec/udf/mapr/db/NotMatchesPlaceholder.java
index 56baebb..248579f 100644
--- a/contrib/format-maprdb/src/main/java/org/apache/drill/exec/udf/mapr/db/NotMatchesPlaceholder.java
+++ b/contrib/format-maprdb/src/main/java/org/apache/drill/exec/udf/mapr/db/NotMatchesPlaceholder.java
@@ -34,10 +34,9 @@ import org.apache.drill.exec.expr.holders.VarCharHolder;
  * which will replace this function with the real OJAI equivalent to be pushed down.
  * Therefore, there's no implementation here.
  */
-@FunctionTemplate(
-                name="ojai_notmatches",
-                scope=FunctionTemplate.FunctionScope.SIMPLE,
-                nulls=FunctionTemplate.NullHandling.INTERNAL)
+@FunctionTemplate(name = "ojai_notmatches",
+                scope = FunctionTemplate.FunctionScope.SIMPLE,
+                nulls = FunctionTemplate.NullHandling.INTERNAL)
 public class NotMatchesPlaceholder implements DrillSimpleFunc {
 
         @Param BigIntHolder /*FieldReader*/ field;
diff --git a/contrib/format-maprdb/src/main/java/org/apache/drill/exec/udf/mapr/db/NotTypeOfPlaceholder.java b/contrib/format-maprdb/src/main/java/org/apache/drill/exec/udf/mapr/db/NotTypeOfPlaceholder.java
index 6c01a48..78abcc0 100644
--- a/contrib/format-maprdb/src/main/java/org/apache/drill/exec/udf/mapr/db/NotTypeOfPlaceholder.java
+++ b/contrib/format-maprdb/src/main/java/org/apache/drill/exec/udf/mapr/db/NotTypeOfPlaceholder.java
@@ -34,10 +34,9 @@ import org.apache.drill.exec.expr.holders.IntHolder;
  * which will replace this function with the real OJAI equivalent to be pushed down.
  * Therefore, there's no implementation here.
  */
-@FunctionTemplate(
-                name="ojai_nottypeof",
-                scope=FunctionTemplate.FunctionScope.SIMPLE,
-                nulls=FunctionTemplate.NullHandling.INTERNAL)
+@FunctionTemplate(name = "ojai_nottypeof",
+                scope = FunctionTemplate.FunctionScope.SIMPLE,
+                nulls = FunctionTemplate.NullHandling.INTERNAL)
 public class NotTypeOfPlaceholder implements DrillSimpleFunc {
 
         @Param BigIntHolder /*FieldReader*/ field;
diff --git a/contrib/format-maprdb/src/main/java/org/apache/drill/exec/udf/mapr/db/SizeOfPlaceholder.java b/contrib/format-maprdb/src/main/java/org/apache/drill/exec/udf/mapr/db/SizeOfPlaceholder.java
index 7d7150c..8fcfee8 100644
--- a/contrib/format-maprdb/src/main/java/org/apache/drill/exec/udf/mapr/db/SizeOfPlaceholder.java
+++ b/contrib/format-maprdb/src/main/java/org/apache/drill/exec/udf/mapr/db/SizeOfPlaceholder.java
@@ -34,10 +34,9 @@ import org.apache.drill.exec.expr.holders.VarCharHolder;
  * which will replace this function with the real OJAI equivalent to be pushed down.
  * Therefore, there's no implementation here.
  */
-@FunctionTemplate(
-                name="ojai_sizeof",
-                scope=FunctionTemplate.FunctionScope.SIMPLE,
-                nulls=FunctionTemplate.NullHandling.INTERNAL)
+@FunctionTemplate(name = "ojai_sizeof",
+                scope = FunctionTemplate.FunctionScope.SIMPLE,
+                nulls = FunctionTemplate.NullHandling.INTERNAL)
 public class SizeOfPlaceholder implements DrillSimpleFunc {
 
         @Param BigIntHolder /*FieldReader*/ field;
diff --git a/contrib/format-maprdb/src/main/java/org/apache/drill/exec/udf/mapr/db/TypeOfPlaceholder.java b/contrib/format-maprdb/src/main/java/org/apache/drill/exec/udf/mapr/db/TypeOfPlaceholder.java
index 1d1efc0..585404f 100644
--- a/contrib/format-maprdb/src/main/java/org/apache/drill/exec/udf/mapr/db/TypeOfPlaceholder.java
+++ b/contrib/format-maprdb/src/main/java/org/apache/drill/exec/udf/mapr/db/TypeOfPlaceholder.java
@@ -34,10 +34,9 @@ import org.apache.drill.exec.expr.holders.IntHolder;
  * which will replace this function with the real OJAI equivalent to be pushed down.
  * Therefore, there's no implementation here.
  */
-@FunctionTemplate(
-                name="ojai_typeof",
-                scope=FunctionTemplate.FunctionScope.SIMPLE,
-                nulls=FunctionTemplate.NullHandling.INTERNAL)
+@FunctionTemplate(name = "ojai_typeof",
+                scope = FunctionTemplate.FunctionScope.SIMPLE,
+                nulls = FunctionTemplate.NullHandling.INTERNAL)
 public class TypeOfPlaceholder implements DrillSimpleFunc {
 
         @Param BigIntHolder /*FieldReader*/ field;
diff --git a/contrib/format-maprdb/src/test/java/com/mapr/drill/maprdb/tests/index/IndexHintPlanTest.java b/contrib/format-maprdb/src/test/java/com/mapr/drill/maprdb/tests/index/IndexHintPlanTest.java
index 9ac27b4..c6ec0fe 100644
--- a/contrib/format-maprdb/src/test/java/com/mapr/drill/maprdb/tests/index/IndexHintPlanTest.java
+++ b/contrib/format-maprdb/src/test/java/com/mapr/drill/maprdb/tests/index/IndexHintPlanTest.java
@@ -46,7 +46,7 @@ public class IndexHintPlanTest extends IndexPlanTest {
                 new String[]{"RowKeyJoin"}
         );
 
-        //default plan picked by optimizer.
+        // default plan picked by optimizer.
         PlanTestBase.testPlanMatchingPatterns(query,
                 new String[] {".*JsonTableGroupScan.*tableName=.*index_test_primary.*indexName=i_ssn"},
                 new String[]{"RowKeyJoin"}
@@ -75,13 +75,12 @@ public class IndexHintPlanTest extends IndexPlanTest {
                 new String[]{"RowKeyJoin"}
         );
 
-        //default plan picked by optimizer
+        // default plan picked by optimizer
         PlanTestBase.testPlanMatchingPatterns(query,
                 new String[] {".*JsonTableGroupScan.*tableName=.*index_test_primary.*indexName=(i_state_city|i_state_age_phone)"},
                 new String[]{"RowKeyJoin"}
         );
 
-        return;
     }
 
     @Test
@@ -101,21 +100,20 @@ public class IndexHintPlanTest extends IndexPlanTest {
                 new String[]{"RowKeyJoin"}
         );
 
-        //default plan picked by query optimizer.
+        // default plan picked by query optimizer.
         PlanTestBase.testPlanMatchingPatterns(query,
                 new String[] {".*JsonTableGroupScan.*tableName=.*index_test_primary.*indexName=(i_state_city|i_state_age_phone)"},
                 new String[]{"RowKeyJoin"}
         );
 
-        return;
     }
 
-    //Negative cases
+    // Negative cases
 
     @Test
     // A testcase where there are multiple index to pick from but none of them equals to the index provided as hint (index hint is wrong).
-    //In this index is not at all present in the table hence it falls back to the case where the index itself is not given.
-    //Hence here one of the i_state_city or i_state_age_lic will be selected depending upon the cost.
+    // In this index is not at all present in the table hence it falls back to the case where the index itself is not given.
+    // Hence here one of the i_state_city or i_state_age_lic will be selected depending upon the cost.
     public void testWithMultipleIndexesButNoIndexWithHint() throws Exception {
 
         String hintquery = "SELECT t.`address`.`state` AS `state` FROM table(hbase.`index_test_primary`(type => 'maprdb', index => 'i_state_and_city')) as t " +
@@ -126,7 +124,6 @@ public class IndexHintPlanTest extends IndexPlanTest {
                 new String[]{"RowKeyJoin"}
         );
 
-        return;
     }
 
     @Test
@@ -150,12 +147,11 @@ public class IndexHintPlanTest extends IndexPlanTest {
                 new String[]{"RowKeyJoin"}
         );
 
-        return;
     }
 
 
     @Test
-    //Covering index should be generated for a simple query instead of a RowKeyJoin.
+    // Covering index should be generated for a simple query instead of a RowKeyJoin.
     public void testSimpleNoRowKeyJoin() throws Exception {
         String query = "SELECT `reverseid` from table(hbase.`index_test_primary`(type => 'maprdb', index => 'hash_i_reverseid'))  " +
                 "where `reverseid` = 1234";
@@ -166,6 +162,5 @@ public class IndexHintPlanTest extends IndexPlanTest {
                 new String[]{"RowKeyJoin"}
         );
 
-        return;
    }
 }
diff --git a/contrib/format-maprdb/src/test/java/com/mapr/drill/maprdb/tests/index/IndexPlanTest.java b/contrib/format-maprdb/src/test/java/com/mapr/drill/maprdb/tests/index/IndexPlanTest.java
index c0ea2a0..a9de9e3 100644
--- a/contrib/format-maprdb/src/test/java/com/mapr/drill/maprdb/tests/index/IndexPlanTest.java
+++ b/contrib/format-maprdb/src/test/java/com/mapr/drill/maprdb/tests/index/IndexPlanTest.java
@@ -146,14 +146,11 @@ public class IndexPlanTest extends BaseJsonTest {
         new String[]{"RowKeyJoin"}
     );
 
-    System.out.println("Covering Plan Verified!");
-
     testBuilder()
         .sqlQuery(query)
         .ordered()
         .baselineColumns("phone").baselineValues("6500005471")
         .go();
-    return;
 
   }
 
@@ -167,8 +164,6 @@ public class IndexPlanTest extends BaseJsonTest {
         new String[]{"RowKeyJoin"}
     );
 
-    System.out.println("Covering Plan Verified!");
-
     testBuilder()
         .optionSettingQueriesForTestQuery(defaultHavingIndexPlan)
         .sqlQuery(query)
@@ -176,7 +171,6 @@ public class IndexPlanTest extends BaseJsonTest {
         .baselineColumns("ssn").baselineValues("100007423")
         .go();
 
-    return;
   }
 
   @Test
@@ -190,8 +184,6 @@ public class IndexPlanTest extends BaseJsonTest {
         new String[]{"RowKeyJoin", "indexName="}
     );
 
-    System.out.println("No Index Plan Verified!");
-
     testBuilder()
         .sqlQuery(query)
         .unOrdered()
@@ -199,7 +191,6 @@ public class IndexPlanTest extends BaseJsonTest {
         .baselineColumns("ssn").baselineValues("100007632")
         .go();
 
-    return;
   }
 
   @Test
@@ -214,15 +205,12 @@ public class IndexPlanTest extends BaseJsonTest {
         new String[]{}
     );
 
-    System.out.println("Non-Covering Plan Verified!");
-
     testBuilder()
         .sqlQuery(query)
         .ordered()
         .baselineColumns("fname").baselineValues("KfFzK")
         .go();
 
-    return;
   }
 
   @Test
@@ -276,7 +264,6 @@ public class IndexPlanTest extends BaseJsonTest {
         .baselineColumns("tid").baselineValues("1012")
         .go();
 
-    return;
   }
 
   @Test
@@ -298,7 +285,6 @@ public class IndexPlanTest extends BaseJsonTest {
         .baselineColumns("rowid").baselineValues("1012")
         .go();
 
-    return;
   }
 
   @Test
@@ -322,7 +308,7 @@ public class IndexPlanTest extends BaseJsonTest {
     } finally {
       test(defaultRowKeyJoinBackIOFactor);
     }
-    return;
+
   }
 
   @Test
@@ -356,7 +342,7 @@ public class IndexPlanTest extends BaseJsonTest {
     } finally {
       test(defaultIntersectPlans + ";" + enableFTS);
     }
-    return;
+
   }
 
   @Test
@@ -391,7 +377,7 @@ public class IndexPlanTest extends BaseJsonTest {
     } finally {
       test(defaultRowKeyJoinBackIOFactor);
     }
-    return;
+
   }
 
   @Test//filter cover indexed, included and not in index at all filter
@@ -420,8 +406,8 @@ public class IndexPlanTest extends BaseJsonTest {
         .build()
         .run();
 
-    return;
   }
+
   @Test
   public void CompositeIndexCoveringPlan() throws Exception {
 
@@ -448,7 +434,7 @@ public class IndexPlanTest extends BaseJsonTest {
         .sqlBaselineQuery(query)
         .build()
         .run();
-    return;
+
   }
 
   @Test
@@ -477,12 +463,12 @@ public class IndexPlanTest extends BaseJsonTest {
       test(defaultHavingIndexPlan);
       test(sliceTargetDefault);
     }
-    return;
+
   }
 
   @Test
   public void TestCastVarCharCoveringPlan() throws Exception {
-    //length 255 is to exact match the casted indexed field's length
+    // length 255 is to exact match the casted indexed field's length
     String query = "SELECT t._id as tid, cast(t.driverlicense as varchar(255)) as driverlicense FROM hbase.`index_test_primary` as t " +
         " where cast(t.driverlicense as varchar(255))='100007423'";
     test(defaultHavingIndexPlan);
@@ -491,8 +477,6 @@ public class IndexPlanTest extends BaseJsonTest {
         new String[]{"RowKeyJoin"}
     );
 
-    System.out.println("TestCastCoveringPlan Plan Verified!");
-
     testBuilder()
         .optionSettingQueriesForTestQuery(defaultHavingIndexPlan)
         .sqlQuery(query)
@@ -500,7 +484,6 @@ public class IndexPlanTest extends BaseJsonTest {
         .baselineColumns("tid", "driverlicense").baselineValues("1012", "100007423")
         .go();
 
-    return;
   }
 
   @Test
@@ -513,8 +496,6 @@ public class IndexPlanTest extends BaseJsonTest {
         new String[]{"RowKeyJoin"}
     );
 
-    System.out.println("TestCastCoveringPlan Plan Verified!");
-
     testBuilder()
         .optionSettingQueriesForTestQuery(defaultHavingIndexPlan)
         .sqlQuery(query)
@@ -522,7 +503,6 @@ public class IndexPlanTest extends BaseJsonTest {
         .baselineColumns("tid", "ssn", "phone").baselineValues("1012", 100007423, "6500005471")
         .go();
 
-    return;
   }
 
   @Test
@@ -535,14 +515,12 @@ public class IndexPlanTest extends BaseJsonTest {
         new String[]{}
     );
 
-    System.out.println("TestCastNonCoveringPlan Plan Verified!");
-
     testBuilder()
         .sqlQuery(query)
         .ordered()
         .baselineColumns("ssn").baselineValues("100007423")
         .go();
-    return;
+
   }
 
   @Test
@@ -555,15 +533,12 @@ public class IndexPlanTest extends BaseJsonTest {
         new String[]{}
     );
 
-    System.out.println("TestCastVarchar_ConvertToRangePlan Verified!");
-
     testBuilder()
         .sqlQuery(query)
         .ordered()
         .baselineColumns("ssn").baselineValues("100007423")
         .go();
 
-    return;
   }
 
   @Test // cast expression in filter is not indexed, but the same field casted to different type was indexed (CAST id.ssn as INT)
@@ -579,7 +554,7 @@ public class IndexPlanTest extends BaseJsonTest {
 
   @Test
   public void TestLongerCastVarCharNoIndex() throws Exception {
-    //length 256 is to exact match the casted indexed field's length
+    // length 256 is to exact match the casted indexed field's length
     String query = "SELECT t._id as tid, cast(t.driverlicense as varchar(500)) as driverlicense FROM hbase.`index_test_primary` as t " +
         " where cast(t.driverlicense as varchar(500))='100007423'";
     test(defaultHavingIndexPlan);
@@ -588,9 +563,6 @@ public class IndexPlanTest extends BaseJsonTest {
         new String[]{"RowKeyJoin", "indexName="}
     );
 
-    System.out.println("TestLongerCastVarCharNoIndex Plan Verified!");
-
-    return;
   }
 
   @Test
@@ -666,7 +638,7 @@ public class IndexPlanTest extends BaseJsonTest {
         new String[]{"Sort"}
     );
 
-    //simple field, driverlicense
+    // simple field, driverlicense
     testBuilder()
         .sqlQuery(query)
         .ordered()
@@ -675,7 +647,7 @@ public class IndexPlanTest extends BaseJsonTest {
         .baselineColumns("phone").baselineValues("6500001595")
         .go();
 
-    //query on field of item expression(having capProject), non-simple field t.id.ssn
+    // query on field of item expression(having capProject), non-simple field t.id.ssn
     testBuilder()
         .sqlQuery(query2)
         .ordered()
@@ -696,7 +668,7 @@ public class IndexPlanTest extends BaseJsonTest {
     }
   }
 
-  //test cases are from TestNonCoveringPlanSortRemoved. Sort was removed when force_sort_noncovering was default(false)
+  // test cases are from TestNonCoveringPlanSortRemoved. Sort was removed when force_sort_noncovering was default(false)
   @Test
   public void TestNonCoveringPlanWithNoRemoveSortOption() throws Exception {
     try {
@@ -717,7 +689,7 @@ public class IndexPlanTest extends BaseJsonTest {
           new String[]{}
       );
 
-      //simple field, driverlicense
+      // simple field, driverlicense
       testBuilder()
           .sqlQuery(query)
           .ordered()
@@ -726,7 +698,7 @@ public class IndexPlanTest extends BaseJsonTest {
           .baselineColumns("phone").baselineValues("6500001595")
           .go();
 
-      //query on field of item expression(having capProject), non-simple field t.id.ssn
+      // query on field of item expression(having capProject), non-simple field t.id.ssn
       testBuilder()
           .sqlQuery(query2)
           .ordered()
@@ -795,7 +767,7 @@ public class IndexPlanTest extends BaseJsonTest {
         " where t.address.state = 'wo' and t.personal.age = 35 and t.contact.phone < '6500003000' order by t.contact.phone";
     test(defaultHavingIndexPlan);
 
-    //we should glue to index i_state_age_phone to make sure we are testing the targeted prefix construction code path
+    // we should glue to index i_state_age_phone to make sure we are testing the targeted prefix construction code path
     PlanTestBase.testPlanMatchingPatterns(query,
         new String[] {".*JsonTableGroupScan.*tableName=.*index_test_primary.*indexName=i_state_age_phone"},
         new String[]{"Sort"}
@@ -833,7 +805,7 @@ public class IndexPlanTest extends BaseJsonTest {
       .run();
   }
 
-  @Test  //ORDER BY last two columns not in the indexed order; Sort SHOULD NOT be dropped
+  @Test  // ORDER BY last two columns not in the indexed order; Sort SHOULD NOT be dropped
   public void TestCoveringPlanSortPrefix_3() throws Exception {
     String query = "SELECT CAST(t.personal.age as VARCHAR) as age, t.contact.phone FROM hbase.`index_test_primary` as t " +
         " where t.address.state = 'wo' and t.personal.age < 35 and t.contact.phone < '6500003000' order by t.contact.phone, t.personal.age";
@@ -976,7 +948,7 @@ public class IndexPlanTest extends BaseJsonTest {
         .go();
   }
 
-  @Test //non-covering plan. order by cast indexed field, sort SHOULD be removed
+  @Test // non-covering plan. order by cast indexed field, sort SHOULD be removed
   public void orderByCastNonCoveringPlan() throws Exception {
     String query = "SELECT t.name.lname as lname FROM hbase.`index_test_primary` as t " +
         " where CAST(t.id.ssn as INT) < 100000003 order by CAST(t.id.ssn as INT)";
@@ -996,8 +968,8 @@ public class IndexPlanTest extends BaseJsonTest {
   }
 
 
-  @Ignore //in statsCache, condition state+city has rowcount 1250, but state only has 1000. so it is picking i_state_age_phone
-  @Test //non-covering, order by non leading field, and leading fields are not in equality condition, Sort SHOULD NOT be removed
+  @Ignore // in statsCache, condition state+city has rowcount 1250, but state only has 1000. so it is picking i_state_age_phone
+  @Test // non-covering, order by non leading field, and leading fields are not in equality condition, Sort SHOULD NOT be removed
   public void NonCoveringPlan_SortPrefix_1() throws Exception {
 
     String query = "SELECT t.`id`.`ssn` AS `ssn` FROM hbase.`index_test_primary` as t " +
@@ -1008,10 +980,10 @@ public class IndexPlanTest extends BaseJsonTest {
             "RowKeyJoin(.*[\n\r])+.*RestrictedJsonTableGroupScan(.*[\n\r])+.*JsonTableGroupScan.*indexName=i_state_city"},
         new String[]{}
     );
-    return;
+
   }
 
-  @Test //non-covering, order by non leading field, and leading fields are in equality condition, Sort SHOULD be removed
+  @Test // non-covering, order by non leading field, and leading fields are in equality condition, Sort SHOULD be removed
   public void NonCoveringPlan_SortPrefix_2() throws Exception {
 
     String query = "SELECT t.`id`.`ssn` AS `ssn` FROM hbase.`index_test_primary` as t " +
@@ -1022,11 +994,11 @@ public class IndexPlanTest extends BaseJsonTest {
             "RowKeyJoin(.*[\n\r])+.*RestrictedJsonTableGroupScan(.*[\n\r])+.*JsonTableGroupScan.*indexName=i_state_city"},
         new String[]{"Sort"}
     );
-    return;
+
   }
 
   @Ignore ("Should be modified to get an index plan; not very useful since most covering plan filters get pushed")
-  @Test //Correct projection and results when filter on non-indexed column in covering plan.
+  @Test // Correct projection and results when filter on non-indexed column in covering plan.
   public void nonIndexedColumnFilterCoveringPlan() throws Exception {
     String query = "SELECT t.name.fname as fname FROM hbase.`index_test_primary` as t " +
         " where t.personal.age > 68 and t.name.fname IN ('CnGobfR', 'THOHP')";
@@ -1046,7 +1018,7 @@ public class IndexPlanTest extends BaseJsonTest {
   }
 
   @Test
-  @Ignore ("Fix after MEP 5.0")
+  @Ignore
   public void orderByLimitNonCoveringPlan() throws Exception {
     String query = "SELECT t.name.lname as lname FROM hbase.`index_test_primary` as t " +
         " where t.id.ssn < '100000003' order by t.id.ssn limit 2";
@@ -1260,7 +1232,7 @@ public class IndexPlanTest extends BaseJsonTest {
     String query = "SELECT cast(t.activity.irs.firstlogin as timestamp) AS `firstlogin`, t.id.ssn as ssn FROM hbase.`index_test_primary` as t " +
         "order by cast(t.activity.irs.firstlogin as timestamp), t.id.ssn limit 2";
     test(defaultHavingIndexPlan);
-    //no collation for hash index so Sort or TopN must have been preserved
+    // no collation for hash index so Sort or TopN must have been preserved
     PlanTestBase.testPlanMatchingPatterns(query,
         new String[] {"(Sort|TopN)"},
         new String[]{"indexName="}
@@ -1295,7 +1267,7 @@ public class IndexPlanTest extends BaseJsonTest {
         .run();
   }
 
-  @Test //negative case for no filter plan
+  @Test // negative case for no filter plan
   public void testNoFilterOrderByNoIndexMatch() throws Exception {
     String query = "SELECT t.`id`.`ssn` AS `ssn`, t.contact.phone as phone FROM hbase.`index_test_primary` as t " +
         "order by t.name.fname limit 2";
@@ -1306,27 +1278,30 @@ public class IndexPlanTest extends BaseJsonTest {
     );
   }
 
-// Enable this testcase once MD-2848 is fixed.
-//  @Test
-//  public void IntersectPlanWithOneSideNoRows() throws Exception {
-//    try {
-//      String query = "SELECT t.`name`.`lname` AS `lname` FROM hbase.`index_test_primary` as t " +
-//              " where t.personal.age = 53 AND t.personal.income=111145";
-//      test(defaultHavingIndexPlan);
-//      test(preferIntersectPlans + ";" + disableFTS);
-//      PlanTestBase.testPlanMatchingPatterns(query,
-//              new String[]{"RowKeyJoin(.*[\n\r])+.*RestrictedJsonTableGroupScan(.*[\n\r])+.*HashJoin(.*[\n\r])+.*JsonTableGroupScan.*indexName=(i_age|i_income)(.*[\n\r])+.*JsonTableGroupScan.*indexName=(i_age|i_income)"},
-//              new String[]{}
-//      );
-//
-//      testNoResult(query);
-//
-//    } finally {
-//      test(defaultIntersectPlans + ";" + enableFTS);
-//    }
-//  }
-
-  //"i_cast_age_state_phone", "$CAST(personal.age@STRING),address.state,contact.phone", "name.fname",
+// This test case encounters an error :
+//  "Error: SYSTEM ERROR: IllegalStateException: Batch data read operation (iterator()) attempted when last
+//                next() call on batch [#16, ScanBatch] returned NONE (not OK or OK_NEW_SCHEMA)."
+// TODO: fix the root cause of the above error then enable the test
+  @Test
+  @Ignore
+  public void IntersectPlanWithOneSideNoRows() throws Exception {
+    try {
+      String query = "SELECT t.`name`.`lname` AS `lname` FROM hbase.`index_test_primary` as t " +
+              " where t.personal.age = 53 AND t.personal.income=111145";
+      test(defaultHavingIndexPlan);
+      test(preferIntersectPlans + ";" + disableFTS);
+      PlanTestBase.testPlanMatchingPatterns(query,
+              new String[]{"RowKeyJoin(.*[\n\r])+.*RestrictedJsonTableGroupScan(.*[\n\r])+.*HashJoin(.*[\n\r])+.*JsonTableGroupScan.*indexName=(i_age|i_income)(.*[\n\r])+.*JsonTableGroupScan.*indexName=(i_age|i_income)"},
+              new String[]{}
+      );
+
+      testNoResult(query);
+
+    } finally {
+      test(defaultIntersectPlans + ";" + enableFTS);
+    }
+  }
+
   @Test
   public void testTrailingFieldIndexCovering() throws Exception {
     String query = "SELECT t.`name`.`fname` AS `fname` FROM hbase.`index_test_primary` as t " +
@@ -1372,7 +1347,7 @@ public class IndexPlanTest extends BaseJsonTest {
     try {
       test(defaultHavingIndexPlan);
       test(disableHashAgg);
-      //no collation for hash index so Sort or TopN must have been preserved
+      // no collation for hash index so Sort or TopN must have been preserved
       PlanTestBase.testPlanMatchingPatterns(query,
               new String[]{"indexName=i_lic", "StreamAgg"},
               new String[]{"(Sort|TopN)"}
@@ -1396,7 +1371,7 @@ public class IndexPlanTest extends BaseJsonTest {
     String query = " select t1.driverlicense from hbase.`index_test_primary` t1" +
             " order by t1.driverlicense desc limit 2";
     test(defaultHavingIndexPlan);
-    //no collation for hash index so Sort or TopN must have been preserved
+    // no collation for hash index so Sort or TopN must have been preserved
     PlanTestBase.testPlanMatchingPatterns(query,
             new String[] {"(Sort|TopN)"},
             new String[]{"indexName="}
@@ -1418,7 +1393,7 @@ public class IndexPlanTest extends BaseJsonTest {
     try {
       test(defaultHavingIndexPlan);
       test(disableHashAgg);
-      //no collation for hash index so Sort or TopN must have been preserved
+      // no collation for hash index so Sort or TopN must have been preserved
       PlanTestBase.testPlanMatchingPatterns(query,
               new String[]{"indexName=i_lic", "StreamAgg"},
               new String[]{"(Sort|TopN)"}
@@ -1492,7 +1467,7 @@ public class IndexPlanTest extends BaseJsonTest {
     try {
       test(defaultHavingIndexPlan);
       test(disableHashAgg);
-      //no collation for hash index so Sort or TopN must have been preserved
+      // no collation for hash index so Sort or TopN must have been preserved
       PlanTestBase.testPlanMatchingPatterns(query,
               new String[]{"(Sort|TopN)", "StreamAgg"},
               new String[]{"indexName="}
@@ -1539,7 +1514,7 @@ public class IndexPlanTest extends BaseJsonTest {
     }
   }
 
-  @Test //negative case for no filter plan
+  @Test // negative case for no filter plan
   public void testNoFilterGroupByNoIndexMatch() throws Exception {
     String query = "SELECT max(t.`id`.`ssn`) AS `ssn`, max(t.contact.phone) as phone FROM hbase.`index_test_primary` as t " +
             "group by t.name.fname limit 2";
diff --git a/contrib/format-maprdb/src/test/java/com/mapr/drill/maprdb/tests/index/LargeTableGen.java b/contrib/format-maprdb/src/test/java/com/mapr/drill/maprdb/tests/index/LargeTableGen.java
index bc857d1..347f7f2 100644
--- a/contrib/format-maprdb/src/test/java/com/mapr/drill/maprdb/tests/index/LargeTableGen.java
+++ b/contrib/format-maprdb/src/test/java/com/mapr/drill/maprdb/tests/index/LargeTableGen.java
@@ -67,7 +67,7 @@ public class LargeTableGen extends LargeTableGenBase {
   Table createOrGetTable(String tableName, int recordNum) {
     if (admin.tableExists(tableName)) {
       return MapRDBImpl.getTable(tableName);
-      //admin.deleteTable(tableName);
+      // admin.deleteTable(tableName);
     }
     else {
       TableDescriptor desc = new TableDescriptorImpl(new Path(tableName));
@@ -76,7 +76,7 @@ public class LargeTableGen extends LargeTableGenBase {
 
       String[] splitsStr = new String[splits];
       StringBuilder strBuilder = new StringBuilder("Splits:");
-      for(int i=0; i<splits; ++i) {
+      for (int i = 0; i < splits; ++i) {
         splitsStr[i] = String.format("%d", (i+1)*SPLIT_SIZE);
         strBuilder.append(splitsStr[i] + ", ");
       }
@@ -87,20 +87,20 @@ public class LargeTableGen extends LargeTableGenBase {
   }
 
   private void createIndex(Table table, String[] indexDef) throws Exception {
-    if(indexDef == null) {
-      //don't create index here. indexes may have been created
+    if (indexDef == null) {
+      // don't create index here. indexes may have been created
       return;
     }
-    for(int i=0; i<indexDef.length / 3; ++i) {
+    for (int i = 0; i < indexDef.length / 3; ++i) {
       String indexCmd = String.format("maprcli table index add"
           + " -path " + table.getPath()
           + " -index %s"
           + " -indexedfields '%s'"
           + ((indexDef[3 * i + 2].length()==0)?"":" -includedfields '%s'")
           + ((indexDef[3 * i].startsWith("hash"))? " -hashed true" : ""),
-          indexDefInCommand(indexDef[3 * i]), //index name
-          indexDefInCommand(indexDef[3 * i + 1]), //indexedfields
-          indexDefInCommand(indexDef[3 * i + 2])); //includedfields
+          indexDefInCommand(indexDef[3 * i]), // index name
+          indexDefInCommand(indexDef[3 * i + 1]), // indexedfields
+          indexDefInCommand(indexDef[3 * i + 2])); // includedfields
       System.out.println(indexCmd);
 
       TestCluster.runCommand(indexCmd);
@@ -111,8 +111,8 @@ public class LargeTableGen extends LargeTableGenBase {
   private String indexDefInCommand(String def) {
     String[] splitted = def.split(",");
     StringBuffer ret = new StringBuffer();
-    for(String field: splitted) {
-      if(ret.length() == 0) {
+    for (String field: splitted) {
+      if (ret.length() == 0) {
         ret.append(field);
       }
       else {
@@ -129,14 +129,14 @@ public class LargeTableGen extends LargeTableGenBase {
     DBTests.setTableStatsSendInterval(1);
 
     if (admin.tableExists(tablePath)) {
-      //admin.deleteTable(tablePath);
+      // admin.deleteTable(tablePath);
     }
 
-    //create Json String
+    // create Json String
     int batch, i;
     int BATCH_SIZE=2000;
     try (Table table = createOrGetTable(tablePath, recordNumber)) {
-      //create index
+      // create index
       createIndex(table, indexDef);
       for (batch = 0; batch < recordNumber; batch += BATCH_SIZE) {
         int batchStop = Math.min(recordNumber, batch + BATCH_SIZE);
@@ -156,13 +156,9 @@ public class LargeTableGen extends LargeTableGenBase {
         }
         try (InputStream in = new StringBufferInputStream(strBuf.toString());
              DocumentStream stream = Json.newDocumentStream(in)) {
-          //write by individual document
-          //for (Document document : stream) {
-          //  table.insert(document, "rowid");
-          //}
           try {
-            table.insert(stream, "rowid"); //insert a batch  of document in stream
-          }catch(Exception e) {
+            table.insert(stream, "rowid"); // insert a batch  of document in stream
+          } catch(Exception e) {
             System.out.println(stream.toString());
             throw e;
           }
diff --git a/contrib/format-maprdb/src/test/java/com/mapr/drill/maprdb/tests/index/LargeTableGenBase.java b/contrib/format-maprdb/src/test/java/com/mapr/drill/maprdb/tests/index/LargeTableGenBase.java
index 917f42a..bbba08f 100644
--- a/contrib/format-maprdb/src/test/java/com/mapr/drill/maprdb/tests/index/LargeTableGenBase.java
+++ b/contrib/format-maprdb/src/test/java/com/mapr/drill/maprdb/tests/index/LargeTableGenBase.java
@@ -46,45 +46,44 @@ public class LargeTableGenBase {
       cities = new ArrayList<>();
       List<String> states = new ArrayList<>();
 
-      int fnNum = 2000; //2k
-      int lnNum = 200000;//200k
-      int cityNum = 10000;//10k
+      int fnNum = 2000;     // 2k
+      int lnNum = 200000;   // 200k
+      int cityNum = 10000;  // 10k
       int stateNum = 50;
       Random rand = new Random(2017);
       int i;
       try {
         Set<String> strSet = new LinkedHashSet<>();
-        while(strSet.size() < stateNum) {
+        while (strSet.size() < stateNum) {
           strSet.add(RandomStringUtils.random(2, 0, 0, true, false, null, rand));
         }
         states.addAll(strSet);
 
         strSet = new LinkedHashSet<>();
-        while(strSet.size() < cityNum) {
+        while (strSet.size() < cityNum) {
           int len = 3 + strSet.size() % 6;
           strSet.add(RandomStringUtils.random(len, 0, 0, true, false, null, rand));
         }
 
         Iterator<String> it = strSet.iterator();
-        for(i=0; i<cityNum; ++i) {
+        for (i = 0; i < cityNum; ++i) {
           cities.add(new String[]{"10000", states.get(i%stateNum),  it.next()});
         }
 
         strSet = new LinkedHashSet<>();
-        while(strSet.size() < fnNum) {
+        while (strSet.size() < fnNum) {
           int len = 3 + strSet.size() % 6;
           strSet.add(RandomStringUtils.random(len, 0, 0, true, false, null, rand));
         }
         firstnames.addAll(strSet);
 
         strSet = new LinkedHashSet<>();
-        while(strSet.size() < lnNum) {
+        while (strSet.size() < lnNum) {
           int len = 3 + strSet.size() % 6;
           strSet.add(RandomStringUtils.random(len, 0, 0, true, false, null, rand));
         }
         lastnames.addAll(strSet);
-      }
-      catch(Exception e) {
+      } catch(Exception e) {
         System.out.println("init data got exception");
         e.printStackTrace();
       }
@@ -109,7 +108,7 @@ public class LargeTableGenBase {
   }
 
   protected String getPhone(int i) {
-    //80% phones are unique,
+    // 80% phones are unique,
     return String.format("%d", 6500*1000*1000L + randomized[ (randomized.length - i) %((int) (randomized.length * 0.8)) ]);
   }
 
@@ -125,14 +124,14 @@ public class LargeTableGenBase {
     return String.format("%d",randomized[i%randomized.length] % 47 + 1);
   }
 
-  //date yyyy-mm-dd
+  // date yyyy-mm-dd
   protected String getBirthdate(int i) {
     int thisseed = randomized[i%randomized.length];
     return String.format("%d-%02d-%02d",
         2016 - (thisseed % 60 + 10), thisseed % 12 + 1, (thisseed * 31) % 28 + 1 );
   }
 
-  //timestamp, yyyy-mm-dd HH:mm:ss
+  // timestamp, yyyy-mm-dd HH:mm:ss
   protected String getFirstLogin(int i) {
     int thisseed = randomized[i%randomized.length];
     int nextseed = randomized[(i+1)%randomized.length];
diff --git a/contrib/format-maprdb/src/test/java/com/mapr/drill/maprdb/tests/index/StatisticsTest.java b/contrib/format-maprdb/src/test/java/com/mapr/drill/maprdb/tests/index/StatisticsTest.java
index 36e25ab..20f1090 100644
--- a/contrib/format-maprdb/src/test/java/com/mapr/drill/maprdb/tests/index/StatisticsTest.java
+++ b/contrib/format-maprdb/src/test/java/com/mapr/drill/maprdb/tests/index/StatisticsTest.java
@@ -56,8 +56,7 @@ public class StatisticsTest extends IndexPlanTest {
         + " where (t.personal.age < 30 or t.personal.age > 100)"
         + " and (t.address.state = 'mo' or t.address.state = 'ca')";
     PlanTestBase.testPlanMatchingPatterns(explain+query,
-        new String[] {".*JsonTableGroupScan.*tableName=.*index_test_primary.*rows=10000"},
-        new String[] {}
+        new String[] {".*JsonTableGroupScan.*tableName=.*index_test_primary.*rows=10000"}
     );
 
     // Top-level ORs - Cannot split top-level ORs so use defaults
@@ -65,8 +64,7 @@ public class StatisticsTest extends IndexPlanTest {
         + " where (t.personal.age > 30 and t.personal.age < 100)"
         + " or (t.address.state = 'mo')";
     PlanTestBase.testPlanMatchingPatterns(explain+query,
-        new String[] {".*JsonTableGroupScan.*tableName=.*index_test_primary.*rows=10000"},
-        new String[] {}
+        new String[] {".*JsonTableGroupScan.*tableName=.*index_test_primary.*rows=10000"}
     );
 
     // ANDed condition - Leading index column(personal.age) and non-leading column(address.city)
@@ -74,8 +72,7 @@ public class StatisticsTest extends IndexPlanTest {
         + " where (t.personal.age < 30 or t.personal.age > 100)"
         + " and `address.city` = 'sf'";
     PlanTestBase.testPlanMatchingPatterns(explain+query,
-        new String[] {".*JsonTableGroupScan.*tableName=.*index_test_primary.*rows=10000"},
-        new String[] {}
+        new String[] {".*JsonTableGroupScan.*tableName=.*index_test_primary.*rows=10000"}
     );
 
     // ANDed condition - Leading index columns (address.state) and (address.city)
@@ -83,8 +80,7 @@ public class StatisticsTest extends IndexPlanTest {
         + " where (`address.state` = 'mo' or `address.state` = 'ca') " // Leading index column
         + " and `address.city` = 'sf'";                                // Non leading index column
     PlanTestBase.testPlanMatchingPatterns(explain+query,
-        new String[] {".*JsonTableGroupScan.*tableName=.*index_test_primary.*rows=10000"},
-        new String[] {}
+        new String[] {".*JsonTableGroupScan.*tableName=.*index_test_primary.*rows=10000"}
     );
 
     // ANDed condition - Leading index columns (address.state) and non-index column (name.fname)
@@ -92,24 +88,21 @@ public class StatisticsTest extends IndexPlanTest {
         + " where (`address.state` = 'mo' or `address.state` = 'ca') " // Leading index column
         + " and `name.fname` = 'VcFahj'";                              // Non index column
     PlanTestBase.testPlanMatchingPatterns(explain+query,
-        new String[] {".*JsonTableGroupScan.*tableName=.*index_test_primary.*rows=10000"},
-        new String[] {}
+        new String[] {".*JsonTableGroupScan.*tableName=.*index_test_primary.*rows=10000"}
     );
 
     // Simple condition - LIKE predicate
     query = "select t._id as rowid from hbase.`index_test_primary` as t "
         + "where t.driverlicense like '100007423%'";
     PlanTestBase.testPlanMatchingPatterns(explain+query,
-        new String[] {".*JsonTableGroupScan.*tableName=.*index_test_primary.*rows=10000"},
-        new String[] {}
+        new String[] {".*JsonTableGroupScan.*tableName=.*index_test_primary.*rows=10000"}
     );
 
     // Simple condition - LIKE predicate with ESCAPE clause
     query = "select t._id as rowid from hbase.`index_test_primary` as t "
         + "where t.driverlicense like '100007423%' ESCAPE '/'";
     PlanTestBase.testPlanMatchingPatterns(explain+query,
-        new String[] {".*JsonTableGroupScan.*tableName=.*index_test_primary.*rows=10000"},
-        new String[] {}
+        new String[] {".*JsonTableGroupScan.*tableName=.*index_test_primary.*rows=10000"}
     );
   }
 }
diff --git a/contrib/format-maprdb/src/test/java/com/mapr/drill/maprdb/tests/index/TableIndexCmd.java b/contrib/format-maprdb/src/test/java/com/mapr/drill/maprdb/tests/index/TableIndexCmd.java
index a501f8f..bf50195 100644
--- a/contrib/format-maprdb/src/test/java/com/mapr/drill/maprdb/tests/index/TableIndexCmd.java
+++ b/contrib/format-maprdb/src/test/java/com/mapr/drill/maprdb/tests/index/TableIndexCmd.java
@@ -71,12 +71,9 @@ public class TableIndexCmd {
   public static void pressEnterKeyToContinue()
   {
     System.out.println("Press any key to continue...");
-    try
-    {
+    try {
       System.in.read();
-    }
-    catch(Exception e)
-    {}
+    } catch(Exception e) {}
   }
 
 
@@ -90,36 +87,35 @@ public class TableIndexCmd {
     boolean waitKeyPress = true;
     long inSize = 10000;
     Map<String, String> params = parseParameter(args);
-    if(args.length >= 2) {
-      if(params.get("host") != null) {
+    if (args.length >= 2) {
+      if (params.get("host") != null) {
         inHost = params.get("host");
       }
-      if(params.get("port") != null) {
+      if (params.get("port") != null) {
         inPort = params.get("port");
       }
-      if(params.get("table") != null) {
+      if (params.get("table") != null) {
         inTable = params.get("table");
       }
-      if(params.get("size") != null) {
+      if (params.get("size") != null) {
         inSize = Long.parseLong(params.get("size"));
       }
-      if(params.get("dict") != null) {
+      if (params.get("dict") != null) {
         dictPath = params.get("dict");
       }
-      if(params.get("wait") != null) {
+      if (params.get("wait") != null) {
         String answer = params.get("wait");
         waitKeyPress = answer.startsWith("y") || answer.startsWith("t")? true : false;
       }
     }
-    if(waitKeyPress == true) {
+    if (waitKeyPress == true) {
       pressEnterKeyToContinue();
     }
     try {
       TestBigTable tbt = new TestBigTable();
       tbt.init(inHost, inPort);
       tbt.gen.generateTableWithIndex(inTable, (int)(inSize & 0xFFFFFFFFL), null);
-    }
-    catch(Exception e) {
+    } catch(Exception e) {
       System.out.println("generate big table got exception:" + e.getMessage());
       e.printStackTrace();
     }
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/base/IndexGroupScan.java b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/base/IndexGroupScan.java
index 1047e82..61db0f7 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/base/IndexGroupScan.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/base/IndexGroupScan.java
@@ -72,5 +72,4 @@ public interface IndexGroupScan extends GroupScan {
 
   @JsonIgnore
   public void setParallelizationWidth(int width);
-
 }
\ No newline at end of file
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
index 7bca6bd..3b5566b 100644
--- 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
@@ -105,7 +105,7 @@ public class RowKeyJoinBatch extends AbstractRecordBatch<RowKeyJoinPOP> implemen
       return;
     }
 
-    for(final VectorWrapper<?> v : left) {
+    for (final VectorWrapper<?> v : left) {
       final TransferPair pair = v.getValueVector().makeTransferPair(
           container.addOrGet(v.getField(), callBack));
       transfers.add(pair);
@@ -148,7 +148,7 @@ public class RowKeyJoinBatch extends AbstractRecordBatch<RowKeyJoinPOP> implemen
         // 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) &&
+        while ((rightUpstream == IterOutcome.OK || rightUpstream == IterOutcome.OK_NEW_SCHEMA) &&
             right.getRecordCount() == 0) {
           rightUpstream = next(right);
           logger.trace("rowkeyjoin loop when recordCount == 0. rightUpstream {}", rightUpstream);
@@ -203,16 +203,16 @@ public class RowKeyJoinBatch extends AbstractRecordBatch<RowKeyJoinPOP> implemen
   }
 
   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.
+    // 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) {
+      for (final VectorWrapper<?> v : left) {
         final TransferPair pair = v.getValueVector().makeTransferPair(
             container.addOrGet(v.getField(), callBack));
         transfers.add(pair);
@@ -223,7 +223,7 @@ public class RowKeyJoinBatch extends AbstractRecordBatch<RowKeyJoinPOP> implemen
       }
     }
 
-    for(TransferPair t : transfers) {
+    for (TransferPair t : transfers) {
       t.transfer();
     }
 
@@ -238,7 +238,7 @@ public class RowKeyJoinBatch extends AbstractRecordBatch<RowKeyJoinPOP> implemen
 
   @Override  // implement RowKeyJoin interface
   public Pair<ValueVector, Integer> nextRowKeyBatch() {
-    if ( hasRowKeyBatch && right.getRecordCount() > 0 ) {
+    if (hasRowKeyBatch && right.getRecordCount() > 0 ) {
       // since entire right row key batch will be returned to the caller, reset
       // the hasRowKeyBatch to false
       hasRowKeyBatch = false;
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/index/DrillIndexDefinition.java b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/index/DrillIndexDefinition.java
index 03c2a44..d756ae0 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/index/DrillIndexDefinition.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/index/DrillIndexDefinition.java
@@ -129,19 +129,19 @@ public class DrillIndexDefinition implements IndexDefinition {
   }
 
   boolean castIsCompatible(CastExpression castExpr, Collection<LogicalExpression> indexFields) {
-    for(LogicalExpression indexExpr : indexFields) {
-      if(indexExpr.getClass() != castExpr.getClass()) {
+    for (LogicalExpression indexExpr : indexFields) {
+      if (indexExpr.getClass() != castExpr.getClass()) {
         continue;
       }
       CastExpression indexCastExpr = (CastExpression)indexExpr;
-      //we compare input using equals because we know we are comparing SchemaPath,
-      //if we extend to support other expression, make sure the equals of that expression
-      //is implemented properly, otherwise it will fall to identity comparison
-      if ( !castExpr.getInput().equals(indexCastExpr.getInput()) ) {
+      // we compare input using equals because we know we are comparing SchemaPath,
+      // if we extend to support other expression, make sure the equals of that expression
+      // is implemented properly, otherwise it will fall to identity comparison
+      if (!castExpr.getInput().equals(indexCastExpr.getInput()) ) {
           continue;
       }
 
-      if( castExpr.getMajorType().getMinorType() != indexCastExpr.getMajorType().getMinorType()) {
+      if (castExpr.getMajorType().getMinorType() != indexCastExpr.getMajorType().getMinorType()) {
         continue;
       }
       return true;
@@ -150,8 +150,8 @@ public class DrillIndexDefinition implements IndexDefinition {
   }
 
   protected boolean columnsInIndexFields(Collection<LogicalExpression> columns, Collection<LogicalExpression> indexFields) {
-    //we need to do extra check, so we could allow the case when query condition expression is not identical with indexed fields
-    //and they still could use the index either by implicit cast or the difference is allowed, e.g. width of varchar
+    // we need to do extra check, so we could allow the case when query condition expression is not identical with indexed fields
+    // and they still could use the index either by implicit cast or the difference is allowed, e.g. width of varchar
     for (LogicalExpression col : columns) {
       if (col instanceof CastExpression) {
         if (!castIsCompatible((CastExpression) col, indexFields)) {
@@ -170,8 +170,8 @@ public class DrillIndexDefinition implements IndexDefinition {
   protected boolean someColumnsInIndexFields(Collection<LogicalExpression> columns,
       Collection<LogicalExpression> indexFields) {
 
-    //we need to do extra check, so we could allow the case when query condition expression is not identical with indexed fields
-    //and they still could use the index either by implicit cast or the difference is allowed, e.g. width of varchar
+    // we need to do extra check, so we could allow the case when query condition expression is not identical with indexed fields
+    // and they still could use the index either by implicit cast or the difference is allowed, e.g. width of varchar
     for (LogicalExpression col : columns) {
       if (col instanceof CastExpression) {
         if (castIsCompatible((CastExpression) col, indexFields)) {
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/index/DrillIndexDescriptor.java b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/index/DrillIndexDescriptor.java
index 4da62c2..3c2d21a 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/index/DrillIndexDescriptor.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/index/DrillIndexDescriptor.java
@@ -56,7 +56,7 @@ public class DrillIndexDescriptor extends AbstractIndexDescriptor {
 
   @Override
   public double getRows(RelNode scan, RexNode indexCondition) {
-    //TODO: real implementation is to use Drill's stats implementation. for now return fake value 1.0
+    // TODO: real implementation is to use Drill's stats implementation. for now return fake value 1.0
     return 1.0;
   }
 
@@ -78,27 +78,39 @@ public class DrillIndexDescriptor extends AbstractIndexDescriptor {
     return null;
   }
 
-  public void attach(String storageName, DrillTable inTable) {
-    storage = storageName;
-    setDrillTable(inTable);
-  }
-
+  /**
+   * Set the storage plugin name
+   * @param storageName
+   */
   public void setStorageName(String storageName) {
     storage = storageName;
   }
 
+  /**
+   * Get storage plugin name for this index descriptor
+   * @return name of the storage plugin
+   */
   public String getStorageName() {
     return storage;
   }
 
+  /**
+   * Set the drill table corresponding to the index
+   * @param table
+   */
   public void setDrillTable(DrillTable table) {
     this.table = table;
   }
 
+  /**
+   * Get the drill table corresponding to the index descriptor
+   * @return instance of DrillTable
+   */
   public DrillTable getDrillTable() {
     return this.table;
   }
 
+  @Override
   public FunctionalIndexInfo getFunctionalInfo() {
     return null;
   }
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
index aa51f9b..dd7f9fc 100644
--- 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
@@ -62,8 +62,7 @@ public class IndexConditionInfo {
     public Builder(RexNode condition,
                    Iterable<IndexDescriptor> indexes,
                    RexBuilder builder,
-                   RelNode scan
-    ) {
+                   RelNode scan) {
       this.condition = condition;
       this.builder = builder;
       this.scan = scan;
@@ -73,8 +72,7 @@ public class IndexConditionInfo {
     public Builder(RexNode condition,
                    IndexDescriptor index,
                    RexBuilder builder,
-                   DrillScanRel scan
-    ) {
+                   DrillScanRel scan) {
       this.condition = condition;
       this.builder = builder;
       this.scan = scan;
@@ -87,7 +85,7 @@ public class IndexConditionInfo {
      */
     public IndexConditionInfo getCollectiveInfo(IndexLogicalPlanCallContext indexContext) {
       Set<LogicalExpression> paths = Sets.newLinkedHashSet();
-      for ( IndexDescriptor index : indexes ) {
+      for (IndexDescriptor index : indexes ) {
         paths.addAll(index.getIndexColumns());
         //paths.addAll(index.getNonIndexColumns());
       }
@@ -98,10 +96,12 @@ public class IndexConditionInfo {
      * A utility function to check whether the given index hint is valid.
      */
     public boolean isValidIndexHint(IndexLogicalPlanCallContext indexContext) {
-      if (indexContext.indexHint.equals("")) { return false; }
+      if (indexContext.indexHint.equals("")) {
+        return false;
+      }
 
-      for ( IndexDescriptor index: indexes ) {
-        if ( indexContext.indexHint.equals(index.getIndexName())) {
+      for (IndexDescriptor index: indexes ) {
+        if (indexContext.indexHint.equals(index.getIndexName())) {
           return true;
         }
       }
@@ -119,15 +119,15 @@ public class IndexConditionInfo {
       Map<IndexDescriptor, IndexConditionInfo> indexInfoMap = Maps.newLinkedHashMap();
 
       RexNode initCondition = condition;
-      for(IndexDescriptor index : indexes) {
+      for (IndexDescriptor index : indexes) {
         List<LogicalExpression> leadingColumns = new ArrayList<>();
-        if(initCondition.isAlwaysTrue()) {
+        if (initCondition.isAlwaysTrue()) {
           break;
         }
-        //TODO: Ensure we dont get NULL pointer exceptions
+        // 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) {
+        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) {
@@ -144,17 +144,25 @@ public class IndexConditionInfo {
       return indexInfoMap;
     }
 
+    /**
+     * Given a RexNode corresponding to the condition expression tree and the index descriptor,
+     * check if one or more columns involved in the condition tree form a prefix of the columns in the
+     * index keys.
+     * @param indexDesc
+     * @param initCondition
+     * @return True if prefix, False if not
+     */
     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;
+        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) {
+          if (info != null && info.hasIndexCol) {
             numPrefix++;
             initCondition = info.remainderCondition;
             if (initCondition.isAlwaysTrue()) {
@@ -194,14 +202,14 @@ public class IndexConditionInfo {
       Map<IndexDescriptor, IndexConditionInfo> indexInfoMap = Maps.newLinkedHashMap();
       RexNode initCondition = condition;
       for (IndexDescriptor index : indexes) {
-        if(initCondition.isAlwaysTrue()) {
+        if (initCondition.isAlwaysTrue()) {
           break;
         }
-        if(!isConditionPrefix(index, initCondition)) {
+        if (!isConditionPrefix(index, initCondition)) {
           continue;
         }
         IndexConditionInfo info = indexConditionRelatedToFields(index.getIndexColumns(), initCondition);
-        if(info == null || info.hasIndexCol == false) {
+        if (info == null || info.hasIndexCol == false) {
           continue;
         }
         initCondition = info.remainderCondition;
@@ -235,7 +243,7 @@ public class IndexConditionInfo {
 
       List<RexNode> conjuncts = RelOptUtil.conjunctions(condition);
       List<RexNode> indexConjuncts = RelOptUtil.conjunctions(indexCondition);
-      for(RexNode indexConjunction: indexConjuncts) {
+      for (RexNode indexConjunction: indexConjuncts) {
         RexUtil.removeAll(conjuncts, indexConjunction);
       }
 
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
index cdad63a..c0758c7 100644
--- 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
@@ -103,8 +103,8 @@ public class IndexPlanUtils {
     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
+      // 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;
@@ -115,8 +115,8 @@ public class IndexPlanUtils {
    * @return
    */
   static public boolean isCoveringIndex(IndexCallContext indexContext, FunctionalIndexInfo functionInfo) {
-    if(functionInfo.hasFunctional()) {
-      //need info from full query
+    if (functionInfo.hasFunctional()) {
+      // need info from full query
       return queryCoveredByIndex(indexContext, functionInfo);
     }
     DbGroupScan groupScan = (DbGroupScan) getGroupScan(indexContext.getScan());
@@ -140,12 +140,12 @@ public class IndexPlanUtils {
    */
   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),
+    // 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 references to schema paths in functional indexes disappear beyond capProject
 
     if (indexContext.getFilter() != null && indexContext.getUpperProject() == null) {
-      if( !isFullQuery(indexContext)) {
+      if (!isFullQuery(indexContext)) {
         return false;
       }
     }
@@ -160,7 +160,7 @@ public class IndexPlanUtils {
           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
+        // 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()) {
@@ -168,7 +168,7 @@ public class IndexPlanUtils {
           exprs.add(expr);
         }
       } else {
-        //we have underneath project, so we have to do more to convert expressions
+        // 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);
@@ -194,22 +194,22 @@ public class IndexPlanUtils {
         exprs.add(expr);
       }
     }
-    else {//upperProject and lowerProject both are null, the only place to find columns being used in query is scan
+    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) {
+    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.
+    // 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
+    // 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());
@@ -218,7 +218,7 @@ public class IndexPlanUtils {
 
   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
+    // 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)) {
@@ -241,13 +241,13 @@ public class IndexPlanUtils {
    * @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
+    // 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) {
+      for (RexNode rex : projectRexs) {
         projectExprs.put(DrillOptiq.toDrill(parserContext, input, rex), idx);
         idx++;
       }
@@ -336,7 +336,7 @@ public class IndexPlanUtils {
   }
 
   public static int getIndexFromCollation(int refIndex, List<RelFieldCollation> inputFieldCollations) {
-    for (int i=0; i < inputFieldCollations.size(); i++) {
+    for (int i = 0; i < inputFieldCollations.size(); i++) {
       if (refIndex == inputFieldCollations.get(i).getFieldIndex()) {
         return i;
       }
@@ -385,14 +385,14 @@ public class IndexPlanUtils {
             indexContext.getUpperProject().getProjects().get(idx));
         indexContext.getSortExprs().add(expr);
       }
-      else {//one project is null now
+      else { // one project is null now
         oneProject = (indexContext.getUpperProject() != null)? indexContext.getUpperProject() : indexContext.getLowerProject();
-        if(oneProject != null) {
+        if (oneProject != null) {
           LogicalExpression expr = RexToExpression.toDrill(parserContext, null, indexContext.getScan(),
               getProjects(oneProject).get(idx));
           indexContext.getSortExprs().add(expr);
         }
-        else {//two projects are null
+        else { // two projects are null
           SchemaPath path;
           RelDataTypeField f = indexContext.getScan().getRowType().getFieldList().get(idx);
           String pathSeg = f.getName().replaceAll("`", "");
@@ -426,14 +426,14 @@ public class IndexPlanUtils {
                 indexContext.upperProject.getProjects().get(idx));
         indexContext.sortExprs.add(expr);
       }
-      else {//one project is null now
+      else { // one project is null now
         oneProject = (indexContext.upperProject != null)? indexContext.upperProject : indexContext.lowerProject;
-        if(oneProject != null) {
+        if (oneProject != null) {
           LogicalExpression expr = RexToExpression.toDrill(parserContext, null, indexContext.scan,
                   oneProject.getProjects().get(idx));
           indexContext.sortExprs.add(expr);
         }
-        else {//two projects are null
+        else { // two projects are null
           SchemaPath path;
           RelDataTypeField f = indexContext.scan.getRowType().getFieldList().get(idx);
           String pathSeg = f.getName().replaceAll("`", "");
@@ -452,8 +452,8 @@ public class IndexPlanUtils {
    * @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) {
+    // if there is no filter, expr wont be in equality
+    if (context.getFilter() == null) {
       return false;
     }
     final Set<LogicalExpression> onlyInEquality = context.getOrigMarker().getExpressionsOnlyInEquality();
@@ -496,7 +496,7 @@ public class IndexPlanUtils {
     Map<LogicalExpression, Integer> projectExprs = Maps.newLinkedHashMap();
     DrillParseContext parserContext = new DrillParseContext(PrelUtil.getPlannerSettings(input.getCluster()));
     int idx=0;
-    for(RexNode rex : projectRexs) {
+    for (RexNode rex : projectRexs) {
       LogicalExpression expr;
       expr = RexToExpression.toDrill(parserContext, project, input, rex);
       projectExprs.put(expr, idx);
@@ -546,7 +546,7 @@ public class IndexPlanUtils {
       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)) {
+        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
@@ -566,7 +566,7 @@ public class IndexPlanUtils {
       RelFieldCollation.NullDirection nullsDir = indexDesc.getNullsOrderingDirection();
       RelFieldCollation.Direction dir = (idxCollation == null)?
           null : idxCollation.getFieldCollations().get(idxFieldCount).direction;
-      if ( dir == null) {
+      if (dir == null) {
         break;
       }
       newFields.add(new RelFieldCollation(projectExprs.get(expr), dir, nullsDir));
@@ -609,7 +609,7 @@ public class IndexPlanUtils {
           FieldReference ref = FieldReference.getWithQuotedRef(f1.getName());
           RelFieldCollation origCollation = collationMap.get(ref);
           if (origCollation != null) {
-            RelFieldCollation fc = new RelFieldCollation(j,//origCollation.getFieldIndex(),
+            RelFieldCollation fc = new RelFieldCollation(j,
                 origCollation.direction, origCollation.nullDirection);
             rsScanCollationMap.put(origCollation.getFieldIndex(), fc);
           }
@@ -703,18 +703,18 @@ public class IndexPlanUtils {
     }
 
     List<SchemaPath> newPaths = Lists.newArrayList(paths);
-    for (int i=0; i<paths.size(); ++i) {
+    for (int i = 0; i < paths.size(); ++i) {
       SchemaPath newPath = functionInfo.getNewPath(paths.get(i));
-      if(newPath == null) {
+      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))) {
+      if (pathOnlyInIndexedFunction(paths.get(i))) {
         newPaths.set(i, newPath);
       }
-      else {// we should not replace this column, instead we add a new "$N" field.
+      else {  // we should not replace this column, instead we add a new "$N" field.
         newPaths.add(newPath);
       }
     }
@@ -797,14 +797,14 @@ public class IndexPlanUtils {
                                   List<LogicalExpression> indexCols,
                                   IndexConditionInfo.Builder builder, RexNode condition) {
     boolean prefix = true;
-    int i=0;
+    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) {
+      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);
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
index b7f2b78..d3a0b13 100644
--- 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
@@ -157,7 +157,7 @@ public class IndexSelector  {
               indexContext.getScan(), functionInfo, indexContext);
     }
 
-    if ( (inputCollation != null) &&
+    if ((inputCollation != null) &&
          (inputCollation.satisfies(indexContext.getCollation()))) {
       return true;
     }
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
index eba44b1..812c5f6 100644
--- 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
@@ -86,31 +86,31 @@ public class SimpleRexRemap {
   public RexNode rewriteEqualOnCharToLike(RexNode expr,
                                           Map<RexNode, LogicalExpression> equalOnCastCharExprs) {
     Map<RexNode, RexNode> srcToReplace = Maps.newIdentityHashMap();
-    for(Map.Entry<RexNode, LogicalExpression> entry: equalOnCastCharExprs.entrySet()) {
+    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 ) {
+      if (operands.size() == 2 ) {
         RexLiteral oplit = null;
         if (operands.get(0) instanceof RexLiteral) {
           oplit = (RexLiteral) operands.get(0);
-          if(oplit.getTypeName() == SqlTypeName.CHAR) {
+          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) {
+          if (oplit.getTypeName() == SqlTypeName.CHAR) {
             newLiteral = builder.makeLiteral(((NlsString) oplit.getValue()).getValue() + "%");
             input = operands.get(0);
           }
         }
       }
-      if(newLiteral != null) {
+      if (newLiteral != null) {
         srcToReplace.put(equalOp, builder.makeCall(SqlStdOperatorTable.LIKE, input, newLiteral));
       }
     }
@@ -130,16 +130,16 @@ public class SimpleRexRemap {
    */
   public RexNode rewriteWithMap(RexNode srcRex, Map<RexNode, LogicalExpression> mapRexToExpr) {
     Map<RexNode, RexNode> destNodeMap = Maps.newHashMap();
-    for(Map.Entry<RexNode, LogicalExpression> entry: mapRexToExpr.entrySet()) {
+    for (Map.Entry<RexNode, LogicalExpression> entry: mapRexToExpr.entrySet()) {
       LogicalExpression entryExpr = entry.getValue();
 
       LogicalExpression destExpr = destExprMap.get(entryExpr);
-      //then build rexNode from the path
+      // 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
+    // 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;
@@ -158,7 +158,7 @@ public class SimpleRexRemap {
 
   public static String getFullPath(PathSegment pathSeg) {
     PathSegment.NameSegment nameSeg = (PathSegment.NameSegment)pathSeg;
-    if(nameSeg.isLastPath()) {
+    if (nameSeg.isLastPath()) {
       return nameSeg.getPath();
     }
     return String.format("%s.%s",
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
index 0449c10..a6c1fe7 100644
--- 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
@@ -27,8 +27,8 @@ public abstract class AbstractMatchFunction<T> implements MatchFunction<T> {
     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
+      // 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;
@@ -37,8 +37,8 @@ public abstract class AbstractMatchFunction<T> implements MatchFunction<T> {
   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
+      // 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()) &&
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 c8c0db3..3a3c409 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
@@ -174,7 +174,7 @@ public class DrillMergeProjectRule extends RelOptRule {
         RelOptUtil.pushPastProject(topProject.getProjects(), bottomProject);
 
     // replace the two projects with a combined projection
-    if(topProject instanceof DrillProjectRel) {
+    if (topProject instanceof DrillProjectRel) {
       RelNode newProjectRel = DrillRelFactories.DRILL_LOGICAL_PROJECT_FACTORY.createProject(
           bottomProject.getInput(), newProjects,
           topProject.getRowType().getFieldNames());


[drill] 08/08: DRILL-6381: Address code review comments (part 3).

Posted by am...@apache.org.
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 7571d52eab9b961687df7d4fb845d0a297b228bb
Author: Aman Sinha <as...@maprtech.com>
AuthorDate: Sat Oct 13 23:38:17 2018 -0700

    DRILL-6381: Address code review comments (part 3).
    
    DRILL-6381: Add missing joinControl logic for INTERSECT_DISTINCT.
    
    - Modified HashJoin's probe phase to process INTERSECT_DISTINCT.
    
    - NOTE: For build phase, the functionality will be same as for SemiJoin when it is added later.
    
    DRILL-6381: Address code review comment for intersect_distinct.
    
    DRILL-6381: Rebase on latest master and fix compilation issues.
    
    DRILL-6381: Generate protobuf files for C++ native client.
    
    DRILL-6381: Use shaded Guava classes.  Add more comments and Javadoc.
---
 .../planner/index/MapRDBFunctionalIndexInfo.java   |  4 +-
 .../exec/planner/index/MapRDBIndexDiscover.java    | 12 ++---
 .../drill/exec/store/mapr/PluginConstants.java     |  3 ++
 .../exec/store/mapr/db/MapRDBFormatPlugin.java     |  6 ++-
 .../store/mapr/db/MapRDBPushFilterIntoScan.java    | 17 ++++---
 .../store/mapr/db/MapRDBPushLimitIntoScan.java     |  2 +-
 .../store/mapr/db/MapRDBPushProjectIntoScan.java   | 35 ++++++-------
 .../mapr/db/MapRDBRestrictedScanBatchCreator.java  |  4 +-
 .../db/json/JsonTableRangePartitionFunction.java   |  4 +-
 .../store/mapr/db/json/MaprDBJsonRecordReader.java |  2 +-
 .../mapr/db/json/RestrictedJsonTableGroupScan.java |  6 +--
 contrib/native/client/src/protobuf/BitData.pb.cc   | 55 ++++++++++++++++++---
 contrib/native/client/src/protobuf/BitData.pb.h    | 36 +++++++++++++-
 .../native/client/src/protobuf/UserBitShared.pb.cc | 57 +++++++++++-----------
 .../native/client/src/protobuf/UserBitShared.pb.h  |  7 +--
 .../exec/store/hbase/HBasePushFilterIntoScan.java  |  2 +-
 .../store/kafka/KafkaPushDownFilterIntoScan.java   |  4 +-
 .../store/mongo/MongoPushDownFilterForScan.java    |  2 +-
 .../apache/drill/exec/physical/impl/ScanBatch.java | 17 +++----
 .../exec/physical/impl/common/HashPartition.java   | 10 ++--
 .../physical/impl/join/HashJoinProbeTemplate.java  | 24 +++++++--
 .../exec/physical/impl/join/RowKeyJoinBatch.java   |  6 +++
 .../physical/impl/partitionsender/Partitioner.java |  3 ++
 .../RangePartitionRecordBatch.java                 |  7 +++
 .../exec/planner/common/DrillScanRelBase.java      |  2 +-
 .../index/InvalidIndexDefinitionException.java     |  4 ++
 .../generators/IndexIntersectPlanGenerator.java    | 18 +++----
 .../planner/logical/DrillMergeProjectRule.java     |  8 +++
 .../planner/physical/BroadcastExchangePrel.java    |  5 +-
 .../planner/physical/ConvertCountToDirectScan.java |  5 +-
 .../drill/exec/planner/physical/ScanPrel.java      |  9 ----
 .../drill/exec/planner/physical/ScanPrule.java     |  2 +-
 .../InfoSchemaPushFilterIntoRecordGenerator.java   |  2 +-
 .../exec/store/parquet/ParquetPushDownFilter.java  |  2 +-
 .../physical/impl/common/HashPartitionTest.java    |  2 +-
 35 files changed, 248 insertions(+), 136 deletions(-)

diff --git a/contrib/format-maprdb/src/main/java/org/apache/drill/exec/planner/index/MapRDBFunctionalIndexInfo.java b/contrib/format-maprdb/src/main/java/org/apache/drill/exec/planner/index/MapRDBFunctionalIndexInfo.java
index 67938f3..564a037 100644
--- a/contrib/format-maprdb/src/main/java/org/apache/drill/exec/planner/index/MapRDBFunctionalIndexInfo.java
+++ b/contrib/format-maprdb/src/main/java/org/apache/drill/exec/planner/index/MapRDBFunctionalIndexInfo.java
@@ -17,8 +17,8 @@
  */
 package org.apache.drill.exec.planner.index;
 
-import com.google.common.collect.Maps;
-import com.google.common.collect.Sets;
+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.drill.common.expression.CastExpression;
 import org.apache.drill.common.expression.LogicalExpression;
 import org.apache.drill.common.expression.SchemaPath;
diff --git a/contrib/format-maprdb/src/main/java/org/apache/drill/exec/planner/index/MapRDBIndexDiscover.java b/contrib/format-maprdb/src/main/java/org/apache/drill/exec/planner/index/MapRDBIndexDiscover.java
index c231e11..aed3e04 100644
--- a/contrib/format-maprdb/src/main/java/org/apache/drill/exec/planner/index/MapRDBIndexDiscover.java
+++ b/contrib/format-maprdb/src/main/java/org/apache/drill/exec/planner/index/MapRDBIndexDiscover.java
@@ -100,7 +100,7 @@ public class MapRDBIndexDiscover extends IndexDiscoverBase implements IndexDisco
       for (IndexDesc idx : indexes) {
         DrillIndexDescriptor hbaseIdx = buildIndexDescriptor(tableName, idx);
         if (hbaseIdx == null) {
-          //not able to build a valid index based on the index info from MFS
+          // not able to build a valid index based on the index info from MFS
           logger.error("Not able to build index for {}", idx.toString());
           continue;
         }
@@ -233,9 +233,9 @@ public class MapRDBIndexDiscover extends IndexDiscoverBase implements IndexDisco
   }
 
   private LogicalExpression castFunctionSQLSyntax(String field, String type) throws InvalidIndexDefinitionException {
-    //get castTypeStr so we can construct SQL syntax string before MapRDB could provide such syntax
+    // get castTypeStr so we can construct SQL syntax string before MapRDB could provide such syntax
     String castTypeStr = getDrillTypeStr(type);
-    if(castTypeStr == null) {//no cast
+    if(castTypeStr == null) {  // no cast
       throw new InvalidIndexDefinitionException("cast function type not recognized: " + type + "for field " + field);
     }
     try {
@@ -255,7 +255,7 @@ public class MapRDBIndexDiscover extends IndexDiscoverBase implements IndexDisco
   private LogicalExpression getIndexExpression(IndexFieldDesc desc) throws InvalidIndexDefinitionException {
     final String fieldName = desc.getFieldPath().asPathString();
     final String functionDef = desc.getFunctionName();
-    if ((functionDef != null)) {//this is a function
+    if ((functionDef != null)) {  // this is a function
       String[] tokens = functionDef.split("\\s+");
       if (tokens[0].equalsIgnoreCase("cast")) {
         if (tokens.length != 3) {
@@ -270,7 +270,7 @@ public class MapRDBIndexDiscover extends IndexDiscoverBase implements IndexDisco
         throw new InvalidIndexDefinitionException("function definition is not supported for indexing: " + functionDef);
       }
     }
-    //else it is a schemaPath
+    // else it is a schemaPath
     return fieldName2SchemaPath(fieldName);
   }
 
@@ -285,7 +285,7 @@ public class MapRDBIndexDiscover extends IndexDiscoverBase implements IndexDisco
 
   private List<RelFieldCollation> getFieldCollations(IndexDesc desc, Collection<IndexFieldDesc> descCollection) {
     List<RelFieldCollation> fieldCollations = new ArrayList<>();
-    int i=0;
+    int i = 0;
     for (IndexFieldDesc field : descCollection) {
       RelFieldCollation.Direction direction = (field.getSortOrder() == IndexFieldDesc.Order.Asc) ?
           RelFieldCollation.Direction.ASCENDING : (field.getSortOrder() == IndexFieldDesc.Order.Desc ?
diff --git a/contrib/format-maprdb/src/main/java/org/apache/drill/exec/store/mapr/PluginConstants.java b/contrib/format-maprdb/src/main/java/org/apache/drill/exec/store/mapr/PluginConstants.java
index 4239c5d..7a175a2 100644
--- a/contrib/format-maprdb/src/main/java/org/apache/drill/exec/store/mapr/PluginConstants.java
+++ b/contrib/format-maprdb/src/main/java/org/apache/drill/exec/store/mapr/PluginConstants.java
@@ -53,6 +53,9 @@ public class PluginConstants {
 
   public static final int JSON_TABLE_NUM_TABLETS_PER_INDEX_DEFAULT = 32;
 
+  public static final int JSON_TABLE_SCAN_SIZE_MB_MIN = 32;
+  public static final int JSON_TABLE_SCAN_SIZE_MB_MAX = 8192;
+
   public static final String JSON_TABLE_SCAN_SIZE_MB = "format-maprdb.json.scanSizeMB";
   public static final int JSON_TABLE_SCAN_SIZE_MB_DEFAULT = 128;
 
diff --git a/contrib/format-maprdb/src/main/java/org/apache/drill/exec/store/mapr/db/MapRDBFormatPlugin.java b/contrib/format-maprdb/src/main/java/org/apache/drill/exec/store/mapr/db/MapRDBFormatPlugin.java
index 0d1bf04..fc8a057 100644
--- a/contrib/format-maprdb/src/main/java/org/apache/drill/exec/store/mapr/db/MapRDBFormatPlugin.java
+++ b/contrib/format-maprdb/src/main/java/org/apache/drill/exec/store/mapr/db/MapRDBFormatPlugin.java
@@ -66,13 +66,15 @@ public class MapRDBFormatPlugin extends TableFormatPlugin {
     connection = ConnectionFactory.createConnection(hbaseConf);
     jsonTableCache = new MapRDBTableCache(context.getConfig());
     int scanRangeSizeMBConfig = context.getConfig().getInt(PluginConstants.JSON_TABLE_SCAN_SIZE_MB);
-    if (scanRangeSizeMBConfig < 32 || scanRangeSizeMBConfig > 8192) {
+    if (scanRangeSizeMBConfig < PluginConstants.JSON_TABLE_SCAN_SIZE_MB_MIN ||
+        scanRangeSizeMBConfig > PluginConstants.JSON_TABLE_SCAN_SIZE_MB_MAX) {
       logger.warn("Invalid scan size {} for MapR-DB tables, using default", scanRangeSizeMBConfig);
       scanRangeSizeMBConfig = PluginConstants.JSON_TABLE_SCAN_SIZE_MB_DEFAULT;
     }
 
     int restrictedScanRangeSizeMBConfig = context.getConfig().getInt(PluginConstants.JSON_TABLE_RESTRICTED_SCAN_SIZE_MB);
-    if (restrictedScanRangeSizeMBConfig < 32 || restrictedScanRangeSizeMBConfig > 8192) {
+    if (restrictedScanRangeSizeMBConfig < PluginConstants.JSON_TABLE_SCAN_SIZE_MB_MIN ||
+        restrictedScanRangeSizeMBConfig > PluginConstants.JSON_TABLE_SCAN_SIZE_MB_MAX) {
       logger.warn("Invalid restricted scan size {} for MapR-DB tables, using default", restrictedScanRangeSizeMBConfig);
       restrictedScanRangeSizeMBConfig = PluginConstants.JSON_TABLE_RESTRICTED_SCAN_SIZE_MB_DEFAULT;
     }
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 511a111..a0f5536 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
@@ -51,8 +51,9 @@ public abstract class MapRDBPushFilterIntoScan extends StoragePluginOptimizerRul
 
     @Override
     public void onMatch(RelOptRuleCall call) {
-      final ScanPrel scan = (ScanPrel) call.rel(1);
-      final FilterPrel filter = (FilterPrel) call.rel(0);
+      final FilterPrel filter = call.rel(0);
+      final ScanPrel scan = call.rel(1);
+
       final RexNode condition = filter.getCondition();
 
       if (scan.getGroupScan() instanceof BinaryTableGroupScan) {
@@ -80,9 +81,9 @@ public abstract class MapRDBPushFilterIntoScan extends StoragePluginOptimizerRul
 
     @Override
     public void onMatch(RelOptRuleCall call) {
-      final ScanPrel scan = (ScanPrel) call.rel(2);
-      final ProjectPrel project = (ProjectPrel) call.rel(1);
-      final FilterPrel filter = (FilterPrel) call.rel(0);
+      final FilterPrel filter = call.rel(0);
+      final ProjectPrel project = call.rel(1);
+      final ScanPrel scan = call.rel(2);
 
       // convert the filter to one that references the child of the project
       final RexNode condition =  RelOptUtil.pushPastProject(filter.getCondition(), project);
@@ -134,13 +135,13 @@ public abstract class MapRDBPushFilterIntoScan extends StoragePluginOptimizerRul
     final JsonConditionBuilder jsonConditionBuilder = new JsonConditionBuilder(groupScan, conditionExp);
     final JsonScanSpec newScanSpec = jsonConditionBuilder.parseTree();
     if (newScanSpec == null) {
-      return; //no filter pushdown ==> No transformation.
+      return; // no filter pushdown ==> No transformation.
     }
 
     final JsonTableGroupScan newGroupsScan = (JsonTableGroupScan) groupScan.clone(newScanSpec);
     newGroupsScan.setFilterPushedDown(true);
 
-    final ScanPrel newScanPrel = new ScanPrel(scan, filter.getTraitSet(), newGroupsScan, scan.getRowType(), scan.getTable());
+    final ScanPrel newScanPrel = new ScanPrel(scan.getCluster(), 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 +187,7 @@ public abstract class MapRDBPushFilterIntoScan extends StoragePluginOptimizerRul
                                                                         groupScan.getTableStats());
     newGroupsScan.setFilterPushedDown(true);
 
-    final ScanPrel newScanPrel = new ScanPrel(scan, filter.getTraitSet(), newGroupsScan, scan.getRowType(), scan.getTable());
+    final ScanPrel newScanPrel = new ScanPrel(scan.getCluster(), 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 a26bc80..28d59d0 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
@@ -47,8 +47,8 @@ public abstract class MapRDBPushLimitIntoScan extends StoragePluginOptimizerRule
 
     @Override
     public void onMatch(RelOptRuleCall call) {
-      final ScanPrel scan = call.rel(1);
       final LimitPrel limit = call.rel(0);
+      final ScanPrel scan = call.rel(1);
       doPushLimitIntoGroupScan(call, limit, null, scan, scan.getGroupScan());
     }
 
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 5215868..d8d0a2c 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
@@ -17,7 +17,7 @@
  */
 package org.apache.drill.exec.store.mapr.db;
 
-import com.google.common.collect.Lists;
+import org.apache.drill.shaded.guava.com.google.common.collect.Lists;
 import org.apache.calcite.plan.RelOptRuleCall;
 import org.apache.calcite.plan.RelOptRuleOperand;
 import org.apache.calcite.plan.RelTrait;
@@ -30,18 +30,21 @@ 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.common.DrillRelOptUtil;
-import org.apache.drill.exec.planner.common.DrillRelOptUtil.ProjectPushInfo;
 import org.apache.drill.exec.planner.physical.Prel;
 import org.apache.drill.exec.planner.physical.ProjectPrel;
 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;
 
+/**
+ * Push a physical Project into Scan. Currently, this rule is only doing projection pushdown for MapRDB-JSON tables
+ * since it was needed for the secondary index feature which only applies to Json tables.
+ * For binary tables, note that the DrillPushProjectIntoScanRule is still applicable during the logical
+ * planning phase.
+ */
 public abstract class MapRDBPushProjectIntoScan extends StoragePluginOptimizerRule {
   static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(MapRDBPushProjectIntoScan.class);
 
@@ -53,17 +56,10 @@ public abstract class MapRDBPushProjectIntoScan extends StoragePluginOptimizerRu
       RelOptHelper.some(ProjectPrel.class, RelOptHelper.any(ScanPrel.class)), "MapRDBPushProjIntoScan:Proj_On_Scan") {
     @Override
     public void onMatch(RelOptRuleCall call) {
-      final ScanPrel scan = (ScanPrel) call.rel(1);
-      final ProjectPrel project = (ProjectPrel) call.rel(0);
-      if (!(scan.getGroupScan() instanceof MapRDBGroupScan)) {
-        return;
-      }
-      doPushProjectIntoGroupScan(call, project, scan, (MapRDBGroupScan) scan.getGroupScan());
-      if (scan.getGroupScan() instanceof BinaryTableGroupScan) {
-        BinaryTableGroupScan groupScan = (BinaryTableGroupScan) scan.getGroupScan();
+      final ProjectPrel project = call.rel(0);
+      final ScanPrel scan = call.rel(1);
 
-      } else {
-        assert (scan.getGroupScan() instanceof JsonTableGroupScan);
+      if (scan.getGroupScan() instanceof JsonTableGroupScan) {
         JsonTableGroupScan groupScan = (JsonTableGroupScan) scan.getGroupScan();
 
         doPushProjectIntoGroupScan(call, project, scan, groupScan);
@@ -72,9 +68,10 @@ public abstract class MapRDBPushProjectIntoScan extends StoragePluginOptimizerRu
 
     @Override
     public boolean matches(RelOptRuleCall call) {
-      final ScanPrel scan = (ScanPrel) call.rel(1);
-      if (scan.getGroupScan() instanceof BinaryTableGroupScan ||
-          scan.getGroupScan() instanceof JsonTableGroupScan) {
+      final ScanPrel scan = call.rel(1);
+
+      // See class level comments above for why only JsonGroupScan is considered
+      if (scan.getGroupScan() instanceof JsonTableGroupScan) {
         return super.matches(call);
       }
       return false;
@@ -82,12 +79,12 @@ public abstract class MapRDBPushProjectIntoScan extends StoragePluginOptimizerRu
   };
 
   protected void doPushProjectIntoGroupScan(RelOptRuleCall call,
-      ProjectPrel project, ScanPrel scan, MapRDBGroupScan groupScan) {
+      ProjectPrel project, ScanPrel scan, JsonTableGroupScan groupScan) {
     try {
 
       DrillRelOptUtil.ProjectPushInfo columnInfo =
           DrillRelOptUtil.getFieldsInformation(scan.getRowType(), project.getProjects());
-      if (columnInfo == null || Utilities.isStarQuery(columnInfo.getFields()) //
+      if (columnInfo == null || Utilities.isStarQuery(columnInfo.getFields())
           || !groupScan.canPushdownProjects(columnInfo.getFields())) {
         return;
       }
diff --git a/contrib/format-maprdb/src/main/java/org/apache/drill/exec/store/mapr/db/MapRDBRestrictedScanBatchCreator.java b/contrib/format-maprdb/src/main/java/org/apache/drill/exec/store/mapr/db/MapRDBRestrictedScanBatchCreator.java
index 89ce95d..cb3732a 100644
--- a/contrib/format-maprdb/src/main/java/org/apache/drill/exec/store/mapr/db/MapRDBRestrictedScanBatchCreator.java
+++ b/contrib/format-maprdb/src/main/java/org/apache/drill/exec/store/mapr/db/MapRDBRestrictedScanBatchCreator.java
@@ -17,8 +17,8 @@
  */
 package org.apache.drill.exec.store.mapr.db;
 
-import com.google.common.base.Preconditions;
-import com.google.common.collect.Lists;
+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.common.exceptions.ExecutionSetupException;
 import org.apache.drill.exec.ops.ExecutorFragmentContext;
 import org.apache.drill.exec.physical.impl.BatchCreator;
diff --git a/contrib/format-maprdb/src/main/java/org/apache/drill/exec/store/mapr/db/json/JsonTableRangePartitionFunction.java b/contrib/format-maprdb/src/main/java/org/apache/drill/exec/store/mapr/db/json/JsonTableRangePartitionFunction.java
index c0b73ee..ca508ca 100644
--- a/contrib/format-maprdb/src/main/java/org/apache/drill/exec/store/mapr/db/json/JsonTableRangePartitionFunction.java
+++ b/contrib/format-maprdb/src/main/java/org/apache/drill/exec/store/mapr/db/json/JsonTableRangePartitionFunction.java
@@ -30,8 +30,8 @@ import com.fasterxml.jackson.annotation.JsonCreator;
 import com.fasterxml.jackson.annotation.JsonIgnore;
 import com.fasterxml.jackson.annotation.JsonProperty;
 import com.fasterxml.jackson.annotation.JsonTypeName;
-import com.google.common.base.Preconditions;
-import com.google.common.collect.Lists;
+import org.apache.drill.shaded.guava.com.google.common.base.Preconditions;
+import org.apache.drill.shaded.guava.com.google.common.collect.Lists;
 import com.mapr.db.Table;
 import com.mapr.db.impl.ConditionImpl;
 import com.mapr.db.impl.IdCodec;
diff --git a/contrib/format-maprdb/src/main/java/org/apache/drill/exec/store/mapr/db/json/MaprDBJsonRecordReader.java b/contrib/format-maprdb/src/main/java/org/apache/drill/exec/store/mapr/db/json/MaprDBJsonRecordReader.java
index 63a9381..0be44e8 100644
--- a/contrib/format-maprdb/src/main/java/org/apache/drill/exec/store/mapr/db/json/MaprDBJsonRecordReader.java
+++ b/contrib/format-maprdb/src/main/java/org/apache/drill/exec/store/mapr/db/json/MaprDBJsonRecordReader.java
@@ -236,7 +236,7 @@ public class MaprDBJsonRecordReader extends AbstractRecordReader {
           idOnly = (scannedFields == null);
         }
 
-        if(projectWholeDocument) {
+        if (projectWholeDocument) {
           projector = new FieldProjector(projectedFieldsSet);
         }
 
diff --git a/contrib/format-maprdb/src/main/java/org/apache/drill/exec/store/mapr/db/json/RestrictedJsonTableGroupScan.java b/contrib/format-maprdb/src/main/java/org/apache/drill/exec/store/mapr/db/json/RestrictedJsonTableGroupScan.java
index 2f06d00..055c5a5 100644
--- a/contrib/format-maprdb/src/main/java/org/apache/drill/exec/store/mapr/db/json/RestrictedJsonTableGroupScan.java
+++ b/contrib/format-maprdb/src/main/java/org/apache/drill/exec/store/mapr/db/json/RestrictedJsonTableGroupScan.java
@@ -24,8 +24,8 @@ import com.fasterxml.jackson.annotation.JsonCreator;
 import com.fasterxml.jackson.annotation.JsonIgnore;
 import com.fasterxml.jackson.annotation.JsonProperty;
 import com.fasterxml.jackson.annotation.JsonTypeName;
-import com.google.common.base.Preconditions;
-import com.google.common.collect.Lists;
+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.common.expression.SchemaPath;
 import org.apache.drill.exec.physical.base.GroupScan;
@@ -57,7 +57,7 @@ public class RestrictedJsonTableGroupScan extends JsonTableGroupScan {
                             @JsonProperty("format") MapRDBFormatPlugin formatPlugin,
                             @JsonProperty("scanSpec") JsonScanSpec scanSpec, /* scan spec of the original table */
                             @JsonProperty("columns") List<SchemaPath> columns,
-                            @JsonProperty("")MapRDBStatistics statistics) {
+                            @JsonProperty("") MapRDBStatistics statistics) {
     super(userName, storagePlugin, formatPlugin, scanSpec, columns, statistics);
   }
 
diff --git a/contrib/native/client/src/protobuf/BitData.pb.cc b/contrib/native/client/src/protobuf/BitData.pb.cc
index b32509c..ddee323 100644
--- a/contrib/native/client/src/protobuf/BitData.pb.cc
+++ b/contrib/native/client/src/protobuf/BitData.pb.cc
@@ -99,13 +99,14 @@ void protobuf_AssignDesc_BitData_2eproto() {
       ::google::protobuf::MessageFactory::generated_factory(),
       sizeof(FragmentRecordBatch));
   RuntimeFilterBDef_descriptor_ = file->message_type(3);
-  static const int RuntimeFilterBDef_offsets_[6] = {
+  static const int RuntimeFilterBDef_offsets_[7] = {
     GOOGLE_PROTOBUF_GENERATED_MESSAGE_FIELD_OFFSET(RuntimeFilterBDef, query_id_),
     GOOGLE_PROTOBUF_GENERATED_MESSAGE_FIELD_OFFSET(RuntimeFilterBDef, major_fragment_id_),
     GOOGLE_PROTOBUF_GENERATED_MESSAGE_FIELD_OFFSET(RuntimeFilterBDef, minor_fragment_id_),
     GOOGLE_PROTOBUF_GENERATED_MESSAGE_FIELD_OFFSET(RuntimeFilterBDef, to_foreman_),
     GOOGLE_PROTOBUF_GENERATED_MESSAGE_FIELD_OFFSET(RuntimeFilterBDef, bloom_filter_size_in_bytes_),
     GOOGLE_PROTOBUF_GENERATED_MESSAGE_FIELD_OFFSET(RuntimeFilterBDef, probe_fields_),
+    GOOGLE_PROTOBUF_GENERATED_MESSAGE_FIELD_OFFSET(RuntimeFilterBDef, hj_op_id_),
   };
   RuntimeFilterBDef_reflection_ =
     new ::google::protobuf::internal::GeneratedMessageReflection(
@@ -177,16 +178,16 @@ void protobuf_AddDesc_BitData_2eproto() {
     " \003(\005\022!\n\031sending_major_fragment_id\030\004 \001(\005\022"
     "!\n\031sending_minor_fragment_id\030\005 \001(\005\022(\n\003de"
     "f\030\006 \001(\0132\033.exec.shared.RecordBatchDef\022\023\n\013"
-    "isLastBatch\030\007 \001(\010\"\277\001\n\021RuntimeFilterBDef\022"
+    "isLastBatch\030\007 \001(\010\"\321\001\n\021RuntimeFilterBDef\022"
     "&\n\010query_id\030\001 \001(\0132\024.exec.shared.QueryId\022"
     "\031\n\021major_fragment_id\030\002 \001(\005\022\031\n\021minor_frag"
     "ment_id\030\003 \001(\005\022\022\n\nto_foreman\030\004 \001(\010\022\"\n\032blo"
     "om_filter_size_in_bytes\030\005 \003(\005\022\024\n\014probe_f"
-    "ields\030\006 \003(\t*n\n\007RpcType\022\r\n\tHANDSHAKE\020\000\022\007\n"
-    "\003ACK\020\001\022\013\n\007GOODBYE\020\002\022\024\n\020REQ_RECORD_BATCH\020"
-    "\003\022\020\n\014SASL_MESSAGE\020\004\022\026\n\022REQ_RUNTIME_FILTE"
-    "R\020\005B(\n\033org.apache.drill.exec.protoB\007BitD"
-    "ataH\001", 885);
+    "ields\030\006 \003(\t\022\020\n\010hj_op_id\030\007 \001(\005*n\n\007RpcType"
+    "\022\r\n\tHANDSHAKE\020\000\022\007\n\003ACK\020\001\022\013\n\007GOODBYE\020\002\022\024\n"
+    "\020REQ_RECORD_BATCH\020\003\022\020\n\014SASL_MESSAGE\020\004\022\026\n"
+    "\022REQ_RUNTIME_FILTER\020\005B(\n\033org.apache.dril"
+    "l.exec.protoB\007BitDataH\001", 903);
   ::google::protobuf::MessageFactory::InternalRegisterGeneratedFile(
     "BitData.proto", &protobuf_RegisterTypes);
   BitClientHandshake::default_instance_ = new BitClientHandshake();
@@ -1208,6 +1209,7 @@ const int RuntimeFilterBDef::kMinorFragmentIdFieldNumber;
 const int RuntimeFilterBDef::kToForemanFieldNumber;
 const int RuntimeFilterBDef::kBloomFilterSizeInBytesFieldNumber;
 const int RuntimeFilterBDef::kProbeFieldsFieldNumber;
+const int RuntimeFilterBDef::kHjOpIdFieldNumber;
 #endif  // !_MSC_VER
 
 RuntimeFilterBDef::RuntimeFilterBDef()
@@ -1231,6 +1233,7 @@ void RuntimeFilterBDef::SharedCtor() {
   major_fragment_id_ = 0;
   minor_fragment_id_ = 0;
   to_foreman_ = false;
+  hj_op_id_ = 0;
   ::memset(_has_bits_, 0, sizeof(_has_bits_));
 }
 
@@ -1273,6 +1276,7 @@ void RuntimeFilterBDef::Clear() {
     major_fragment_id_ = 0;
     minor_fragment_id_ = 0;
     to_foreman_ = false;
+    hj_op_id_ = 0;
   }
   bloom_filter_size_in_bytes_.Clear();
   probe_fields_.Clear();
@@ -1384,6 +1388,22 @@ bool RuntimeFilterBDef::MergePartialFromCodedStream(
           goto handle_uninterpreted;
         }
         if (input->ExpectTag(50)) goto parse_probe_fields;
+        if (input->ExpectTag(56)) goto parse_hj_op_id;
+        break;
+      }
+
+      // optional int32 hj_op_id = 7;
+      case 7: {
+        if (::google::protobuf::internal::WireFormatLite::GetTagWireType(tag) ==
+            ::google::protobuf::internal::WireFormatLite::WIRETYPE_VARINT) {
+         parse_hj_op_id:
+          DO_((::google::protobuf::internal::WireFormatLite::ReadPrimitive<
+                   ::google::protobuf::int32, ::google::protobuf::internal::WireFormatLite::TYPE_INT32>(
+                 input, &hj_op_id_)));
+          set_has_hj_op_id();
+        } else {
+          goto handle_uninterpreted;
+        }
         if (input->ExpectAtEnd()) return true;
         break;
       }
@@ -1442,6 +1462,11 @@ void RuntimeFilterBDef::SerializeWithCachedSizes(
       6, this->probe_fields(i), output);
   }
 
+  // optional int32 hj_op_id = 7;
+  if (has_hj_op_id()) {
+    ::google::protobuf::internal::WireFormatLite::WriteInt32(7, this->hj_op_id(), output);
+  }
+
   if (!unknown_fields().empty()) {
     ::google::protobuf::internal::WireFormat::SerializeUnknownFields(
         unknown_fields(), output);
@@ -1487,6 +1512,11 @@ void RuntimeFilterBDef::SerializeWithCachedSizes(
       WriteStringToArray(6, this->probe_fields(i), target);
   }
 
+  // optional int32 hj_op_id = 7;
+  if (has_hj_op_id()) {
+    target = ::google::protobuf::internal::WireFormatLite::WriteInt32ToArray(7, this->hj_op_id(), target);
+  }
+
   if (!unknown_fields().empty()) {
     target = ::google::protobuf::internal::WireFormat::SerializeUnknownFieldsToArray(
         unknown_fields(), target);
@@ -1524,6 +1554,13 @@ int RuntimeFilterBDef::ByteSize() const {
       total_size += 1 + 1;
     }
 
+    // optional int32 hj_op_id = 7;
+    if (has_hj_op_id()) {
+      total_size += 1 +
+        ::google::protobuf::internal::WireFormatLite::Int32Size(
+          this->hj_op_id());
+    }
+
   }
   // repeated int32 bloom_filter_size_in_bytes = 5;
   {
@@ -1582,6 +1619,9 @@ void RuntimeFilterBDef::MergeFrom(const RuntimeFilterBDef& from) {
     if (from.has_to_foreman()) {
       set_to_foreman(from.to_foreman());
     }
+    if (from.has_hj_op_id()) {
+      set_hj_op_id(from.hj_op_id());
+    }
   }
   mutable_unknown_fields()->MergeFrom(from.unknown_fields());
 }
@@ -1611,6 +1651,7 @@ void RuntimeFilterBDef::Swap(RuntimeFilterBDef* other) {
     std::swap(to_foreman_, other->to_foreman_);
     bloom_filter_size_in_bytes_.Swap(&other->bloom_filter_size_in_bytes_);
     probe_fields_.Swap(&other->probe_fields_);
+    std::swap(hj_op_id_, other->hj_op_id_);
     std::swap(_has_bits_[0], other->_has_bits_[0]);
     _unknown_fields_.Swap(&other->_unknown_fields_);
     std::swap(_cached_size_, other->_cached_size_);
diff --git a/contrib/native/client/src/protobuf/BitData.pb.h b/contrib/native/client/src/protobuf/BitData.pb.h
index 8a0b60c..7ee0bc6 100644
--- a/contrib/native/client/src/protobuf/BitData.pb.h
+++ b/contrib/native/client/src/protobuf/BitData.pb.h
@@ -521,6 +521,13 @@ class RuntimeFilterBDef : public ::google::protobuf::Message {
   inline const ::google::protobuf::RepeatedPtrField< ::std::string>& probe_fields() const;
   inline ::google::protobuf::RepeatedPtrField< ::std::string>* mutable_probe_fields();
 
+  // optional int32 hj_op_id = 7;
+  inline bool has_hj_op_id() const;
+  inline void clear_hj_op_id();
+  static const int kHjOpIdFieldNumber = 7;
+  inline ::google::protobuf::int32 hj_op_id() const;
+  inline void set_hj_op_id(::google::protobuf::int32 value);
+
   // @@protoc_insertion_point(class_scope:exec.bit.data.RuntimeFilterBDef)
  private:
   inline void set_has_query_id();
@@ -531,6 +538,8 @@ class RuntimeFilterBDef : public ::google::protobuf::Message {
   inline void clear_has_minor_fragment_id();
   inline void set_has_to_foreman();
   inline void clear_has_to_foreman();
+  inline void set_has_hj_op_id();
+  inline void clear_has_hj_op_id();
 
   ::google::protobuf::UnknownFieldSet _unknown_fields_;
 
@@ -538,11 +547,12 @@ class RuntimeFilterBDef : public ::google::protobuf::Message {
   ::google::protobuf::int32 major_fragment_id_;
   ::google::protobuf::int32 minor_fragment_id_;
   ::google::protobuf::RepeatedField< ::google::protobuf::int32 > bloom_filter_size_in_bytes_;
-  ::google::protobuf::RepeatedPtrField< ::std::string> probe_fields_;
   bool to_foreman_;
+  ::google::protobuf::int32 hj_op_id_;
+  ::google::protobuf::RepeatedPtrField< ::std::string> probe_fields_;
 
   mutable int _cached_size_;
-  ::google::protobuf::uint32 _has_bits_[(6 + 31) / 32];
+  ::google::protobuf::uint32 _has_bits_[(7 + 31) / 32];
 
   friend void  protobuf_AddDesc_BitData_2eproto();
   friend void protobuf_AssignDesc_BitData_2eproto();
@@ -1043,6 +1053,28 @@ RuntimeFilterBDef::mutable_probe_fields() {
   return &probe_fields_;
 }
 
+// optional int32 hj_op_id = 7;
+inline bool RuntimeFilterBDef::has_hj_op_id() const {
+  return (_has_bits_[0] & 0x00000040u) != 0;
+}
+inline void RuntimeFilterBDef::set_has_hj_op_id() {
+  _has_bits_[0] |= 0x00000040u;
+}
+inline void RuntimeFilterBDef::clear_has_hj_op_id() {
+  _has_bits_[0] &= ~0x00000040u;
+}
+inline void RuntimeFilterBDef::clear_hj_op_id() {
+  hj_op_id_ = 0;
+  clear_has_hj_op_id();
+}
+inline ::google::protobuf::int32 RuntimeFilterBDef::hj_op_id() const {
+  return hj_op_id_;
+}
+inline void RuntimeFilterBDef::set_hj_op_id(::google::protobuf::int32 value) {
+  set_has_hj_op_id();
+  hj_op_id_ = value;
+}
+
 
 // @@protoc_insertion_point(namespace_scope)
 
diff --git a/contrib/native/client/src/protobuf/UserBitShared.pb.cc b/contrib/native/client/src/protobuf/UserBitShared.pb.cc
index d0e4aa5..8bb6e07 100644
--- a/contrib/native/client/src/protobuf/UserBitShared.pb.cc
+++ b/contrib/native/client/src/protobuf/UserBitShared.pb.cc
@@ -750,40 +750,40 @@ void protobuf_AddDesc_UserBitShared_2eproto() {
     "TATEMENT\020\005*\207\001\n\rFragmentState\022\013\n\007SENDING\020"
     "\000\022\027\n\023AWAITING_ALLOCATION\020\001\022\013\n\007RUNNING\020\002\022"
     "\014\n\010FINISHED\020\003\022\r\n\tCANCELLED\020\004\022\n\n\006FAILED\020\005"
-    "\022\032\n\026CANCELLATION_REQUESTED\020\006*\367\010\n\020CoreOpe"
+    "\022\032\n\026CANCELLATION_REQUESTED\020\006*\222\t\n\020CoreOpe"
     "ratorType\022\021\n\rSINGLE_SENDER\020\000\022\024\n\020BROADCAS"
     "T_SENDER\020\001\022\n\n\006FILTER\020\002\022\022\n\016HASH_AGGREGATE"
     "\020\003\022\r\n\tHASH_JOIN\020\004\022\016\n\nMERGE_JOIN\020\005\022\031\n\025HAS"
     "H_PARTITION_SENDER\020\006\022\t\n\005LIMIT\020\007\022\024\n\020MERGI"
     "NG_RECEIVER\020\010\022\034\n\030ORDERED_PARTITION_SENDE"
     "R\020\t\022\013\n\007PROJECT\020\n\022\026\n\022UNORDERED_RECEIVER\020\013"
-    "\022\020\n\014RANGE_SENDER\020\014\022\n\n\006SCREEN\020\r\022\034\n\030SELECT"
-    "ION_VECTOR_REMOVER\020\016\022\027\n\023STREAMING_AGGREG"
-    "ATE\020\017\022\016\n\nTOP_N_SORT\020\020\022\021\n\rEXTERNAL_SORT\020\021"
-    "\022\t\n\005TRACE\020\022\022\t\n\005UNION\020\023\022\014\n\010OLD_SORT\020\024\022\032\n\026"
-    "PARQUET_ROW_GROUP_SCAN\020\025\022\021\n\rHIVE_SUB_SCA"
-    "N\020\026\022\025\n\021SYSTEM_TABLE_SCAN\020\027\022\021\n\rMOCK_SUB_S"
-    "CAN\020\030\022\022\n\016PARQUET_WRITER\020\031\022\023\n\017DIRECT_SUB_"
-    "SCAN\020\032\022\017\n\013TEXT_WRITER\020\033\022\021\n\rTEXT_SUB_SCAN"
-    "\020\034\022\021\n\rJSON_SUB_SCAN\020\035\022\030\n\024INFO_SCHEMA_SUB"
-    "_SCAN\020\036\022\023\n\017COMPLEX_TO_JSON\020\037\022\025\n\021PRODUCER"
-    "_CONSUMER\020 \022\022\n\016HBASE_SUB_SCAN\020!\022\n\n\006WINDO"
-    "W\020\"\022\024\n\020NESTED_LOOP_JOIN\020#\022\021\n\rAVRO_SUB_SC"
-    "AN\020$\022\021\n\rPCAP_SUB_SCAN\020%\022\022\n\016KAFKA_SUB_SCA"
-    "N\020&\022\021\n\rKUDU_SUB_SCAN\020\'\022\013\n\007FLATTEN\020(\022\020\n\014L"
-    "ATERAL_JOIN\020)\022\n\n\006UNNEST\020*\022,\n(HIVE_DRILL_"
-    "NATIVE_PARQUET_ROW_GROUP_SCAN\020+\022\r\n\tJDBC_"
-    "SCAN\020,\022\022\n\016REGEX_SUB_SCAN\020-\022\023\n\017MAPRDB_SUB"
-    "_SCAN\020.\022\022\n\016MONGO_SUB_SCAN\020/\022\017\n\013KUDU_WRIT"
-    "ER\0200\022\026\n\022OPEN_TSDB_SUB_SCAN\0201\022\017\n\013JSON_WRI"
-    "TER\0202\022\026\n\022HTPPD_LOG_SUB_SCAN\0203\022\022\n\016IMAGE_S"
-    "UB_SCAN\0204\022\025\n\021SEQUENCE_SUB_SCAN\0205\022\023\n\017PART"
-    "ITION_LIMIT\0206\022\023\n\017PCAPNG_SUB_SCAN\0207\022\022\n\016RU"
-    "NTIME_FILTER\0208*g\n\nSaslStatus\022\020\n\014SASL_UNK"
-    "NOWN\020\000\022\016\n\nSASL_START\020\001\022\024\n\020SASL_IN_PROGRE"
-    "SS\020\002\022\020\n\014SASL_SUCCESS\020\003\022\017\n\013SASL_FAILED\020\004B"
-    ".\n\033org.apache.drill.exec.protoB\rUserBitS"
-    "haredH\001", 5447);
+    "\022\032\n\026RANGE_PARTITION_SENDER\020\014\022\n\n\006SCREEN\020\r"
+    "\022\034\n\030SELECTION_VECTOR_REMOVER\020\016\022\027\n\023STREAM"
+    "ING_AGGREGATE\020\017\022\016\n\nTOP_N_SORT\020\020\022\021\n\rEXTER"
+    "NAL_SORT\020\021\022\t\n\005TRACE\020\022\022\t\n\005UNION\020\023\022\014\n\010OLD_"
+    "SORT\020\024\022\032\n\026PARQUET_ROW_GROUP_SCAN\020\025\022\021\n\rHI"
+    "VE_SUB_SCAN\020\026\022\025\n\021SYSTEM_TABLE_SCAN\020\027\022\021\n\r"
+    "MOCK_SUB_SCAN\020\030\022\022\n\016PARQUET_WRITER\020\031\022\023\n\017D"
+    "IRECT_SUB_SCAN\020\032\022\017\n\013TEXT_WRITER\020\033\022\021\n\rTEX"
+    "T_SUB_SCAN\020\034\022\021\n\rJSON_SUB_SCAN\020\035\022\030\n\024INFO_"
+    "SCHEMA_SUB_SCAN\020\036\022\023\n\017COMPLEX_TO_JSON\020\037\022\025"
+    "\n\021PRODUCER_CONSUMER\020 \022\022\n\016HBASE_SUB_SCAN\020"
+    "!\022\n\n\006WINDOW\020\"\022\024\n\020NESTED_LOOP_JOIN\020#\022\021\n\rA"
+    "VRO_SUB_SCAN\020$\022\021\n\rPCAP_SUB_SCAN\020%\022\022\n\016KAF"
+    "KA_SUB_SCAN\020&\022\021\n\rKUDU_SUB_SCAN\020\'\022\013\n\007FLAT"
+    "TEN\020(\022\020\n\014LATERAL_JOIN\020)\022\n\n\006UNNEST\020*\022,\n(H"
+    "IVE_DRILL_NATIVE_PARQUET_ROW_GROUP_SCAN\020"
+    "+\022\r\n\tJDBC_SCAN\020,\022\022\n\016REGEX_SUB_SCAN\020-\022\023\n\017"
+    "MAPRDB_SUB_SCAN\020.\022\022\n\016MONGO_SUB_SCAN\020/\022\017\n"
+    "\013KUDU_WRITER\0200\022\026\n\022OPEN_TSDB_SUB_SCAN\0201\022\017"
+    "\n\013JSON_WRITER\0202\022\026\n\022HTPPD_LOG_SUB_SCAN\0203\022"
+    "\022\n\016IMAGE_SUB_SCAN\0204\022\025\n\021SEQUENCE_SUB_SCAN"
+    "\0205\022\023\n\017PARTITION_LIMIT\0206\022\023\n\017PCAPNG_SUB_SC"
+    "AN\0207\022\022\n\016RUNTIME_FILTER\0208\022\017\n\013ROWKEY_JOIN\020"
+    "9*g\n\nSaslStatus\022\020\n\014SASL_UNKNOWN\020\000\022\016\n\nSAS"
+    "L_START\020\001\022\024\n\020SASL_IN_PROGRESS\020\002\022\020\n\014SASL_"
+    "SUCCESS\020\003\022\017\n\013SASL_FAILED\020\004B.\n\033org.apache"
+    ".drill.exec.protoB\rUserBitSharedH\001", 5474);
   ::google::protobuf::MessageFactory::InternalRegisterGeneratedFile(
     "UserBitShared.proto", &protobuf_RegisterTypes);
   UserCredentials::default_instance_ = new UserCredentials();
@@ -961,6 +961,7 @@ bool CoreOperatorType_IsValid(int value) {
     case 54:
     case 55:
     case 56:
+    case 57:
       return true;
     default:
       return false;
diff --git a/contrib/native/client/src/protobuf/UserBitShared.pb.h b/contrib/native/client/src/protobuf/UserBitShared.pb.h
index 8494857..ab3063d 100644
--- a/contrib/native/client/src/protobuf/UserBitShared.pb.h
+++ b/contrib/native/client/src/protobuf/UserBitShared.pb.h
@@ -216,7 +216,7 @@ enum CoreOperatorType {
   ORDERED_PARTITION_SENDER = 9,
   PROJECT = 10,
   UNORDERED_RECEIVER = 11,
-  RANGE_SENDER = 12,
+  RANGE_PARTITION_SENDER = 12,
   SCREEN = 13,
   SELECTION_VECTOR_REMOVER = 14,
   STREAMING_AGGREGATE = 15,
@@ -260,11 +260,12 @@ enum CoreOperatorType {
   SEQUENCE_SUB_SCAN = 53,
   PARTITION_LIMIT = 54,
   PCAPNG_SUB_SCAN = 55,
-  RUNTIME_FILTER = 56
+  RUNTIME_FILTER = 56,
+  ROWKEY_JOIN = 57
 };
 bool CoreOperatorType_IsValid(int value);
 const CoreOperatorType CoreOperatorType_MIN = SINGLE_SENDER;
-const CoreOperatorType CoreOperatorType_MAX = RUNTIME_FILTER;
+const CoreOperatorType CoreOperatorType_MAX = ROWKEY_JOIN;
 const int CoreOperatorType_ARRAYSIZE = CoreOperatorType_MAX + 1;
 
 const ::google::protobuf::EnumDescriptor* CoreOperatorType_descriptor();
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 91ca787..3faa089 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(), scan.getTable());
+    final ScanPrel newScanPrel = new ScanPrel(scan.getCluster(), 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-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 019a67e..002d043 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,8 +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(), scan.getTable());
+    final ScanPrel newScanPrel =
+      new ScanPrel(scan.getCluster(), 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 8ad84c1..be157b4 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
@@ -77,7 +77,7 @@ public class MongoPushDownFilterForScan extends StoragePluginOptimizerRule {
     }
     newGroupsScan.setFilterPushedDown(true);
 
-    final ScanPrel newScanPrel = new ScanPrel(scan, filter.getTraitSet(),
+    final ScanPrel newScanPrel = new ScanPrel(scan.getCluster(), filter.getTraitSet(),
         newGroupsScan, scan.getRowType(), scan.getTable());
 
     if (mongoFilterBuilder.isAllExpressionsConverted()) {
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 70b2852..a688f37 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
@@ -182,11 +182,11 @@ public class ScanBatch implements CloseableRecordBatch {
     if(isRepeatableScan) {
       readers = readerList.iterator();
       return IterOutcome.NONE;
-    } 
-    else {
+    } else {
       releaseAssets(); // All data has been read. Release resource.
       done = true;
-      return IterOutcome.NONE;}
+      return IterOutcome.NONE;
+    }
   }
 
   /**
@@ -204,11 +204,10 @@ public class ScanBatch implements CloseableRecordBatch {
         return false;
       }
       return true;
-    }
-    else {// Regular scan
+    } 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
+      return true; // In regular case, we always continue the iteration, if no more reader, we will break out at the head of loop
     }
   }
 
@@ -283,7 +282,7 @@ public class ScanBatch implements CloseableRecordBatch {
         }
       }
       lastOutcome = IterOutcome.STOP;
-      throw UserException.systemError(e)
+      throw UserException.internalError(e)
           .addContext("Setup failed for", currentReaderClassName)
           .build(logger);
     } catch (UserException ex) {
@@ -291,7 +290,7 @@ public class ScanBatch implements CloseableRecordBatch {
       throw ex;
     } catch (Exception ex) {
       lastOutcome = IterOutcome.STOP;
-      throw UserException.systemError(ex).build(logger);
+      throw UserException.internalError(ex).build(logger);
     } finally {
       oContext.getStats().stopProcessing();
     }
@@ -334,7 +333,7 @@ public class ScanBatch implements CloseableRecordBatch {
       }
     } catch(SchemaChangeException e) {
       // No exception should be thrown here.
-      throw UserException.systemError(e)
+      throw UserException.internalError(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 1f7da38..86b870d 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
@@ -378,7 +378,7 @@ public class HashPartition implements HashJoinMemoryCalculator.PartitionStat {
   public int probeForKey(int recordsProcessed, int hashCode) throws SchemaChangeException {
     return hashTable.probeForKey(recordsProcessed, hashCode);
   }
-  public int getStartIndex(int probeIndex) {
+  public Pair<Integer, Boolean> getStartIndex(int probeIndex) {
     /* The current probe record has a key that matches. Get the index
      * of the first row in the build side that matches the current key
      */
@@ -387,15 +387,15 @@ public class HashPartition implements HashJoinMemoryCalculator.PartitionStat {
      * side. Set the bit corresponding to this index so if we are doing a FULL or RIGHT
      * join we keep track of which records we need to project at the end
      */
-    hjHelper.setRecordMatched(compositeIndex);
-    return compositeIndex;
+    boolean matchExists = hjHelper.setRecordMatched(compositeIndex);
+    return Pair.of(compositeIndex, matchExists);
   }
   public int getNextIndex(int compositeIndex) {
     // in case of iner rows with duplicate keys, get the next one
     return hjHelper.getNextIndex(compositeIndex);
   }
-  public void setRecordMatched(int compositeIndex) {
-    hjHelper.setRecordMatched(compositeIndex);
+  public boolean setRecordMatched(int compositeIndex) {
+    return hjHelper.setRecordMatched(compositeIndex);
   }
   public List<Integer> getNextUnmatchedIndex() {
     return hjHelper.getNextUnmatchedIndex();
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 c16812e..57b2d5b 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
@@ -30,6 +30,7 @@ import org.apache.drill.exec.record.RecordBatch.IterOutcome;
 import org.apache.drill.exec.record.VectorContainer;
 import org.apache.drill.exec.record.VectorWrapper;
 import org.apache.calcite.rel.core.JoinRelType;
+import org.apache.commons.lang3.tuple.Pair;
 import org.apache.drill.exec.vector.IntVector;
 import org.apache.drill.exec.vector.ValueVector;
 
@@ -47,8 +48,7 @@ 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
+  // joinControl determines how to handle INTERSECT_DISTINCT vs. INTERSECT_ALL
   private JoinControl joinControl;
 
   private HashJoinBatch outgoingJoinBatch = null;
@@ -325,16 +325,30 @@ public abstract class HashJoinProbeTemplate implements HashJoinProbe {
            * of the first row in the build side that matches the current key
            * (and record this match in the bitmap, in case of a FULL/RIGHT join)
            */
-          currentCompositeIdx = currPartition.getStartIndex(probeIndex);
+          Pair<Integer, Boolean> matchStatus = currPartition.getStartIndex(probeIndex);
+
+          boolean matchExists = matchStatus.getRight();
+
+          if (joinControl.isIntersectDistinct() && matchExists) {
+            // since it is intersect distinct and we already have one record matched, move to next probe row
+            recordsProcessed++;
+            continue;
+          }
+
+          currentCompositeIdx = matchStatus.getLeft();
 
           outputRecords =
             outputRow(currPartition.getContainers(), currentCompositeIdx,
               probeBatch.getContainer(), recordsProcessed);
 
           /* Projected single row from the build side with matching key but there
-           * may be more rows with the same key. Check if that's the case
+           * may be more rows with the same key. Check if that's the case as long as
+           * we are not doing intersect distinct since it only cares about
+           * distinct values.
            */
-          currentCompositeIdx = currPartition.getNextIndex(currentCompositeIdx);
+          currentCompositeIdx = joinControl.isIntersectDistinct() ? -1 :
+            currPartition.getNextIndex(currentCompositeIdx);
+
           if (currentCompositeIdx == -1) {
             /* We only had one row in the build side that matched the current key
              * from the probe side. Drain the next row in the probe side.
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
index 3b5566b..941f321 100644
--- 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
@@ -281,4 +281,10 @@ public class RowKeyJoinBatch extends AbstractRecordBatch<RowKeyJoinPOP> implemen
     super.close();
   }
 
+  @Override
+  public void dump() {
+    logger.error("RowKeyJoinBatch[container={}, left={}, right={}, hasRowKeyBatch={}, rkJoinState={}]",
+        container, left, right, hasRowKeyBatch, rkJoinState);
+  }
+
 }
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 2e2e760..a2fc069 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,9 @@ import org.apache.drill.exec.physical.config.HashPartitionSender;
 import org.apache.drill.exec.record.RecordBatch;
 
 public interface Partitioner {
+  // Keep the recordCount as (2^x) - 1 to better utilize the memory allocation in ValueVectors; however
+  // other criteria such as batch sizing in terms of actual MBytes rather than record count could also be applied
+  // by the operator.
   int DEFAULT_RECORD_BATCH_SIZE = (1 << 10) - 1;
 
   void setup(ExchangeFragmentContext context,
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
index d8fc94d..6a54828 100644
--- 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
@@ -183,4 +183,11 @@ public class RangePartitionRecordBatch extends AbstractSingleRecordBatch<RangePa
     return counter;
   }
 
+
+  @Override
+  public void dump() {
+    logger.error("RangePartitionRecordBatch[container={}, numPartitions={}, recordCount={}, partitionIdVector={}]",
+        container, numPartitions, recordCount, partitionIdVector);
+  }
+
 }
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 6f8ee0e..6409c8e 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
@@ -34,7 +34,7 @@ import org.apache.drill.exec.util.Utilities;
  * Base class for logical/physical scan rel implemented in Drill.
  */
 public abstract class DrillScanRelBase extends TableScan implements DrillRelNode {
-  private GroupScan groupScan;
+  protected GroupScan groupScan;
   protected final DrillTable drillTable;
 
   public DrillScanRelBase(RelOptCluster cluster,
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/index/InvalidIndexDefinitionException.java b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/index/InvalidIndexDefinitionException.java
index c17d09f..518aa67 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/index/InvalidIndexDefinitionException.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/index/InvalidIndexDefinitionException.java
@@ -19,6 +19,10 @@ package org.apache.drill.exec.planner.index;
 
 import org.apache.drill.common.exceptions.DrillRuntimeException;
 
+/**
+ * An InvalidIndexDefinitionException may be thrown if Drill does not recognize the
+ * type or expression of the index during the index discovery phase
+ */
 public class InvalidIndexDefinitionException extends DrillRuntimeException {
   public InvalidIndexDefinitionException(String message) {
     super(message);
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
index b380c28..11d7358 100644
--- 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
@@ -94,7 +94,7 @@ public class IndexIntersectPlanGenerator extends AbstractIndexPlanGenerator {
         RelOptUtil.createEquiJoinCondition(left, leftJoinKeys,
             right, rightJoinKeys, builder);
 
-    if (isRowKeyJoin == true) {
+    if (isRowKeyJoin) {
       RelNode newRel;
       if (settings.isIndexUseHashJoinNonCovering()) {
         HashJoinPrel hjPrel = new HashJoinPrel(left.getCluster(), left.getTraitSet(), left,
@@ -109,7 +109,7 @@ public class IndexIntersectPlanGenerator extends AbstractIndexPlanGenerator {
       // 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
+      // 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);
@@ -185,7 +185,7 @@ public class IndexIntersectPlanGenerator extends AbstractIndexPlanGenerator {
         indexScanRowType, builder, functionInfo));
     // project the rowkey column from the index scan
     List<RexNode> indexProjectExprs = Lists.newArrayList();
-    int rowKeyIndex = getRowKeyIndex(indexScanPrel.getRowType(), origScan);//indexGroupScan.getRowKeyOrdinal();
+    int rowKeyIndex = getRowKeyIndex(indexScanPrel.getRowType(), origScan);
     assert rowKeyIndex >= 0;
 
     indexProjectExprs.add(RexInputRef.of(rowKeyIndex, indexScanPrel.getRowType()));
@@ -204,7 +204,7 @@ public class IndexIntersectPlanGenerator extends AbstractIndexPlanGenerator {
         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 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()) {
@@ -220,7 +220,7 @@ public class IndexIntersectPlanGenerator extends AbstractIndexPlanGenerator {
       return converted;
     }
 
-    //if build(right) side exist, the plan we got in 'converted' is left (probe). Intersect with right(build) side
+    // 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 &&
@@ -310,7 +310,7 @@ public class IndexIntersectPlanGenerator extends AbstractIndexPlanGenerator {
   public RelNode convertChild(final RelNode filter, final RelNode input) throws InvalidRelException {
     Map<IndexDescriptor, RexNode> idxConditionMap = Maps.newLinkedHashMap();
     boolean isAnyIndexAsync = false;
-    for(IndexDescriptor idx : indexInfoMap.keySet()) {
+    for (IndexDescriptor idx : indexInfoMap.keySet()) {
       idxConditionMap.put(idx, indexInfoMap.get(idx).indexCondition);
       if (!isAnyIndexAsync && idx.isAsyncIndex()) {
         isAnyIndexAsync = true;
@@ -322,7 +322,7 @@ public class IndexIntersectPlanGenerator extends AbstractIndexPlanGenerator {
     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!
+      // 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;
@@ -333,12 +333,12 @@ public class IndexIntersectPlanGenerator extends AbstractIndexPlanGenerator {
     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
+    // now with index plan constructed, build plan of left(probe) side to use restricted db scan
 
     Pair<RelNode, DbGroupScan> leftRelAndScan = buildRestrictedDBScan(remnant, isAnyIndexAsync);
 
     RelNode finalRel = buildRowKeyJoin(leftRelAndScan.left, rangeDistRight, true, JoinControl.DEFAULT);
-    if ( upperProject != null) {
+    if (upperProject != null) {
       ProjectPrel cap = new ProjectPrel(finalRel.getCluster(), finalRel.getTraitSet(),
           finalRel, IndexPlanUtils.getProjects(upperProject), upperProject.getRowType());
       finalRel = cap;
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 3a3c409..10f9567 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
@@ -169,6 +169,14 @@ public class DrillMergeProjectRule extends RelOptRule {
     return list;
   }
 
+  /**
+   * The purpose of the replace() method is to allow the caller to replace a 'top' and 'bottom' project with
+   * a single merged project with the assumption that caller knows exactly the semantics/correctness of merging
+   * the two projects. This is not applying the full fledged DrillMergeProjectRule.
+   * @param topProject
+   * @param bottomProject
+   * @return new project after replacement
+   */
   public static Project replace(Project topProject, Project bottomProject) {
     final List<RexNode> newProjects =
         RelOptUtil.pushPastProject(topProject.getProjects(), bottomProject);
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 1ae375b..0155734 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
@@ -58,7 +58,10 @@ public class BroadcastExchangePrel extends ExchangePrel{
     final int  rowWidth = child.getRowType().getFieldCount() * DrillCostBase.AVG_FIELD_WIDTH;
     final double cpuCost = broadcastFactor * DrillCostBase.SVR_CPU_COST * inputRows;
 
-    //we assume localhost network cost is 1/10 of regular network cost
+    // We assume localhost network cost is 1/10 of regular network cost
+    //  ( c * num_bytes * (N - 1) ) + ( c * num_bytes * 0.1)
+    // = c * num_bytes * (N - 0.9)
+    // TODO: a similar adjustment should be made to HashExchangePrel
     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 5204495..d903165 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,8 +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 DirectScanPrel newScan = new DirectScanPrel(scan.getCluster(),
+        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());
@@ -158,7 +158,6 @@ public class ConvertCountToDirectScan extends Prule {
 
     for (int i = 0; i < agg.getAggCallList().size(); i++) {
       AggregateCall aggCall = agg.getAggCallList().get(i);
-    //for (AggregateCall aggCall : agg.getAggCallList()) {
       long cnt;
 
       // rule can be applied only for count function, return empty counts
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/ScanPrel.java
index 4db5ba2..8a46f86 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/ScanPrel.java
@@ -48,15 +48,6 @@ public class ScanPrel extends DrillScanRelBase implements Prel, HasDistributionA
 
   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) {
-    super(cluster, traits);
-    this.groupScan = getCopy(groupScan);
-  }
-
   public ScanPrel(RelOptCluster cluster, RelTraitSet traits,
                   GroupScan groupScan, RelDataType rowType, RelOptTable table) {
     super(cluster, traits, getCopy(groupScan), table);
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/physical/ScanPrule.java b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/physical/ScanPrule.java
index 6ee05eb..b4fada6 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/physical/ScanPrule.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/physical/ScanPrule.java
@@ -44,7 +44,7 @@ public class ScanPrule extends Prule{
 
     final RelTraitSet traits = scan.getTraitSet().plus(Prel.DRILL_PHYSICAL).plus(partition);
 
-    final ScanPrel newScan = new ScanPrel(scan, traits, groupScan, scan.getRowType(), scan.getTable());
+    final ScanPrel newScan = new ScanPrel(scan.getCluster(), traits, groupScan, scan.getRowType(), scan.getTable());
 
     call.transformTo(newScan);
   }
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/store/ischema/InfoSchemaPushFilterIntoRecordGenerator.java b/exec/java-exec/src/main/java/org/apache/drill/exec/store/ischema/InfoSchemaPushFilterIntoRecordGenerator.java
index 92332c3..44d2394 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/store/ischema/InfoSchemaPushFilterIntoRecordGenerator.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/store/ischema/InfoSchemaPushFilterIntoRecordGenerator.java
@@ -99,7 +99,7 @@ public abstract class InfoSchemaPushFilterIntoRecordGenerator extends StoragePlu
     final InfoSchemaGroupScan newGroupsScan = new InfoSchemaGroupScan(groupScan.getTable(), infoSchemaFilter);
     newGroupsScan.setFilterPushedDown(true);
 
-    RelNode input = new ScanPrel(scan, filter.getTraitSet(), newGroupsScan, scan.getRowType(), scan.getTable());
+    RelNode input = new ScanPrel(scan.getCluster(), filter.getTraitSet(), newGroupsScan, scan.getRowType(), scan.getTable());
     if (project != null) {
       input = project.copy(project.getTraitSet(), input, project.getProjects(), filter.getRowType());
     }
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/store/parquet/ParquetPushDownFilter.java b/exec/java-exec/src/main/java/org/apache/drill/exec/store/parquet/ParquetPushDownFilter.java
index a8b981c..6efd44d 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/store/parquet/ParquetPushDownFilter.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/store/parquet/ParquetPushDownFilter.java
@@ -167,7 +167,7 @@ public abstract class ParquetPushDownFilter extends StoragePluginOptimizerRule {
       return;
     }
 
-    RelNode newScan = new ScanPrel(scan, scan.getTraitSet(), newGroupScan, scan.getRowType(), scan.getTable());
+    RelNode newScan = new ScanPrel(scan.getCluster(), scan.getTraitSet(), newGroupScan, scan.getRowType(), scan.getTable());
 
     if (project != null) {
       newScan = project.copy(project.getTraitSet(), ImmutableList.of(newScan));
diff --git a/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/common/HashPartitionTest.java b/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/common/HashPartitionTest.java
index aafcb4d..e2f80d8 100644
--- a/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/common/HashPartitionTest.java
+++ b/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/common/HashPartitionTest.java
@@ -132,7 +132,7 @@ public class HashPartitionTest {
 
         {
           int compositeIndex = hashPartition.probeForKey(1, 12);
-          int startIndex = hashPartition.getStartIndex(compositeIndex);
+          int startIndex = hashPartition.getStartIndex(compositeIndex).getLeft();
           int nextIndex = hashPartition.getNextIndex(startIndex);
 
           Assert.assertEquals(2, startIndex);


[drill] 02/08: DRILL-6381: (Part 2) MapRDB plugin update to 6.0.1

Posted by am...@apache.org.
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 a4f62e9b5d0e69585b91042236c1ca68e1d628bc
Author: rebase <bu...@mapr.com>
AuthorDate: Mon Feb 12 14:31:49 2018 -0800

    DRILL-6381: (Part 2) MapRDB plugin update to 6.0.1
    
      1. MD-3960: Update Drill to build with MapR-6.0.1 libraries
      2. MD-3995: Do not pushdown limit 0 past project with CONVERT_FROMJSON
      3. MD-4054: Restricted scan limit is changed to dynamically read rows using the rowcount of the rightside instead of 4096.
      4. MD-3688: Impersonating a view owner doesn't work with security disabled in 6.0
      5. MD-4492: Missing limit pushdown changes in JsonTableGroupScan
    
    Co-authored-by: chunhui-shi <cs...@maprtech.com>
    Co-authored-by: Gautam Parai <gp...@maprtech.com>
    Co-authored-by: Vlad Rozov <vr...@mapr.com>
    
    Conflicts:
    	contrib/format-maprdb/src/main/java/org/apache/drill/exec/store/mapr/db/MapRDBFormatPlugin.java
    	contrib/format-maprdb/src/main/java/org/apache/drill/exec/store/mapr/db/MapRDBSubScan.java
    	contrib/format-maprdb/src/main/java/org/apache/drill/exec/store/mapr/db/binary/BinaryTableGroupScan.java
    	contrib/format-maprdb/src/main/java/org/apache/drill/exec/store/mapr/db/json/CompareFunctionsProcessor.java
    	contrib/format-maprdb/src/main/java/org/apache/drill/exec/store/mapr/db/json/JsonConditionBuilder.java
    	contrib/format-maprdb/src/main/java/org/apache/drill/exec/store/mapr/db/json/JsonTableGroupScan.java
    	contrib/format-maprdb/src/main/java/org/apache/drill/exec/store/mapr/db/json/MaprDBJsonRecordReader.java
    	exec/java-exec/src/main/java/org/apache/drill/exec/planner/physical/PrelUtil.java
    	pom.xml
---
 contrib/format-maprdb/pom.xml                      |  89 ++++
 .../planner/index/MapRDBFunctionalIndexInfo.java   | 163 +++++++
 .../drill/exec/planner/index/MapRDBStatistics.java | 337 +++++++++++++
 .../planner/index/MapRDBStatisticsPayload.java     |  56 +++
 .../drill/exec/store/mapr/PluginConstants.java     |  90 ++++
 .../drill/exec/store/mapr/PluginErrorHandler.java  |  50 ++
 .../drill/exec/store/mapr/db/MapRDBCost.java       | 104 ++++
 .../exec/store/mapr/db/MapRDBFormatMatcher.java    |   4 +-
 .../exec/store/mapr/db/MapRDBFormatPlugin.java     |  67 ++-
 .../store/mapr/db/MapRDBFormatPluginConfig.java    |   4 +
 .../drill/exec/store/mapr/db/MapRDBGroupScan.java  |  75 ++-
 .../store/mapr/db/MapRDBPushFilterIntoScan.java    |   3 +-
 .../store/mapr/db/MapRDBPushLimitIntoScan.java     | 157 ++++++
 .../store/mapr/db/MapRDBPushProjectIntoScan.java   | 141 ++++++
 .../mapr/db/MapRDBRestrictedScanBatchCreator.java  |  50 ++
 .../exec/store/mapr/db/MapRDBScanBatchCreator.java |   6 +-
 .../drill/exec/store/mapr/db/MapRDBSubScan.java    |  23 +-
 .../exec/store/mapr/db/MapRDBSubScanSpec.java      |  54 ++-
 .../drill/exec/store/mapr/db/MapRDBTableCache.java | 232 +++++++++
 .../store/mapr/db/RestrictedMapRDBSubScan.java     |  80 +++
 .../store/mapr/db/RestrictedMapRDBSubScanSpec.java | 219 +++++++++
 .../store/mapr/db/binary/BinaryTableGroupScan.java |  34 +-
 .../store/mapr/db/json/AllTextValueWriter.java     |  80 +++
 .../mapr/db/json/CompareFunctionsProcessor.java    |  15 +-
 .../mapr/db/json/DocumentReaderVectorWriter.java   |  42 ++
 .../exec/store/mapr/db/json/FieldPathHelper.java   |  75 +++
 .../mapr/db/json/FieldTransferVectorWriter.java    |  49 ++
 .../store/mapr/db/json/IdOnlyVectorWriter.java     |  65 +++
 .../store/mapr/db/json/JsonConditionBuilder.java   |  59 ++-
 .../exec/store/mapr/db/json/JsonScanSpec.java      |  93 +++-
 .../exec/store/mapr/db/json/JsonSubScanSpec.java   |  69 +--
 .../store/mapr/db/json/JsonTableGroupScan.java     | 485 ++++++++++++++++---
 .../store/mapr/db/json/MaprDBJsonRecordReader.java | 538 ++++++++-------------
 .../mapr/db/json/NumbersAsDoubleValueWriter.java   |  51 ++
 .../store/mapr/db/json/OjaiFunctionsProcessor.java | 214 ++++++++
 .../exec/store/mapr/db/json/OjaiValueWriter.java   | 194 ++++++++
 .../db/json/ProjectionPassthroughVectorWriter.java |  83 ++++
 .../mapr/db/json/RestrictedJsonRecordReader.java   | 248 ++++++++++
 .../store/mapr/db/json/RowCountVectorWriter.java   |  40 ++
 .../exec/udf/mapr/db/ConditionPlaceholder.java     |  54 +++
 .../drill/exec/udf/mapr/db/DecodeFieldPath.java    |  65 +++
 .../drill/exec/udf/mapr/db/MatchesPlaceholder.java |  54 +++
 .../exec/udf/mapr/db/NotMatchesPlaceholder.java    |  54 +++
 .../exec/udf/mapr/db/NotTypeOfPlaceholder.java     |  54 +++
 .../drill/exec/udf/mapr/db/SizeOfPlaceholder.java  |  55 +++
 .../drill/exec/udf/mapr/db/TypeOfPlaceholder.java  |  54 +++
 .../src/main/resources/drill-module.conf           |  20 +-
 .../mapr/drill/maprdb/tests/MaprDBTestsSuite.java  |  63 +--
 .../mapr/drill/maprdb/tests/json/BaseJsonTest.java |  13 +
 .../maprdb/tests/json/TestEncodedFieldPaths.java   | 128 +++++
 .../maprdb/tests/json/TestFieldPathHelper.java     |  52 ++
 .../drill/maprdb/tests/json/TestScanRanges.java    | 158 ++++++
 .../drill/maprdb/tests/json/TestSimpleJson.java    | 216 +++++----
 .../{ => com/mapr/drill}/json/business.json        |   0
 .../mapr/drill/json/encoded_fields_userdata.json   |   5 +
 .../resources/{hbase-site.xml => core-site.xml}    |   5 +
 .../exec/physical/base/AbstractGroupScan.java      |   6 +
 .../apache/drill/exec/physical/base/GroupScan.java |   3 +
 .../drill/exec/planner/physical/PrelUtil.java      | 231 +++++++++
 .../java/org/apache/drill/test/BaseTestQuery.java  |  12 +-
 pom.xml                                            |   4 +-
 61 files changed, 5079 insertions(+), 665 deletions(-)

diff --git a/contrib/format-maprdb/pom.xml b/contrib/format-maprdb/pom.xml
index 816eafb..bfc3131 100644
--- a/contrib/format-maprdb/pom.xml
+++ b/contrib/format-maprdb/pom.xml
@@ -83,6 +83,41 @@
           </systemProperties>
         </configuration>
       </plugin>
+      <plugin>
+        <groupId>org.apache.maven.plugins</groupId>
+        <artifactId>maven-jar-plugin</artifactId>
+        <configuration>
+          <excludes>
+            <!-- do not package test configurations in the jar files as it can override
+                 the installed one -->
+            <exclude>**/core-site.xml</exclude>
+            <exclude>**/logback.xml</exclude>
+          </excludes>
+        </configuration>
+      </plugin>
+
+      <!-- we need the source plugin for the UDFs -->
+      <plugin>
+        <groupId>org.codehaus.mojo</groupId>
+        <artifactId>build-helper-maven-plugin</artifactId>
+        <version>1.9.1</version>
+        <executions>
+          <execution>
+            <id>add-sources-as-resources</id>
+            <phase>process-sources</phase>
+            <goals>
+              <goal>add-resource</goal>
+            </goals>
+            <configuration>
+              <resources>
+                <resource>
+                  <directory>src/main/java</directory>
+                </resource>
+              </resources>
+            </configuration>
+          </execution>
+        </executions>
+      </plugin>
     </plugins>
   </build>
 
@@ -132,6 +167,14 @@
           <artifactId>log4j-over-slf4j</artifactId>
           <groupId>org.slf4j</groupId>
         </exclusion>
+        <exclusion>
+            <groupId>io.netty</groupId>
+            <artifactId>netty</artifactId>
+        </exclusion>
+        <exclusion>
+            <groupId>io.netty</groupId>
+            <artifactId>netty-all</artifactId>
+        </exclusion>
       </exclusions>
     </dependency>
     <dependency>
@@ -140,6 +183,24 @@
     </dependency>
     <!-- Test dependencies -->
     <dependency>
+      <groupId>com.mapr.db</groupId>
+      <artifactId>maprdb</artifactId>
+      <version>${mapr.release.version}</version>
+      <classifier>tests</classifier>
+    </dependency>
+    <dependency>
+      <groupId>com.mapr.hadoop</groupId>
+      <artifactId>maprfs</artifactId>
+      <version>${mapr.release.version}</version>
+      <classifier>tests</classifier>
+    </dependency>
+    <dependency>
+      <groupId>com.jcraft</groupId>
+      <artifactId>jsch</artifactId>
+      <version>0.1.54</version>
+      <scope>test</scope>
+    </dependency>
+    <dependency>
       <groupId>com.mapr</groupId>
       <artifactId>mapr-java-utils</artifactId>
       <version>${mapr.release.version}</version>
@@ -157,6 +218,16 @@
       <version>${project.version}</version>
       <classifier>tests</classifier>
       <scope>test</scope>
+      <exclusions>
+        <exclusion>
+            <groupId>io.netty</groupId>
+            <artifactId>netty</artifactId>
+        </exclusion>
+        <exclusion>
+            <groupId>io.netty</groupId>
+            <artifactId>netty-all</artifactId>
+        </exclusion>
+      </exclusions>
     </dependency>
     <dependency>
       <groupId>org.apache.drill</groupId>
@@ -169,6 +240,14 @@
           <artifactId>log4j-over-slf4j</artifactId>
           <groupId>org.slf4j</groupId>
         </exclusion>
+        <exclusion>
+            <groupId>io.netty</groupId>
+            <artifactId>netty</artifactId>
+        </exclusion>
+        <exclusion>
+            <groupId>io.netty</groupId>
+            <artifactId>netty-all</artifactId>
+        </exclusion>
       </exclusions>
     </dependency>
     <dependency>
@@ -177,6 +256,16 @@
       <version>${project.version}</version>
       <classifier>tests</classifier>
       <scope>test</scope>
+      <exclusions>
+        <exclusion>
+            <groupId>io.netty</groupId>
+            <artifactId>netty</artifactId>
+        </exclusion>
+        <exclusion>
+            <groupId>io.netty</groupId>
+            <artifactId>netty-all</artifactId>
+        </exclusion>
+      </exclusions>
     </dependency>
 
   </dependencies>
diff --git a/contrib/format-maprdb/src/main/java/org/apache/drill/exec/planner/index/MapRDBFunctionalIndexInfo.java b/contrib/format-maprdb/src/main/java/org/apache/drill/exec/planner/index/MapRDBFunctionalIndexInfo.java
new file mode 100644
index 0000000..01561a3
--- /dev/null
+++ b/contrib/format-maprdb/src/main/java/org/apache/drill/exec/planner/index/MapRDBFunctionalIndexInfo.java
@@ -0,0 +1,163 @@
+/*
+ * 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 com.google.common.collect.Maps;
+import com.google.common.collect.Sets;
+import org.apache.drill.common.expression.CastExpression;
+import org.apache.drill.common.expression.LogicalExpression;
+import org.apache.drill.common.expression.SchemaPath;
+
+import java.util.Map;
+import java.util.Set;
+
+public class MapRDBFunctionalIndexInfo implements FunctionalIndexInfo {
+
+  final private IndexDescriptor indexDesc;
+
+  private boolean hasFunctionalField = false;
+
+  //when we scan schemaPath in groupscan's columns, we check if this column(schemaPath) should be rewritten to '$N',
+  //When there are more than two functions on the same column in index, CAST(a.b as INT), CAST(a.b as VARCHAR),
+  // then we should map SchemaPath a.b to a set of SchemaPath, e.g. $1, $2
+  private Map<SchemaPath, Set<SchemaPath>> columnToConvert;
+
+  // map of functional index expression to destination SchemaPath e.g. $N
+  private Map<LogicalExpression, LogicalExpression> exprToConvert;
+
+  //map of SchemaPath involved in a functional field
+  private Map<LogicalExpression, Set<SchemaPath>> pathsInExpr;
+
+  private Set<SchemaPath> newPathsForIndexedFunction;
+
+  private Set<SchemaPath> allPathsInFunction;
+
+  public MapRDBFunctionalIndexInfo(IndexDescriptor indexDesc) {
+    this.indexDesc = indexDesc;
+    columnToConvert = Maps.newHashMap();
+    exprToConvert = Maps.newHashMap();
+    pathsInExpr = Maps.newHashMap();
+    //keep the order of new paths, it may be related to the naming policy
+    newPathsForIndexedFunction = Sets.newLinkedHashSet();
+    allPathsInFunction = Sets.newHashSet();
+    init();
+  }
+
+  private void init() {
+    int count = 0;
+    for(LogicalExpression indexedExpr : indexDesc.getIndexColumns()) {
+      if( !(indexedExpr instanceof SchemaPath) ) {
+        hasFunctionalField = true;
+        SchemaPath functionalFieldPath = SchemaPath.getSimplePath("$"+count);
+        newPathsForIndexedFunction.add(functionalFieldPath);
+
+        //now we handle only cast expression
+        if(indexedExpr instanceof CastExpression) {
+          //We handle only CAST directly on SchemaPath for now.
+          SchemaPath pathBeingCasted = (SchemaPath)((CastExpression) indexedExpr).getInput();
+          addTargetPathForOriginalPath(pathBeingCasted, functionalFieldPath);
+          addPathInExpr(indexedExpr, pathBeingCasted);
+          exprToConvert.put(indexedExpr, functionalFieldPath);
+          allPathsInFunction.add(pathBeingCasted);
+        }
+
+        count++;
+      }
+    }
+  }
+
+  private void addPathInExpr(LogicalExpression expr, SchemaPath path) {
+    if (!pathsInExpr.containsKey(expr)) {
+      Set<SchemaPath> newSet = Sets.newHashSet();
+      newSet.add(path);
+      pathsInExpr.put(expr, newSet);
+    }
+    else {
+      pathsInExpr.get(expr).add(path);
+    }
+  }
+
+  private void addTargetPathForOriginalPath(SchemaPath origPath, SchemaPath newPath) {
+    if (!columnToConvert.containsKey(origPath)) {
+      Set<SchemaPath> newSet = Sets.newHashSet();
+      newSet.add(newPath);
+      columnToConvert.put(origPath, newSet);
+    }
+    else {
+      columnToConvert.get(origPath).add(newPath);
+    }
+  }
+
+
+  public boolean hasFunctional() {
+    return hasFunctionalField;
+  }
+
+  public IndexDescriptor getIndexDesc() {
+    return indexDesc;
+  }
+
+  /**
+   * getNewPath: for an original path, return new rename '$N' path, notice there could be multiple renamed paths
+   * if the there are multiple functional indexes refer original path.
+   * @param path
+   * @return
+   */
+  public SchemaPath getNewPath(SchemaPath path) {
+    if(columnToConvert.containsKey(path)) {
+      return columnToConvert.get(path).iterator().next();
+    }
+    return null;
+  }
+
+  /**
+   * return a plain field path if the incoming index expression 'expr' is replaced to be a plain field
+   * @param expr suppose to be an indexed expression
+   * @return the renamed schemapath in index table for the indexed expression
+   */
+  public SchemaPath getNewPathFromExpr(LogicalExpression expr) {
+    if(exprToConvert.containsKey(expr)) {
+      return (SchemaPath)exprToConvert.get(expr);
+    }
+    return null;
+  }
+
+  /**
+   * @return the map of indexed expression --> the involved schema paths in a indexed expression
+   */
+  public Map<LogicalExpression, Set<SchemaPath>> getPathsInFunctionExpr() {
+    return pathsInExpr;
+  }
+
+
+  public Map<LogicalExpression, LogicalExpression> getExprMap() {
+    return exprToConvert;
+  }
+
+  public Set<SchemaPath> allNewSchemaPaths() {
+    return newPathsForIndexedFunction;
+  }
+
+  public Set<SchemaPath> allPathsInFunction() {
+    return allPathsInFunction;
+  }
+
+  public boolean supportEqualCharConvertToLike() {
+    return true;
+  }
+}
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
new file mode 100644
index 0000000..dc89a4d
--- /dev/null
+++ b/contrib/format-maprdb/src/main/java/org/apache/drill/exec/planner/index/MapRDBStatistics.java
@@ -0,0 +1,337 @@
+/*
+ * 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.calcite.plan.RelOptUtil;
+import org.apache.calcite.rel.RelNode;
+import org.apache.calcite.rel.type.RelDataType;
+import org.apache.calcite.rex.RexCall;
+import org.apache.calcite.rex.RexNode;
+import org.apache.calcite.rex.RexInputRef;
+import org.apache.calcite.sql.SqlKind;
+import org.apache.drill.exec.physical.base.DbGroupScan;
+import org.apache.drill.exec.planner.common.DrillScanRelBase;
+import org.apache.drill.exec.planner.logical.DrillScanRel;
+import org.apache.drill.exec.planner.physical.ScanPrel;
+import org.ojai.store.QueryCondition;
+
+import java.util.HashMap;
+import java.util.Map;
+
+public class MapRDBStatistics implements Statistics {
+  static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(MapRDBStatistics.class);
+  static final String nullConditionAsString = "<NULL>";
+  private double rowKeyJoinBackIOFactor = 1.0;
+  private boolean statsAvailable = false;
+  private StatisticsPayload fullTableScanPayload = null;
+  /*
+   * The computed statistics are cached in <statsCache> so that any subsequent calls are returned
+   * from the cache. The <statsCache> is a map of <RexNode, map<Index, Stats Payload>>. The <RexNode>
+   * does not have a comparator so it is converted to a String for serving as a Map key. This may result
+   * in logically equivalent conditions considered differently e.g. sal<10 OR sal>100, sal>100 OR sal<10
+   * the second map maintains statistics per index as not all statistics are independent of the index
+   * e.g. average row size.
+   */
+  private Map<String, Map<String, StatisticsPayload>> statsCache;
+  /*
+   * The filter independent computed statistics are cached in <fIStatsCache> so that any subsequent
+   * calls are returned from the cache. The <fIStatsCache> is a map of <Index, Stats Payload>. This
+   * cache maintains statistics per index as not all statistics are independent of the index
+   * e.g. average row size.
+   */
+  private Map<String, StatisticsPayload> fIStatsCache;
+  /*
+  /*
+   * The mapping between <QueryCondition> and <RexNode> is kept in <conditionRexNodeMap>. This mapping
+   * is useful to obtain rowCount for condition specified as <QueryCondition> required during physical
+   * planning. Again, both the <QueryCondition> and <RexNode> are converted to Strings for the lack
+   * of a comparator.
+   */
+  private Map<String, String> conditionRexNodeMap;
+
+  public MapRDBStatistics() {
+    statsCache = new HashMap<>();
+    fIStatsCache = new HashMap<>();
+    conditionRexNodeMap = new HashMap<>();
+  }
+
+  public double getRowKeyJoinBackIOFactor() {
+    return rowKeyJoinBackIOFactor;
+  }
+
+  @Override
+  public boolean isStatsAvailable() {
+    return statsAvailable;
+  }
+
+  @Override
+  public String buildUniqueIndexIdentifier(IndexDescriptor idx) {
+    if (idx == null) {
+      return null;
+    } else {
+      return idx.getTableName() + "_" + idx.getIndexName();
+    }
+  }
+
+  public String buildUniqueIndexIdentifier(String tableName, String idxName) {
+    if (tableName == null || idxName == null) {
+      return null;
+    } else {
+      return tableName + "_" + idxName;
+    }
+  }
+
+  @Override
+  /** Returns the number of rows satisfying the given FILTER condition
+   *  @param condition - FILTER specified as a {@link RexNode}
+   *  @param tabIdxName - The table/index identifier
+   *  @return approximate rows satisfying the filter
+   */
+  public double getRowCount(RexNode condition, String tabIdxName, RelNode scanRel) {
+    String conditionAsStr = nullConditionAsString;
+    Map<String, StatisticsPayload> payloadMap;
+    if ((scanRel instanceof DrillScanRel && ((DrillScanRel)scanRel).getGroupScan() instanceof DbGroupScan)
+        || (scanRel instanceof ScanPrel && ((ScanPrel)scanRel).getGroupScan() instanceof DbGroupScan)) {
+      if (condition == null && fullTableScanPayload != null) {
+        return fullTableScanPayload.getRowCount();
+      } else if (condition != null) {
+        conditionAsStr = convertRexToString(condition, scanRel.getRowType());
+        payloadMap = statsCache.get(conditionAsStr);
+        if (payloadMap != null) {
+          if (payloadMap.get(tabIdxName) != null) {
+            return payloadMap.get(tabIdxName).getRowCount();
+          } else {
+            // We might not have computed rowcount for the given condition from the tab/index in question.
+            // For rowcount it does not matter which index was used to get the rowcount for the given condition.
+            // Hence, just use the first one!
+            for (String payloadKey : payloadMap.keySet()) {
+              if (payloadKey != null && payloadMap.get(payloadKey) != null) {
+                return payloadMap.get(payloadKey).getRowCount();
+              }
+            }
+            StatisticsPayload anyPayload = payloadMap.entrySet().iterator().next().getValue();
+            return anyPayload.getRowCount();
+          }
+        }
+      }
+    }
+    if (statsAvailable) {
+      logger.debug("Statistics: Filter row count is UNKNOWN for filter: {}", conditionAsStr);
+    }
+    return ROWCOUNT_UNKNOWN;
+  }
+
+  /** Returns the number of rows satisfying the given FILTER condition
+   *  @param condition - FILTER specified as a {@link QueryCondition}
+   *  @param tabIdxName - The table/index identifier
+   *  @return approximate rows satisfying the filter
+   */
+  public double getRowCount(QueryCondition condition, String tabIdxName) {
+    String conditionAsStr = nullConditionAsString;
+    Map<String, StatisticsPayload> payloadMap;
+    if (condition != null
+        && conditionRexNodeMap.get(condition.toString()) != null) {
+      String rexConditionAsString = conditionRexNodeMap.get(condition.toString());
+      payloadMap = statsCache.get(rexConditionAsString);
+      if (payloadMap != null) {
+        if (payloadMap.get(tabIdxName) != null) {
+          return payloadMap.get(tabIdxName).getRowCount();
+        } else {
+          // We might not have computed rowcount for the given condition from the tab/index in question.
+          // For rowcount it does not matter which index was used to get the rowcount for the given condition.
+          // if tabIdxName is null, most likely we have found one from payloadMap and won't come to here.
+          // If we come to here, we are looking for payload for an index, so let us use any index's payload first!
+          for (String payloadKey : payloadMap.keySet()) {
+            if (payloadKey != null && payloadMap.get(payloadKey) != null) {
+              return payloadMap.get(payloadKey).getRowCount();
+            }
+          }
+          StatisticsPayload anyPayload = payloadMap.entrySet().iterator().next().getValue();
+          return anyPayload.getRowCount();
+        }
+      }
+    } else if (condition == null
+        && fullTableScanPayload != null) {
+      return fullTableScanPayload.getRowCount();
+    }
+    if (condition != null) {
+      conditionAsStr = condition.toString();
+    }
+    if (statsAvailable) {
+      logger.debug("Statistics: Filter row count is UNKNOWN for filter: {}", conditionAsStr);
+    }
+    return ROWCOUNT_UNKNOWN;
+  }
+
+  /** Returns the number of leading rows satisfying the given FILTER condition
+   *  @param condition - FILTER specified as a {@link RexNode}
+   *  @param tabIdxName - The table/index identifier
+   *  @param scanRel - The current scanRel
+   *  @return approximate rows satisfying the leading filter
+   */
+  @Override
+  public double getLeadingRowCount(RexNode condition, String tabIdxName, RelNode scanRel) {
+    String conditionAsStr = nullConditionAsString;
+    Map<String, StatisticsPayload> payloadMap;
+    if ((scanRel instanceof DrillScanRel && ((DrillScanRel)scanRel).getGroupScan() instanceof DbGroupScan)
+        || (scanRel instanceof ScanPrel && ((ScanPrel)scanRel).getGroupScan() instanceof DbGroupScan)) {
+      if (condition == null && fullTableScanPayload != null) {
+        return fullTableScanPayload.getLeadingRowCount();
+      } else if (condition != null) {
+        conditionAsStr = convertRexToString(condition, scanRel.getRowType());
+        payloadMap = statsCache.get(conditionAsStr);
+        if (payloadMap != null) {
+          if (payloadMap.get(tabIdxName) != null) {
+            return payloadMap.get(tabIdxName).getLeadingRowCount();
+          }
+          // Unlike rowcount, leading rowcount is dependent on the index. So, if tab/idx is
+          // not found, we are out of luck!
+        }
+      }
+    }
+    if (statsAvailable) {
+      logger.debug("Statistics: Leading filter row count is UNKNOWN for filter: {}", conditionAsStr);
+    }
+    return ROWCOUNT_UNKNOWN;
+  }
+
+  /** Returns the number of leading rows satisfying the given FILTER condition
+   *  @param condition - FILTER specified as a {@link QueryCondition}
+   *  @param tabIdxName - The table/index identifier
+   *  @return approximate rows satisfying the leading filter
+   */
+  public double getLeadingRowCount(QueryCondition condition, String tabIdxName) {
+    String conditionAsStr = nullConditionAsString;
+    Map<String, StatisticsPayload> payloadMap;
+    if (condition != null
+        && conditionRexNodeMap.get(condition.toString()) != null) {
+      String rexConditionAsString = conditionRexNodeMap.get(condition.toString());
+      payloadMap = statsCache.get(rexConditionAsString);
+      if (payloadMap != null) {
+        if (payloadMap.get(tabIdxName) != null) {
+          return payloadMap.get(tabIdxName).getLeadingRowCount();
+        }
+        // Unlike rowcount, leading rowcount is dependent on the index. So, if tab/idx is
+        // not found, we are out of luck!
+      }
+    } else if (condition == null
+        && fullTableScanPayload != null) {
+      return fullTableScanPayload.getLeadingRowCount();
+    }
+    if (condition != null) {
+      conditionAsStr = condition.toString();
+    }
+    if (statsAvailable) {
+      logger.debug("Statistics: Leading filter row count is UNKNOWN for filter: {}", conditionAsStr);
+    }
+    return ROWCOUNT_UNKNOWN;
+  }
+
+  @Override
+  public double getAvgRowSize(String tabIdxName, boolean isTableScan) {
+    StatisticsPayload payloadMap;
+    if (isTableScan && fullTableScanPayload != null) {
+      return fullTableScanPayload.getAvgRowSize();
+    } else if (!isTableScan) {
+      payloadMap = fIStatsCache.get(tabIdxName);
+      if (payloadMap != null) {
+        return payloadMap.getAvgRowSize();
+      }
+    }
+    if (statsAvailable) {
+      logger.debug("Statistics: Average row size is UNKNOWN for table: {}", tabIdxName);
+    }
+    return AVG_ROWSIZE_UNKNOWN;
+  }
+
+  public boolean initialize(RexNode condition, DrillScanRelBase scanRel, IndexCallContext context) {
+    //XXX to implement for complete secondary index framework
+    return false;
+  }
+
+  /*
+ * Convert the given RexNode to a String representation while also replacing the RexInputRef references
+ * to actual column names. Since, we compare String representations of RexNodes, two equivalent RexNode
+ * expressions may differ in the RexInputRef positions but otherwise the same.
+ * e.g. $1 = 'CA' projection (State, Country) , $2 = 'CA' projection (Country, State)
+ */
+  private String convertRexToString(RexNode condition, RelDataType rowType) {
+    StringBuilder sb = new StringBuilder();
+    if (condition == null) {
+      return null;
+    }
+    if (condition.getKind() == SqlKind.AND) {
+      boolean first = true;
+      for(RexNode pred : RelOptUtil.conjunctions(condition)) {
+        if (first) {
+          sb.append(convertRexToString(pred, rowType));
+          first = false;
+        } else {
+          sb.append(" " + SqlKind.AND.toString() + " ");
+          sb.append(convertRexToString(pred, rowType));
+        }
+      }
+      return sb.toString();
+    } else if (condition.getKind() == SqlKind.OR) {
+      boolean first = true;
+      for(RexNode pred : RelOptUtil.disjunctions(condition)) {
+        if (first) {
+          sb.append(convertRexToString(pred, rowType));
+          first = false;
+        } else {
+          sb.append(" " + SqlKind.OR.toString() + " ");
+          sb.append(convertRexToString(pred, rowType));
+        }
+      }
+      return sb.toString();
+    } else {
+      HashMap<String, String> inputRefMapping = new HashMap<>();
+      /* Based on the rel projection the input reference for the same column may change
+       * during planning. We want the cache to be agnostic to it. Hence, the entry stored
+       * in the cache has the input reference ($i) replaced with the column name
+       */
+      getInputRefMapping(condition, rowType, inputRefMapping);
+      if (inputRefMapping.keySet().size() > 0) {
+        //Found input ref - replace it
+        String replCondition = condition.toString();
+        for (String inputRef : inputRefMapping.keySet()) {
+          replCondition = replCondition.replace(inputRef, inputRefMapping.get(inputRef));
+        }
+        return replCondition;
+      } else {
+        return condition.toString();
+      }
+    }
+  }
+
+  /*
+ * Generate the input reference to column mapping for reference replacement. Please
+ * look at the usage in convertRexToString() to understand why this mapping is required.
+ */
+  private void getInputRefMapping(RexNode condition, RelDataType rowType,
+                                  HashMap<String, String> mapping) {
+    if (condition instanceof RexCall) {
+      for (RexNode op : ((RexCall) condition).getOperands()) {
+        getInputRefMapping(op, rowType, mapping);
+      }
+    } else if (condition instanceof RexInputRef) {
+      mapping.put(condition.toString(),
+          rowType.getFieldNames().get(condition.hashCode()));
+    }
+  }
+}
diff --git a/contrib/format-maprdb/src/main/java/org/apache/drill/exec/planner/index/MapRDBStatisticsPayload.java b/contrib/format-maprdb/src/main/java/org/apache/drill/exec/planner/index/MapRDBStatisticsPayload.java
new file mode 100644
index 0000000..4930282
--- /dev/null
+++ b/contrib/format-maprdb/src/main/java/org/apache/drill/exec/planner/index/MapRDBStatisticsPayload.java
@@ -0,0 +1,56 @@
+/*
+ * 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;
+
+public class MapRDBStatisticsPayload implements StatisticsPayload {
+
+  double rowCount;
+  double leadingRowCount;
+  double avgRowSize;
+
+  public MapRDBStatisticsPayload(double rowCount, double leadingRowCount, double avgRowSize) {
+    this.rowCount = rowCount;
+    this.leadingRowCount = leadingRowCount;
+    this.avgRowSize = avgRowSize;
+  }
+
+  @Override
+  public String toString() {
+    return "MapRDBStatisticsPayload{" +
+        "rowCount=" + rowCount +
+        ", leadingRowCount=" + leadingRowCount +
+        ", avgRowSize=" + avgRowSize +
+        '}';
+  }
+
+  @Override
+  public double getRowCount() {
+    return rowCount;
+  }
+
+  @Override
+  public double getLeadingRowCount() {
+    return  leadingRowCount;
+  }
+
+  @Override
+  public double getAvgRowSize() {
+    return avgRowSize;
+  }
+
+}
diff --git a/contrib/format-maprdb/src/main/java/org/apache/drill/exec/store/mapr/PluginConstants.java b/contrib/format-maprdb/src/main/java/org/apache/drill/exec/store/mapr/PluginConstants.java
new file mode 100644
index 0000000..4239c5d
--- /dev/null
+++ b/contrib/format-maprdb/src/main/java/org/apache/drill/exec/store/mapr/PluginConstants.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.store.mapr;
+
+import static org.ojai.DocumentConstants.ID_KEY;
+import org.apache.drill.common.expression.SchemaPath;
+import com.mapr.db.DBConstants;
+import org.apache.drill.exec.planner.cost.DrillCostBase;
+import org.apache.drill.exec.planner.cost.PluginCost.CheckValid;
+
+public class PluginConstants {
+  static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(PluginConstants.class);
+
+  public final static CheckValid alwaysValid = new CheckValid<Integer>() {
+    @Override
+    public boolean isValid(Integer parameter) {
+      return true;
+    }
+  };
+
+  public final static CheckValid isNonNegative = new CheckValid<Integer>() {
+    @Override
+    public boolean isValid(Integer paramValue) {
+      if (paramValue > 0 && paramValue <= Integer.MAX_VALUE) {
+        return true;
+      } else {
+        logger.warn("Setting default value as the supplied parameter value is less than/equals to 0");
+        return false;
+      }
+    }
+  };
+
+  public static final String SSD = "SSD";
+  public static final String HDD = "HDD";
+  public static final SchemaPath ID_SCHEMA_PATH = SchemaPath.getSimplePath(ID_KEY);
+
+  public static final SchemaPath DOCUMENT_SCHEMA_PATH = SchemaPath.getSimplePath(DBConstants.DOCUMENT_FIELD);
+
+  public static final int JSON_TABLE_NUM_TABLETS_PER_INDEX_DEFAULT = 32;
+
+  public static final String JSON_TABLE_SCAN_SIZE_MB = "format-maprdb.json.scanSizeMB";
+  public static final int JSON_TABLE_SCAN_SIZE_MB_DEFAULT = 128;
+
+  public static final String JSON_TABLE_RESTRICTED_SCAN_SIZE_MB = "format-maprdb.json.restrictedScanSizeMB";
+  public static final int JSON_TABLE_RESTRICTED_SCAN_SIZE_MB_DEFAULT = 4096;
+
+  public static final String JSON_TABLE_USE_NUM_REGIONS_FOR_DISTRIBUTION_PLANNING = "format-maprdb.json.useNumRegionsForDistribution";
+  public static final boolean JSON_TABLE_USE_NUM_REGIONS_FOR_DISTRIBUTION_PLANNING_DEFAULT = false;
+
+  public static final String JSON_TABLE_BLOCK_SIZE = "format-maprdb.json.pluginCost.blockSize";
+  public static final int JSON_TABLE_BLOCK_SIZE_DEFAULT = 8192;
+
+  public static final String JSON_TABLE_MEDIA_TYPE = "format-maprdb.json.mediaType";
+  public static final String JSON_TABLE_MEDIA_TYPE_DEFAULT = SSD;
+
+  public static final String JSON_TABLE_SSD_BLOCK_SEQ_READ_COST = "format-maprdb.json.pluginCost.ssdBlockSequentialReadCost";
+  public static final int JSON_TABLE_SSD_BLOCK_SEQ_READ_COST_DEFAULT = 32 * DrillCostBase.BASE_CPU_COST * JSON_TABLE_BLOCK_SIZE_DEFAULT;
+
+  // for SSD random and sequential costs are the same
+  public static final String JSON_TABLE_SSD_BLOCK_RANDOM_READ_COST = "format-maprdb.json.pluginCost.ssdBlockRandomReadCost";
+  public static final int JSON_TABLE_SSD_BLOCK_RANDOM_READ_COST_DEFAULT = JSON_TABLE_SSD_BLOCK_SEQ_READ_COST_DEFAULT;
+
+  public static final String JSON_TABLE_AVERGE_COLUMN_SIZE = "format-maprdb.json.pluginCost.averageColumnSize";
+  public static final int JSON_TABLE_AVERGE_COLUMN_SIZE_DEFAULT = 10;
+
+  public static final int TABLE_BLOCK_SIZE_DEFAULT = 8192;
+  public static final int TABLE_SSD_BLOCK_SEQ_READ_COST_DEFAULT = 32 * DrillCostBase.BASE_CPU_COST * TABLE_BLOCK_SIZE_DEFAULT;
+  public static final int TABLE_SSD_BLOCK_RANDOM_READ_COST_DEFAULT = TABLE_SSD_BLOCK_SEQ_READ_COST_DEFAULT;
+  public static final int TABLE_AVERGE_COLUMN_SIZE_DEFAULT = 10;
+  public static final String JSON_TABLE_HDD_BLOCK_SEQ_READ_COST = "format-maprdb.json.pluginCost.hddBlockSequentialReadCost";
+  public static final int JSON_TABLE_HDD_BLOCK_SEQ_READ_COST_DEFAULT = 6 * JSON_TABLE_SSD_BLOCK_SEQ_READ_COST_DEFAULT;
+
+  public static final String JSON_TABLE_HDD_BLOCK_RANDOM_READ_COST = "format-maprdb.json.pluginCost.hddBlockRandomReadCost";
+  public static final int JSON_TABLE_HDD_BLOCK_RANDOM_READ_COST_DEFAULT = 1000 * JSON_TABLE_HDD_BLOCK_SEQ_READ_COST_DEFAULT;
+}
diff --git a/contrib/format-maprdb/src/main/java/org/apache/drill/exec/store/mapr/PluginErrorHandler.java b/contrib/format-maprdb/src/main/java/org/apache/drill/exec/store/mapr/PluginErrorHandler.java
new file mode 100644
index 0000000..d106d6e
--- /dev/null
+++ b/contrib/format-maprdb/src/main/java/org/apache/drill/exec/store/mapr/PluginErrorHandler.java
@@ -0,0 +1,50 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.drill.exec.store.mapr;
+
+import org.apache.drill.common.exceptions.UserException;
+import org.apache.drill.exec.exception.SchemaChangeException;
+import org.slf4j.Logger;
+
+public final class PluginErrorHandler {
+
+  public static UserException unsupportedError(Logger logger, String format, Object... args) {
+    return UserException.unsupportedError()
+        .message(String.format(format, args))
+        .build(logger);
+  }
+
+  public static UserException dataReadError(Logger logger, Throwable t) {
+    return dataReadError(logger, t, null);
+  }
+
+  public static UserException dataReadError(Logger logger, String format, Object... args) {
+    return dataReadError(null, format, args);
+  }
+
+  public static UserException dataReadError(Logger logger, Throwable t, String format, Object... args) {
+    return UserException.dataReadError(t)
+        .message(format == null ? null : String.format(format, args))
+        .build(logger);
+  }
+
+  public static SchemaChangeException schemaChangeException(Logger logger, Throwable t, String format, Object... args) {
+    return new SchemaChangeException(format, t, args);
+  }
+
+}
diff --git a/contrib/format-maprdb/src/main/java/org/apache/drill/exec/store/mapr/db/MapRDBCost.java b/contrib/format-maprdb/src/main/java/org/apache/drill/exec/store/mapr/db/MapRDBCost.java
new file mode 100644
index 0000000..0c40a02
--- /dev/null
+++ b/contrib/format-maprdb/src/main/java/org/apache/drill/exec/store/mapr/db/MapRDBCost.java
@@ -0,0 +1,104 @@
+/*
+ * 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.store.mapr.db;
+
+import org.apache.drill.common.config.DrillConfig;
+import org.apache.drill.exec.physical.base.GroupScan;
+import org.apache.drill.exec.planner.cost.PluginCost;
+import org.apache.drill.exec.store.mapr.PluginConstants;
+import org.apache.drill.exec.store.mapr.db.json.JsonTableGroupScan;
+
+public class MapRDBCost implements PluginCost {
+
+  private int JSON_AVG_COLUMN_SIZE;
+  private int JSON_TABLE_BLOCK_SIZE;  // bytes per block
+  private int JSON_BLOCK_SEQ_READ_COST;
+  private int JSON_BLOCK_RANDOM_READ_COST;
+  private int JSON_HDD_BLOCK_SEQ_READ_COST;
+  private int JSON_HDD_BLOCK_RANDOM_READ_COST;
+  private int JSON_SSD_BLOCK_SEQ_READ_COST;
+  private int JSON_SSD_BLOCK_RANDOM_READ_COST;
+
+  public MapRDBCost(DrillConfig config, String mediaType) {
+    JSON_AVG_COLUMN_SIZE = setConfigValue(config, PluginConstants.JSON_TABLE_AVERGE_COLUMN_SIZE,
+        PluginConstants.JSON_TABLE_AVERGE_COLUMN_SIZE_DEFAULT, PluginConstants.alwaysValid);
+    JSON_TABLE_BLOCK_SIZE = setConfigValue(config, PluginConstants.JSON_TABLE_BLOCK_SIZE,
+        PluginConstants.JSON_TABLE_BLOCK_SIZE_DEFAULT, PluginConstants.alwaysValid);
+    JSON_SSD_BLOCK_SEQ_READ_COST = setConfigValue(config, PluginConstants.JSON_TABLE_SSD_BLOCK_SEQ_READ_COST,
+        PluginConstants.JSON_TABLE_SSD_BLOCK_SEQ_READ_COST_DEFAULT, PluginConstants.isNonNegative);
+    JSON_SSD_BLOCK_RANDOM_READ_COST = setConfigValue(config, PluginConstants.JSON_TABLE_SSD_BLOCK_RANDOM_READ_COST,
+        PluginConstants.JSON_TABLE_SSD_BLOCK_RANDOM_READ_COST_DEFAULT, new greaterThanEquals(JSON_SSD_BLOCK_SEQ_READ_COST));
+    JSON_HDD_BLOCK_SEQ_READ_COST = setConfigValue(config, PluginConstants.JSON_TABLE_HDD_BLOCK_SEQ_READ_COST,
+            PluginConstants.JSON_TABLE_HDD_BLOCK_SEQ_READ_COST_DEFAULT, PluginConstants.isNonNegative);
+    JSON_HDD_BLOCK_RANDOM_READ_COST = setConfigValue(config, PluginConstants.JSON_TABLE_HDD_BLOCK_RANDOM_READ_COST,
+            PluginConstants.JSON_TABLE_HDD_BLOCK_RANDOM_READ_COST_DEFAULT, new greaterThanEquals(JSON_HDD_BLOCK_SEQ_READ_COST));
+    JSON_BLOCK_SEQ_READ_COST = mediaType.equals(PluginConstants.SSD) ? JSON_SSD_BLOCK_SEQ_READ_COST :
+                                    JSON_HDD_BLOCK_SEQ_READ_COST;
+    JSON_BLOCK_RANDOM_READ_COST = mediaType.equals(PluginConstants.SSD) ? JSON_SSD_BLOCK_RANDOM_READ_COST :
+                                    JSON_HDD_BLOCK_RANDOM_READ_COST;
+  }
+
+  private int setConfigValue(DrillConfig config, String configPath,
+                             int defaultValue, CheckValid check) {
+    int configValue;
+    try {
+      configValue = config.getInt(configPath);
+      if (!check.isValid(configValue)) { configValue = defaultValue; }
+    } catch (Exception ex) {
+      // Use defaults, if config values not present or any other issue
+      configValue = defaultValue;
+    }
+    return configValue;
+  }
+
+  @Override
+  public int getAverageColumnSize(GroupScan scan) {
+    if (scan instanceof JsonTableGroupScan) {
+      return JSON_AVG_COLUMN_SIZE;
+    } else {
+      return PluginConstants.TABLE_AVERGE_COLUMN_SIZE_DEFAULT;
+    }
+  }
+
+  @Override
+  public int getBlockSize(GroupScan scan) {
+    if (scan instanceof JsonTableGroupScan) {
+      return JSON_TABLE_BLOCK_SIZE;
+    } else {
+      return PluginConstants.TABLE_BLOCK_SIZE_DEFAULT;
+    }
+  }
+
+  @Override
+  public int getSequentialBlockReadCost(GroupScan scan) {
+    if (scan instanceof JsonTableGroupScan) {
+      return JSON_BLOCK_SEQ_READ_COST;
+    } else {
+      return PluginConstants.TABLE_SSD_BLOCK_SEQ_READ_COST_DEFAULT;
+    }
+  }
+
+  @Override
+  public int getRandomBlockReadCost(GroupScan scan) {
+    if (scan instanceof JsonTableGroupScan) {
+      return JSON_BLOCK_RANDOM_READ_COST;
+    } else {
+      return PluginConstants.TABLE_SSD_BLOCK_RANDOM_READ_COST_DEFAULT;
+    }
+  }
+}
diff --git a/contrib/format-maprdb/src/main/java/org/apache/drill/exec/store/mapr/db/MapRDBFormatMatcher.java b/contrib/format-maprdb/src/main/java/org/apache/drill/exec/store/mapr/db/MapRDBFormatMatcher.java
index d4978b9..ee35a68 100644
--- a/contrib/format-maprdb/src/main/java/org/apache/drill/exec/store/mapr/db/MapRDBFormatMatcher.java
+++ b/contrib/format-maprdb/src/main/java/org/apache/drill/exec/store/mapr/db/MapRDBFormatMatcher.java
@@ -28,6 +28,7 @@ import org.apache.drill.exec.store.dfs.DrillFileSystem;
 import org.apache.drill.exec.store.dfs.FileSelection;
 import org.apache.drill.exec.store.dfs.FileSystemPlugin;
 import org.apache.drill.exec.store.dfs.FormatSelection;
+
 import org.apache.drill.exec.store.mapr.TableFormatMatcher;
 import org.apache.drill.exec.store.mapr.TableFormatPlugin;
 
@@ -44,8 +45,7 @@ public class MapRDBFormatMatcher extends TableFormatMatcher {
   protected boolean isSupportedTable(MapRFileStatus status) throws IOException {
     return !getFormatPlugin()
         .getMaprFS()
-        .getTableProperties(status.getPath())
-        .getAttr()
+        .getTableBasicAttrs(status.getPath())
         .getIsMarlinTable();
   }
 
diff --git a/contrib/format-maprdb/src/main/java/org/apache/drill/exec/store/mapr/db/MapRDBFormatPlugin.java b/contrib/format-maprdb/src/main/java/org/apache/drill/exec/store/mapr/db/MapRDBFormatPlugin.java
index fb3ee4d..da4829f 100644
--- a/contrib/format-maprdb/src/main/java/org/apache/drill/exec/store/mapr/db/MapRDBFormatPlugin.java
+++ b/contrib/format-maprdb/src/main/java/org/apache/drill/exec/store/mapr/db/MapRDBFormatPlugin.java
@@ -29,6 +29,7 @@ import org.apache.drill.exec.store.StoragePluginOptimizerRule;
 import org.apache.drill.exec.store.dfs.FileSelection;
 import org.apache.drill.exec.store.dfs.FormatMatcher;
 import org.apache.drill.exec.store.hbase.HBaseScanSpec;
+import org.apache.drill.exec.store.mapr.PluginConstants;
 import org.apache.drill.exec.store.mapr.TableFormatPlugin;
 import org.apache.drill.exec.store.mapr.db.binary.BinaryTableGroupScan;
 import org.apache.drill.exec.store.mapr.db.json.JsonScanSpec;
@@ -41,6 +42,7 @@ import org.apache.hadoop.hbase.client.ConnectionFactory;
 
 import com.fasterxml.jackson.annotation.JsonIgnore;
 import org.apache.drill.shaded.guava.com.google.common.collect.ImmutableSet;
+import com.mapr.db.index.IndexDesc;
 import com.mapr.fs.tables.TableProperties;
 
 public class MapRDBFormatPlugin extends TableFormatPlugin {
@@ -49,6 +51,11 @@ public class MapRDBFormatPlugin extends TableFormatPlugin {
   private final MapRDBFormatMatcher matcher;
   private final Configuration hbaseConf;
   private final Connection connection;
+  private final MapRDBTableCache jsonTableCache;
+  private final int scanRangeSizeMB;
+  private final String mediaType;
+  private final MapRDBCost pluginCostModel;
+  private final int restrictedScanRangeSizeMB;
 
   public MapRDBFormatPlugin(String name, DrillbitContext context, Configuration fsConf,
       StoragePluginConfig storageConfig, MapRDBFormatPluginConfig formatConfig) throws IOException {
@@ -57,6 +64,29 @@ public class MapRDBFormatPlugin extends TableFormatPlugin {
     hbaseConf = HBaseConfiguration.create(fsConf);
     hbaseConf.set(ConnectionFactory.DEFAULT_DB, ConnectionFactory.MAPR_ENGINE2);
     connection = ConnectionFactory.createConnection(hbaseConf);
+    jsonTableCache = new MapRDBTableCache(context.getConfig());
+    int scanRangeSizeMBConfig = context.getConfig().getInt(PluginConstants.JSON_TABLE_SCAN_SIZE_MB);
+    if (scanRangeSizeMBConfig < 32 || scanRangeSizeMBConfig > 8192) {
+      logger.warn("Invalid scan size {} for MapR-DB tables, using default", scanRangeSizeMBConfig);
+      scanRangeSizeMBConfig = PluginConstants.JSON_TABLE_SCAN_SIZE_MB_DEFAULT;
+    }
+
+    int restrictedScanRangeSizeMBConfig = context.getConfig().getInt(PluginConstants.JSON_TABLE_RESTRICTED_SCAN_SIZE_MB);
+    if (restrictedScanRangeSizeMBConfig < 32 || restrictedScanRangeSizeMBConfig > 8192) {
+      logger.warn("Invalid restricted scan size {} for MapR-DB tables, using default", restrictedScanRangeSizeMBConfig);
+      restrictedScanRangeSizeMBConfig = PluginConstants.JSON_TABLE_RESTRICTED_SCAN_SIZE_MB_DEFAULT;
+    }
+
+    String mediaTypeConfig = context.getConfig().getString(PluginConstants.JSON_TABLE_MEDIA_TYPE);
+    if (!(mediaTypeConfig.equals(PluginConstants.SSD) ||
+        mediaTypeConfig.equals(PluginConstants.HDD))) {
+      logger.warn("Invalid media Type {} for MapR-DB JSON tables, using default 'SSD'", mediaTypeConfig);
+      mediaTypeConfig = PluginConstants.JSON_TABLE_MEDIA_TYPE_DEFAULT;
+    }
+    mediaType = mediaTypeConfig;
+    scanRangeSizeMB = scanRangeSizeMBConfig;
+    restrictedScanRangeSizeMB = restrictedScanRangeSizeMBConfig;
+    pluginCostModel = new MapRDBCost(context.getConfig(), mediaType);
   }
 
   @Override
@@ -65,19 +95,30 @@ public class MapRDBFormatPlugin extends TableFormatPlugin {
   }
 
   @Override
+  public MapRDBFormatPluginConfig getConfig() {
+    return (MapRDBFormatPluginConfig)(super.getConfig());
+  }
+
+  public MapRDBTableCache getJsonTableCache() {
+    return jsonTableCache;
+  }
+
+  @Override
   @JsonIgnore
   public Set<StoragePluginOptimizerRule> getOptimizerRules() {
-    return ImmutableSet.of(MapRDBPushFilterIntoScan.FILTER_ON_SCAN, MapRDBPushFilterIntoScan.FILTER_ON_PROJECT);
+    return ImmutableSet.of(MapRDBPushFilterIntoScan.FILTER_ON_SCAN, MapRDBPushFilterIntoScan.FILTER_ON_PROJECT,
+        MapRDBPushProjectIntoScan.PROJECT_ON_SCAN, MapRDBPushLimitIntoScan.LIMIT_ON_PROJECT,
+        MapRDBPushLimitIntoScan.LIMIT_ON_SCAN);
   }
 
-  @Override
+
   public AbstractGroupScan getGroupScan(String userName, FileSelection selection,
-      List<SchemaPath> columns) throws IOException {
+      List<SchemaPath> columns, IndexDesc indexDesc) throws IOException {
     String tableName = getTableName(selection);
     TableProperties props = getMaprFS().getTableProperties(new Path(tableName));
 
     if (props.getAttr().getJson()) {
-      JsonScanSpec scanSpec = new JsonScanSpec(tableName, null/*condition*/);
+      JsonScanSpec scanSpec = new JsonScanSpec(tableName, indexDesc, null/*condition*/);
       return new JsonTableGroupScan(userName, getStoragePlugin(), this, scanSpec, columns);
     } else {
       HBaseScanSpec scanSpec = new HBaseScanSpec(tableName);
@@ -85,6 +126,12 @@ public class MapRDBFormatPlugin extends TableFormatPlugin {
     }
   }
 
+  @Override
+  public AbstractGroupScan getGroupScan(String userName, FileSelection selection,
+      List<SchemaPath> columns) throws IOException {
+    return getGroupScan(userName, selection, columns, null /* indexDesc */);
+  }
+
   @JsonIgnore
   public Configuration getHBaseConf() {
     return hbaseConf;
@@ -95,6 +142,18 @@ public class MapRDBFormatPlugin extends TableFormatPlugin {
     return connection;
   }
 
+  public int getScanRangeSizeMB() {
+    return scanRangeSizeMB;
+  }
+
+  public int getRestrictedScanRangeSizeMB() {
+    return restrictedScanRangeSizeMB;
+  }
+
+  public MapRDBCost getPluginCostModel() {
+    return pluginCostModel;
+  }
+
   /**
    * Allows to get a table name from FileSelection object
    *
diff --git a/contrib/format-maprdb/src/main/java/org/apache/drill/exec/store/mapr/db/MapRDBFormatPluginConfig.java b/contrib/format-maprdb/src/main/java/org/apache/drill/exec/store/mapr/db/MapRDBFormatPluginConfig.java
index ad153fe..07943f6 100644
--- a/contrib/format-maprdb/src/main/java/org/apache/drill/exec/store/mapr/db/MapRDBFormatPluginConfig.java
+++ b/contrib/format-maprdb/src/main/java/org/apache/drill/exec/store/mapr/db/MapRDBFormatPluginConfig.java
@@ -38,6 +38,7 @@ public class MapRDBFormatPluginConfig extends TableFormatPluginConfig {
    * is not regressing performance of reading maprdb table.
    */
   public boolean nonExistentFieldSupport = true;
+  public String index = "";
 
   @Override
   public int hashCode() {
@@ -65,6 +66,8 @@ public class MapRDBFormatPluginConfig extends TableFormatPluginConfig {
       return false;
     } else if (nonExistentFieldSupport != other.nonExistentFieldSupport) {
       return false;
+    } else if (!index.equals(other.index)) {
+      return false;
     }
     return true;
   }
@@ -91,4 +94,5 @@ public class MapRDBFormatPluginConfig extends TableFormatPluginConfig {
     return ignoreSchemaChange;
   }
 
+  public String getIndex() { return this.index; }
 }
diff --git a/contrib/format-maprdb/src/main/java/org/apache/drill/exec/store/mapr/db/MapRDBGroupScan.java b/contrib/format-maprdb/src/main/java/org/apache/drill/exec/store/mapr/db/MapRDBGroupScan.java
index 9e6a744..1e6bcec 100644
--- a/contrib/format-maprdb/src/main/java/org/apache/drill/exec/store/mapr/db/MapRDBGroupScan.java
+++ b/contrib/format-maprdb/src/main/java/org/apache/drill/exec/store/mapr/db/MapRDBGroupScan.java
@@ -34,7 +34,11 @@ import java.util.concurrent.TimeUnit;
 import org.apache.drill.common.expression.SchemaPath;
 import org.apache.drill.common.logical.StoragePluginConfig;
 import org.apache.drill.exec.physical.EndpointAffinity;
-import org.apache.drill.exec.physical.base.AbstractGroupScan;
+import org.apache.drill.exec.physical.base.AbstractDbGroupScan;
+import org.apache.calcite.rel.RelNode;
+import org.apache.drill.exec.planner.index.IndexCollection;
+
+import org.apache.drill.exec.planner.cost.PluginCost;
 import org.apache.drill.exec.proto.CoordinationProtos.DrillbitEndpoint;
 import org.apache.drill.exec.store.AbstractStoragePlugin;
 
@@ -46,7 +50,7 @@ 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;
 
-public abstract class MapRDBGroupScan extends AbstractGroupScan {
+public abstract class MapRDBGroupScan extends AbstractDbGroupScan {
   private static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(MapRDBGroupScan.class);
 
   protected AbstractStoragePlugin storagePlugin;
@@ -59,7 +63,9 @@ public abstract class MapRDBGroupScan extends AbstractGroupScan {
 
   protected Map<Integer, List<MapRDBSubScanSpec>> endpointFragmentMapping;
 
-  protected NavigableMap<TabletFragmentInfo, String> regionsToScan;
+  protected NavigableMap<TabletFragmentInfo, String> doNotAccessRegionsToScan;
+
+  protected double costFactor = 1.0;
 
   private boolean filterPushedDown = false;
 
@@ -80,8 +86,13 @@ public abstract class MapRDBGroupScan extends AbstractGroupScan {
     this.formatPlugin = that.formatPlugin;
     this.formatPluginConfig = that.formatPluginConfig;
     this.storagePlugin = that.storagePlugin;
-    this.regionsToScan = that.regionsToScan;
     this.filterPushedDown = that.filterPushedDown;
+    this.costFactor = that.costFactor;
+    /* this is the only place we access the field `doNotAccessRegionsToScan` directly
+     * because we do not want the sub-scan spec for JSON tables to be calculated
+     * during the copy-constructor
+     */
+    this.doNotAccessRegionsToScan = that.doNotAccessRegionsToScan;
   }
 
   public MapRDBGroupScan(AbstractStoragePlugin storagePlugin,
@@ -102,8 +113,8 @@ public abstract class MapRDBGroupScan extends AbstractGroupScan {
       endpointMap.put(ep.getAddress(), ep);
     }
 
-    Map<DrillbitEndpoint, EndpointAffinity> affinityMap = new HashMap<DrillbitEndpoint, EndpointAffinity>();
-    for (String serverName : regionsToScan.values()) {
+    final Map<DrillbitEndpoint, EndpointAffinity> affinityMap = new HashMap<DrillbitEndpoint, EndpointAffinity>();
+    for (String serverName : getRegionsToScan().values()) {
       DrillbitEndpoint ep = endpointMap.get(serverName);
       if (ep != null) {
         EndpointAffinity affinity = affinityMap.get(ep);
@@ -127,6 +138,7 @@ public abstract class MapRDBGroupScan extends AbstractGroupScan {
     watch.reset();
     watch.start();
 
+    final NavigableMap<TabletFragmentInfo, String> regionsToScan = getRegionsToScan();
     final int numSlots = incomingEndpoints.size();
     Preconditions.checkArgument(numSlots <= regionsToScan.size(),
         String.format("Incoming endpoints %d is greater than number of scan regions %d", numSlots, regionsToScan.size()));
@@ -161,7 +173,7 @@ public abstract class MapRDBGroupScan extends AbstractGroupScan {
       hostIndexQueue.add(i);
     }
 
-    Set<Entry<TabletFragmentInfo, String>> regionsToAssignSet = Sets.newHashSet(regionsToScan.entrySet());
+    Set<Entry<TabletFragmentInfo, String>> regionsToAssignSet = Sets.newLinkedHashSet(regionsToScan.entrySet());
 
     /*
      * First, we assign regions which are hosted on region servers running on drillbit endpoints
@@ -175,7 +187,8 @@ public abstract class MapRDBGroupScan extends AbstractGroupScan {
       if (endpointIndexlist != null) {
         Integer slotIndex = endpointIndexlist.poll();
         List<MapRDBSubScanSpec> endpointSlotScanList = endpointFragmentMapping.get(slotIndex);
-        endpointSlotScanList.add(getSubScanSpec(regionEntry.getKey()));
+        MapRDBSubScanSpec subScanSpec = getSubScanSpec(regionEntry.getKey());
+        endpointSlotScanList.add(subScanSpec);
         // add to the tail of the slot list, to add more later in round robin fashion
         endpointIndexlist.offer(slotIndex);
         // this region has been assigned
@@ -202,7 +215,8 @@ public abstract class MapRDBGroupScan extends AbstractGroupScan {
     if (regionsToAssignSet.size() > 0) {
       for (Entry<TabletFragmentInfo, String> regionEntry : regionsToAssignSet) {
         List<MapRDBSubScanSpec> smallestList = minHeap.poll();
-        smallestList.add(getSubScanSpec(regionEntry.getKey()));
+        MapRDBSubScanSpec subScanSpec = getSubScanSpec(regionEntry.getKey());
+        smallestList.add(subScanSpec);
         if (smallestList.size() < maxPerEndpointSlot) {
           minHeap.offer(smallestList);
         }
@@ -224,6 +238,10 @@ public abstract class MapRDBGroupScan extends AbstractGroupScan {
       }
     }
 
+    for (Entry<Integer, List<MapRDBSubScanSpec>> endpoint : endpointFragmentMapping.entrySet()) {
+      Collections.sort(endpoint.getValue());
+    }
+
     /* no slot should be empty at this point */
     assert (minHeap.peek() == null || minHeap.peek().size() > 0) : String.format(
         "Unable to assign tasks to some endpoints.\nEndpoints: {}.\nAssignment Map: {}.",
@@ -235,7 +253,7 @@ public abstract class MapRDBGroupScan extends AbstractGroupScan {
 
   @Override
   public int getMaxParallelizationWidth() {
-    return regionsToScan.size();
+    return getRegionsToScan().size();
   }
 
   @JsonIgnore
@@ -273,11 +291,48 @@ public abstract class MapRDBGroupScan extends AbstractGroupScan {
     this.filterPushedDown = true;
   }
 
+  public String getIndexHint() { return this.formatPluginConfig.getIndex(); }
+
   @JsonIgnore
+  @Override
   public boolean isFilterPushedDown() {
     return filterPushedDown;
   }
 
   protected abstract MapRDBSubScanSpec getSubScanSpec(TabletFragmentInfo key);
 
+  public void setCostFactor(double sel) {
+    this.costFactor = sel;
+  }
+
+  @Override
+  public IndexCollection getSecondaryIndexCollection(RelNode scanRel) {
+    //XXX to implement for complete secondary index framework
+    return null;
+  }
+
+  @JsonIgnore
+  public abstract String getTableName();
+
+  @JsonIgnore
+  public int getRowKeyOrdinal() {
+    return 0;
+  }
+
+  protected NavigableMap<TabletFragmentInfo, String> getRegionsToScan() {
+    return doNotAccessRegionsToScan;
+  }
+
+  protected void resetRegionsToScan() {
+    this.doNotAccessRegionsToScan = null;
+  }
+
+  protected void setRegionsToScan(NavigableMap<TabletFragmentInfo, String> regionsToScan) {
+    this.doNotAccessRegionsToScan = regionsToScan;
+  }
+
+  @Override
+  public PluginCost getPluginCostModel() {
+    return formatPlugin.getPluginCostModel();
+  }
 }
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 c233a6b..cf49714 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
@@ -137,8 +137,7 @@ public abstract class MapRDBPushFilterIntoScan extends StoragePluginOptimizerRul
       return; //no filter pushdown ==> No transformation.
     }
 
-    // clone the groupScan with the newScanSpec.
-    final JsonTableGroupScan newGroupsScan = groupScan.clone(newScanSpec);
+    final JsonTableGroupScan newGroupsScan = (JsonTableGroupScan) groupScan.clone(newScanSpec);
     newGroupsScan.setFilterPushedDown(true);
 
     final ScanPrel newScanPrel = new ScanPrel(scan, filter.getTraitSet(), newGroupsScan, scan.getRowType());
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
new file mode 100644
index 0000000..79eec12
--- /dev/null
+++ b/contrib/format-maprdb/src/main/java/org/apache/drill/exec/store/mapr/db/MapRDBPushLimitIntoScan.java
@@ -0,0 +1,157 @@
+/*
+ * 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.store.mapr.db;
+
+import org.apache.calcite.plan.RelOptRuleCall;
+import org.apache.calcite.plan.RelOptRuleOperand;
+import org.apache.calcite.rel.RelNode;
+import org.apache.calcite.rex.RexLiteral;
+import org.apache.drill.exec.physical.base.GroupScan;
+import org.apache.drill.exec.planner.common.DrillRelOptUtil;
+import org.apache.drill.exec.planner.logical.RelOptHelper;
+import org.apache.drill.exec.planner.physical.LimitPrel;
+import org.apache.drill.exec.planner.physical.ProjectPrel;
+import org.apache.drill.exec.planner.physical.ScanPrel;
+import org.apache.drill.exec.store.StoragePluginOptimizerRule;
+import org.apache.drill.exec.store.hbase.HBaseScanSpec;
+import org.apache.drill.exec.store.mapr.db.binary.BinaryTableGroupScan;
+import org.apache.drill.exec.store.mapr.db.json.JsonTableGroupScan;
+
+public abstract class MapRDBPushLimitIntoScan extends StoragePluginOptimizerRule {
+  static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(MapRDBPushLimitIntoScan.class);
+
+  private MapRDBPushLimitIntoScan(RelOptRuleOperand operand, String description) {
+    super(operand, description);
+  }
+
+  public static final StoragePluginOptimizerRule LIMIT_ON_SCAN =
+      new MapRDBPushLimitIntoScan(RelOptHelper.some(LimitPrel.class, RelOptHelper.any(ScanPrel.class)),
+          "MapRDBPushLimitIntoScan:Limit_On_Scan") {
+
+    @Override
+    public void onMatch(RelOptRuleCall call) {
+      final ScanPrel scan = call.rel(1);
+      final LimitPrel limit = call.rel(0);
+      doPushLimitIntoGroupScan(call, limit, null, scan, scan.getGroupScan());
+    }
+
+    @Override
+    public boolean matches(RelOptRuleCall call) {
+      final ScanPrel scan = call.rel(1);
+      final LimitPrel limit = call.rel(0);
+      // pushdown only apply limit but not offset,
+      // so if getFetch() return null no need to run this rule.
+      if (scan.getGroupScan().supportsLimitPushdown()
+            && !limit.isPushDown() && limit.getFetch() != null) {
+        if ((scan.getGroupScan() instanceof JsonTableGroupScan
+              && ((JsonTableGroupScan) scan.getGroupScan()).isIndexScan()) ) {
+            //|| (scan.getGroupScan() instanceof RestrictedJsonTableGroupScan)) {
+          return true;
+        }
+      }
+      return false;
+    }
+  };
+
+  public static final StoragePluginOptimizerRule LIMIT_ON_PROJECT =
+      new MapRDBPushLimitIntoScan(RelOptHelper.some(LimitPrel.class,
+          RelOptHelper.any(ProjectPrel.class)), "MapRDBPushLimitIntoScan:Limit_On_Project") {
+
+    @Override
+    public void onMatch(RelOptRuleCall call) {
+      final ProjectPrel project = call.rel(1);
+      final LimitPrel limit = call.rel(0);
+      RelNode child = project.getInput();
+      final RelNode limitUnderProject = new LimitPrel(child.getCluster(), child.getTraitSet(),
+          child, limit.getOffset(), limit.getFetch());
+      final RelNode newProject = new ProjectPrel(project.getCluster(), project.getTraitSet(),
+          limitUnderProject, project.getProjects(), project.getRowType());
+      if (DrillRelOptUtil.isProjectFlatten(project)) {
+        //Preserve limit above the project since Flatten can produce more rows. Also mark it so we do not fire the rule again.
+        child = newProject;
+        final RelNode limitAboveProject = new LimitPrel(child.getCluster(), child.getTraitSet(),
+            child, limit.getOffset(), limit.getFetch(), true);
+        call.transformTo(limitAboveProject);
+      } else {
+        call.transformTo(newProject);
+      }
+    }
+
+    @Override
+    public boolean matches(RelOptRuleCall call) {
+      LimitPrel limitPrel = call.rel(0);
+      ProjectPrel projectPrel = call.rel(1);
+      // pushdown only apply limit but not offset,
+      // so if getFetch() return null no need to run this rule.
+      // Do not push across Project containing CONVERT_FROMJSON for limit 0 queries. For limit 0 queries, this would
+      // mess up the schema since Convert_FromJson() is different from other regular functions in that it only knows
+      // the output schema after evaluation is performed. When input has 0 row, Drill essentially does not have a way
+      // to know the output type.
+      if (!limitPrel.isPushDown() && (limitPrel.getFetch() != null)
+          && (!DrillRelOptUtil.isLimit0(limitPrel.getFetch())
+          || !DrillRelOptUtil.isProjectOutputSchemaUnknown(projectPrel))) {
+        return true;
+      }
+      return false;
+    }
+  };
+
+  protected void doPushLimitIntoGroupScan(RelOptRuleCall call,
+      LimitPrel limit, final ProjectPrel project, ScanPrel scan, GroupScan groupScan) {
+    try {
+      final GroupScan newGroupScan = getGroupScanWithLimit(groupScan, limit);
+      if (newGroupScan == null) {
+        return;
+      }
+      final ScanPrel newScan = new ScanPrel(scan.getCluster(), scan.getTraitSet(), newGroupScan,
+          scan.getRowType());
+      final RelNode newChild;
+      if (project != null) {
+        final ProjectPrel newProject = new ProjectPrel(project.getCluster(), project.getTraitSet(),
+            newScan, project.getProjects(), project.getRowType());
+        newChild = newProject;
+      } else {
+        newChild = newScan;
+      }
+      call.transformTo(newChild);
+      logger.debug("pushLimitIntoGroupScan: Converted to a new ScanPrel " + newScan.getGroupScan());
+    } catch (Exception e) {
+      logger.warn("pushLimitIntoGroupScan: Exception while trying limit pushdown!", e);
+    }
+  }
+
+  private GroupScan getGroupScanWithLimit(GroupScan groupScan, LimitPrel limit) {
+    final int offset = limit.getOffset() != null ? Math.max(0, RexLiteral.intValue(limit.getOffset())) : 0;
+    final int fetch = Math.max(0, RexLiteral.intValue(limit.getFetch()));
+    // Scan Limit uses conservative approach:  use offset 0 and fetch = parent limit offset + parent limit fetch.
+    if (groupScan instanceof JsonTableGroupScan) {
+      JsonTableGroupScan jsonTableGroupScan = (JsonTableGroupScan) groupScan;
+      return (jsonTableGroupScan.clone(jsonTableGroupScan.getScanSpec()).applyLimit(offset + fetch));
+    } else if (groupScan instanceof BinaryTableGroupScan) {
+      BinaryTableGroupScan binaryTableGroupScan = (BinaryTableGroupScan) groupScan;
+      final HBaseScanSpec oldScanSpec = binaryTableGroupScan.getHBaseScanSpec();
+      final HBaseScanSpec newScanSpec = new HBaseScanSpec(oldScanSpec.getTableName(), oldScanSpec.getStartRow(),
+          oldScanSpec.getStopRow(), oldScanSpec.getFilter());
+      return new BinaryTableGroupScan(binaryTableGroupScan.getUserName(), binaryTableGroupScan.getStoragePlugin(),
+          binaryTableGroupScan.getFormatPlugin(), newScanSpec, binaryTableGroupScan.getColumns(),
+          binaryTableGroupScan.getTableStats()).applyLimit(offset + fetch);
+    }
+    return null;
+  }
+}
+
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
new file mode 100644
index 0000000..8d4f549
--- /dev/null
+++ b/contrib/format-maprdb/src/main/java/org/apache/drill/exec/store/mapr/db/MapRDBPushProjectIntoScan.java
@@ -0,0 +1,141 @@
+/*
+ * 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.store.mapr.db;
+
+import com.google.common.collect.Lists;
+import org.apache.calcite.plan.RelOptRuleCall;
+import org.apache.calcite.plan.RelOptRuleOperand;
+import org.apache.calcite.plan.RelTrait;
+import org.apache.calcite.plan.RelTraitSet;
+import org.apache.calcite.rel.RelCollation;
+import org.apache.calcite.rel.rules.ProjectRemoveRule;
+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.logical.RelOptHelper;
+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.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 java.util.List;
+
+public abstract class MapRDBPushProjectIntoScan extends StoragePluginOptimizerRule {
+  static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(MapRDBPushProjectIntoScan.class);
+
+  private MapRDBPushProjectIntoScan(RelOptRuleOperand operand, String description) {
+    super(operand, description);
+  }
+
+  public static final StoragePluginOptimizerRule PROJECT_ON_SCAN = new MapRDBPushProjectIntoScan(
+      RelOptHelper.some(ProjectPrel.class, RelOptHelper.any(ScanPrel.class)), "MapRDBPushProjIntoScan:Proj_On_Scan") {
+    @Override
+    public void onMatch(RelOptRuleCall call) {
+      final ScanPrel scan = (ScanPrel) call.rel(1);
+      final ProjectPrel project = (ProjectPrel) call.rel(0);
+      if (!(scan.getGroupScan() instanceof MapRDBGroupScan)) {
+        return;
+      }
+      doPushProjectIntoGroupScan(call, project, scan, (MapRDBGroupScan) scan.getGroupScan());
+      if (scan.getGroupScan() instanceof BinaryTableGroupScan) {
+        BinaryTableGroupScan groupScan = (BinaryTableGroupScan) scan.getGroupScan();
+
+      } else {
+        assert (scan.getGroupScan() instanceof JsonTableGroupScan);
+        JsonTableGroupScan groupScan = (JsonTableGroupScan) scan.getGroupScan();
+
+        doPushProjectIntoGroupScan(call, project, scan, groupScan);
+      }
+    }
+
+    @Override
+    public boolean matches(RelOptRuleCall call) {
+      final ScanPrel scan = (ScanPrel) call.rel(1);
+      if (scan.getGroupScan() instanceof BinaryTableGroupScan ||
+          scan.getGroupScan() instanceof JsonTableGroupScan) {
+        return super.matches(call);
+      }
+      return false;
+    }
+  };
+
+  protected void doPushProjectIntoGroupScan(RelOptRuleCall call,
+      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)) {
+        return;
+      }
+      RelTraitSet newTraits = call.getPlanner().emptyTraitSet();
+      // Clear out collation trait
+      for (RelTrait trait : scan.getTraitSet()) {
+        if (!(trait instanceof RelCollation)) {
+          newTraits.plus(trait);
+        }
+      }
+      final ScanPrel newScan = new ScanPrel(scan.getCluster(), newTraits.plus(Prel.DRILL_PHYSICAL),
+          groupScan.clone(columnInfo.columns),
+          columnInfo.createNewRowType(project.getInput().getCluster().getTypeFactory()));
+
+      List<RexNode> newProjects = Lists.newArrayList();
+      for (RexNode n : project.getChildExps()) {
+        newProjects.add(n.accept(columnInfo.getInputRewriter()));
+      }
+
+      final ProjectPrel newProj =
+          new ProjectPrel(project.getCluster(),
+              project.getTraitSet().plus(Prel.DRILL_PHYSICAL),
+              newScan,
+              newProjects,
+              project.getRowType());
+
+      if (ProjectRemoveRule.isTrivial(newProj) &&
+          // the old project did not involve any column renaming
+          sameRowTypeProjectionsFields(project.getRowType(), newScan.getRowType())) {
+        call.transformTo(newScan);
+      } else {
+        call.transformTo(newProj);
+      }
+    } catch (Exception e) {
+      throw new DrillRuntimeException(e);
+    }
+  }
+
+  private boolean sameRowTypeProjectionsFields(RelDataType oldRowType, RelDataType newRowType) {
+    for (RelDataTypeField oldField : oldRowType.getFieldList()) {
+      String oldProjName = oldField.getName();
+      boolean match = false;
+      for (RelDataTypeField newField : newRowType.getFieldList()) {
+        if (oldProjName.equals(newField.getName())) {
+          match = true;
+          break;
+        }
+      }
+      if (!match) {
+        return false;
+      }
+    }
+    return true;
+  }
+}
diff --git a/contrib/format-maprdb/src/main/java/org/apache/drill/exec/store/mapr/db/MapRDBRestrictedScanBatchCreator.java b/contrib/format-maprdb/src/main/java/org/apache/drill/exec/store/mapr/db/MapRDBRestrictedScanBatchCreator.java
new file mode 100644
index 0000000..89ce95d
--- /dev/null
+++ b/contrib/format-maprdb/src/main/java/org/apache/drill/exec/store/mapr/db/MapRDBRestrictedScanBatchCreator.java
@@ -0,0 +1,50 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.drill.exec.store.mapr.db;
+
+import com.google.common.base.Preconditions;
+import com.google.common.collect.Lists;
+import org.apache.drill.common.exceptions.ExecutionSetupException;
+import org.apache.drill.exec.ops.ExecutorFragmentContext;
+import org.apache.drill.exec.physical.impl.BatchCreator;
+import org.apache.drill.exec.physical.impl.ScanBatch;
+import org.apache.drill.exec.record.RecordBatch;
+import org.apache.drill.exec.store.RecordReader;
+import org.apache.drill.exec.store.mapr.db.json.RestrictedJsonRecordReader;
+
+import java.util.List;
+
+public class MapRDBRestrictedScanBatchCreator implements BatchCreator<RestrictedMapRDBSubScan> {
+  static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(MapRDBRestrictedScanBatchCreator.class);
+
+  @Override
+  public ScanBatch getBatch(ExecutorFragmentContext context, RestrictedMapRDBSubScan subScan, List<RecordBatch> children) throws ExecutionSetupException {
+    Preconditions.checkArgument(children.isEmpty());
+    List<RecordReader> readers = Lists.newArrayList();
+    for(MapRDBSubScanSpec scanSpec : subScan.getRegionScanSpecList()){
+      try {
+        readers.add(new RestrictedJsonRecordReader((RestrictedMapRDBSubScanSpec)scanSpec, subScan.getFormatPlugin(), subScan.getColumns(),
+            context, subScan.getMaxRecordsToRead()));
+      } catch (Exception e1) {
+        throw new ExecutionSetupException(e1);
+      }
+    }
+    return new ScanBatch(subScan, context, readers, true);
+  }
+
+}
diff --git a/contrib/format-maprdb/src/main/java/org/apache/drill/exec/store/mapr/db/MapRDBScanBatchCreator.java b/contrib/format-maprdb/src/main/java/org/apache/drill/exec/store/mapr/db/MapRDBScanBatchCreator.java
index 7e4e244..de2817e 100644
--- a/contrib/format-maprdb/src/main/java/org/apache/drill/exec/store/mapr/db/MapRDBScanBatchCreator.java
+++ b/contrib/format-maprdb/src/main/java/org/apache/drill/exec/store/mapr/db/MapRDBScanBatchCreator.java
@@ -33,7 +33,9 @@ import org.apache.drill.exec.store.mapr.db.json.MaprDBJsonRecordReader;
 
 import org.apache.drill.shaded.guava.com.google.common.base.Preconditions;
 
-public class MapRDBScanBatchCreator implements BatchCreator<MapRDBSubScan> {
+public class MapRDBScanBatchCreator implements BatchCreator<MapRDBSubScan>{
+  private static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(MapRDBScanBatchCreator.class);
+
   @Override
   public ScanBatch getBatch(ExecutorFragmentContext context, MapRDBSubScan subScan, List<RecordBatch> children) throws ExecutionSetupException {
     Preconditions.checkArgument(children.isEmpty());
@@ -46,7 +48,7 @@ public class MapRDBScanBatchCreator implements BatchCreator<MapRDBSubScan> {
               getHBaseSubScanSpec(scanSpec),
               subScan.getColumns()));
         } else {
-          readers.add(new MaprDBJsonRecordReader(scanSpec, subScan.getFormatPluginConfig(), subScan.getColumns(), context));
+          readers.add(new MaprDBJsonRecordReader(scanSpec, subScan.getFormatPlugin(), subScan.getColumns(), context, subScan.getMaxRecordsToRead()));
         }
       } catch (Exception e) {
         throw new ExecutionSetupException(e);
diff --git a/contrib/format-maprdb/src/main/java/org/apache/drill/exec/store/mapr/db/MapRDBSubScan.java b/contrib/format-maprdb/src/main/java/org/apache/drill/exec/store/mapr/db/MapRDBSubScan.java
index 5766b9b..159b850 100644
--- a/contrib/format-maprdb/src/main/java/org/apache/drill/exec/store/mapr/db/MapRDBSubScan.java
+++ b/contrib/format-maprdb/src/main/java/org/apache/drill/exec/store/mapr/db/MapRDBSubScan.java
@@ -24,7 +24,7 @@ import com.fasterxml.jackson.annotation.JsonIgnore;
 import org.apache.drill.common.exceptions.ExecutionSetupException;
 import org.apache.drill.common.expression.SchemaPath;
 import org.apache.drill.common.logical.StoragePluginConfig;
-import org.apache.drill.exec.physical.base.AbstractBase;
+import org.apache.drill.exec.physical.base.AbstractDbSubScan;
 import org.apache.drill.exec.physical.base.PhysicalOperator;
 import org.apache.drill.exec.physical.base.PhysicalVisitor;
 import org.apache.drill.exec.physical.base.SubScan;
@@ -40,12 +40,14 @@ import org.apache.drill.shaded.guava.com.google.common.collect.ImmutableSet;
 
 // Class containing information for reading a single HBase region
 @JsonTypeName("maprdb-sub-scan")
-public class MapRDBSubScan extends AbstractBase implements SubScan {
-  private static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(MapRDBSubScan.class);
+
+public class MapRDBSubScan extends AbstractDbSubScan {
+  static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(MapRDBSubScan.class);
 
   private final MapRDBFormatPlugin formatPlugin;
   private final List<MapRDBSubScanSpec> regionScanSpecList;
   private final List<SchemaPath> columns;
+  private final int maxRecordsToRead;
   private final String tableType;
 
   @JsonCreator
@@ -55,20 +57,28 @@ public class MapRDBSubScan extends AbstractBase implements SubScan {
                        @JsonProperty("storageConfig") StoragePluginConfig storageConfig,
                        @JsonProperty("regionScanSpecList") List<MapRDBSubScanSpec> regionScanSpecList,
                        @JsonProperty("columns") List<SchemaPath> columns,
+                       @JsonProperty("maxRecordsToRead") int maxRecordsToRead,
                        @JsonProperty("tableType") String tableType) throws ExecutionSetupException {
     this(userName,
         (MapRDBFormatPlugin) engineRegistry.getFormatPlugin(storageConfig, formatPluginConfig),
         regionScanSpecList,
         columns,
+        maxRecordsToRead,
         tableType);
   }
 
   public MapRDBSubScan(String userName, MapRDBFormatPlugin formatPlugin,
       List<MapRDBSubScanSpec> maprSubScanSpecs, List<SchemaPath> columns, String tableType) {
+    this(userName, formatPlugin, maprSubScanSpecs, columns, -1, tableType);
+  }
+
+  public MapRDBSubScan(String userName, MapRDBFormatPlugin formatPlugin,
+                       List<MapRDBSubScanSpec> maprSubScanSpecs, List<SchemaPath> columns, int maxRecordsToRead, String tableType) {
     super(userName);
     this.formatPlugin = formatPlugin;
     this.regionScanSpecList = maprSubScanSpecs;
     this.columns = columns;
+    this.maxRecordsToRead = maxRecordsToRead;
     this.tableType = tableType;
   }
 
@@ -93,6 +103,11 @@ public class MapRDBSubScan extends AbstractBase implements SubScan {
     return columns;
   }
 
+  @JsonProperty("maxRecordsToRead")
+  public int getMaxRecordsToRead() {
+    return maxRecordsToRead;
+  }
+
   @JsonProperty("tableType")
   public String getTableType() {
     return tableType;
@@ -121,7 +136,7 @@ public class MapRDBSubScan extends AbstractBase implements SubScan {
 
   @Override
   public int getOperatorType() {
-    return  CoreOperatorType.MAPRDB_SUB_SCAN_VALUE;
+    return CoreOperatorType.MAPRDB_SUB_SCAN_VALUE;
   }
 
   @JsonIgnore
diff --git a/contrib/format-maprdb/src/main/java/org/apache/drill/exec/store/mapr/db/MapRDBSubScanSpec.java b/contrib/format-maprdb/src/main/java/org/apache/drill/exec/store/mapr/db/MapRDBSubScanSpec.java
index 40ab4f4..e24438e 100644
--- a/contrib/format-maprdb/src/main/java/org/apache/drill/exec/store/mapr/db/MapRDBSubScanSpec.java
+++ b/contrib/format-maprdb/src/main/java/org/apache/drill/exec/store/mapr/db/MapRDBSubScanSpec.java
@@ -19,32 +19,39 @@ package org.apache.drill.exec.store.mapr.db;
 
 import com.fasterxml.jackson.annotation.JsonCreator;
 import com.fasterxml.jackson.annotation.JsonProperty;
+import com.mapr.db.index.IndexDesc;
 import com.mapr.fs.jni.MapRConstants;
 import com.mapr.org.apache.hadoop.hbase.util.Bytes;
 
-public class MapRDBSubScanSpec {
+public class MapRDBSubScanSpec implements Comparable<MapRDBSubScanSpec>{
 
   protected String tableName;
+  protected IndexDesc indexDesc;
   protected String regionServer;
   protected byte[] startRow;
   protected byte[] stopRow;
   protected byte[] serializedFilter;
+  protected String userName;
 
   @JsonCreator
   public MapRDBSubScanSpec(@JsonProperty("tableName") String tableName,
+                           @JsonProperty("indexDesc") IndexDesc indexDesc,
                            @JsonProperty("regionServer") String regionServer,
                            @JsonProperty("startRow") byte[] startRow,
                            @JsonProperty("stopRow") byte[] stopRow,
                            @JsonProperty("serializedFilter") byte[] serializedFilter,
-                           @JsonProperty("filterString") String filterString) {
+                           @JsonProperty("filterString") String filterString,
+                           @JsonProperty("username") String userName) {
     if (serializedFilter != null && filterString != null) {
       throw new IllegalArgumentException("The parameters 'serializedFilter' or 'filterString' cannot be specified at the same time.");
     }
     this.tableName = tableName;
+    this.indexDesc = indexDesc;
     this.regionServer = regionServer;
     this.startRow = startRow;
     this.stopRow = stopRow;
     this.serializedFilter = serializedFilter;
+    this.userName = userName;
   }
 
   /* package */ MapRDBSubScanSpec() {
@@ -55,6 +62,10 @@ public class MapRDBSubScanSpec {
     return tableName;
   }
 
+  public IndexDesc getIndexDesc() {
+    return indexDesc;
+  }
+
   public MapRDBSubScanSpec setTableName(String tableName) {
     this.tableName = tableName;
     return this;
@@ -102,13 +113,50 @@ public class MapRDBSubScanSpec {
     return this;
   }
 
+  public String getUserName() {
+    return userName;
+  }
+
+  public void setUserName(String userName) {
+    this.userName = userName;
+  }
+
   @Override
   public String toString() {
     return "MapRDBSubScanSpec [tableName=" + tableName
         + ", startRow=" + (startRow == null ? null : Bytes.toStringBinary(startRow))
         + ", stopRow=" + (stopRow == null ? null : Bytes.toStringBinary(stopRow))
         + ", filter=" + (getSerializedFilter() == null ? null : Bytes.toBase64(getSerializedFilter()))
-        + ", regionServer=" + regionServer + "]";
+        + ", regionServer=" + regionServer
+        + ", userName=" + userName + "]";
   }
 
+  @Override
+  /*
+   * The semantics of the compareTo function is same as that of TabletInfoImpl.
+   * It compares the startRows of the two subScanSpec and returns the status
+   * if one is greater than the other. If the two startRows are same then it
+   * compares the stopRows.
+   */
+  public int compareTo(MapRDBSubScanSpec o) {
+    if (o == null) {
+      return 1;
+    } else {
+      int result = Bytes.compareTo(this.getStartRow(), o.getStartRow());
+      if (result != 0) {
+        return result;
+      } else {
+        result = Bytes.compareTo(this.getStopRow(), o.getStopRow());
+        if (result != 0) {
+          if (this.getStartRow().length != 0 && this.getStopRow().length == 0) {
+            return 1;
+          } else {
+            return o.getStartRow().length != 0 && o.getStopRow().length == 0 ? -1 : result;
+          }
+        } else {
+          return 0;
+        }
+      }
+    }
+  }
 }
diff --git a/contrib/format-maprdb/src/main/java/org/apache/drill/exec/store/mapr/db/MapRDBTableCache.java b/contrib/format-maprdb/src/main/java/org/apache/drill/exec/store/mapr/db/MapRDBTableCache.java
new file mode 100644
index 0000000..f35a4c4
--- /dev/null
+++ b/contrib/format-maprdb/src/main/java/org/apache/drill/exec/store/mapr/db/MapRDBTableCache.java
@@ -0,0 +1,232 @@
+/*
+ * 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.store.mapr.db;
+
+import com.google.common.cache.CacheBuilder;
+import com.google.common.cache.CacheLoader;
+import com.google.common.cache.LoadingCache;
+import com.google.common.cache.RemovalListener;
+import com.google.common.cache.RemovalNotification;
+import com.mapr.db.Table;
+import com.mapr.db.impl.MapRDBImpl;
+import com.mapr.db.index.IndexDesc;
+import org.apache.drill.common.config.DrillConfig;
+import org.apache.drill.common.exceptions.DrillRuntimeException;
+import org.apache.drill.exec.util.ImpersonationUtil;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.security.UserGroupInformation;
+
+import java.io.IOException;
+import java.security.PrivilegedExceptionAction;
+import java.util.concurrent.TimeUnit;
+
+public class MapRDBTableCache {
+  static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(MapRDBFormatPlugin.class);
+
+  public static final String FORMAT_MAPRDB_JSON_TABLE_CACHE_ENABLED = "format-maprdb.json.tableCache.enabled";
+
+  public static final String FORMAT_MAPRDB_JSON_TABLE_CACHE_SIZE = "format-maprdb.json.tableCache.size";
+
+  public static final String FORMAT_MAPRDB_JSON_TABLE_CACHE_TIMEOUT = "format-maprdb.json.tableCache.expireTimeInMinutes";
+
+  private static final int MIN_TABLE_CACHE_SIZE = 1;
+
+  private static final int MIN_TABLE_CACHE_ENTRY_TIMEOUT = 10;
+
+  LoadingCache<MapRDBTableCache.Key, Table> tableCache;
+
+  private final boolean tableCachingEnabled;
+
+  public MapRDBTableCache(DrillConfig config) {
+    tableCachingEnabled = config.getBoolean(FORMAT_MAPRDB_JSON_TABLE_CACHE_ENABLED);
+    if (tableCachingEnabled) {
+      final int tableCacheSize = Math.max((int) (config.getDouble(FORMAT_MAPRDB_JSON_TABLE_CACHE_SIZE)), MIN_TABLE_CACHE_SIZE);
+      final int tableCacheExpiryTime = Math.max((int) (config.getDouble(FORMAT_MAPRDB_JSON_TABLE_CACHE_TIMEOUT)), MIN_TABLE_CACHE_ENTRY_TIMEOUT);
+
+      RemovalListener<MapRDBTableCache.Key, Table> removalListener = new RemovalListener<MapRDBTableCache.Key, Table>() {
+        public void onRemoval(RemovalNotification<MapRDBTableCache.Key, Table> removal) {
+          Table table = removal.getValue();
+          MapRDBTableCache.Key key = removal.getKey();
+          logger.debug("time {} closing the tablePath {} tableHandle {} index {} userName {}",
+              System.nanoTime(),
+              key.path == null ? "null" : key.path, table == null ? "null" : table,
+              key.indexDesc == null ? "null" : key.indexDesc.getIndexName(),
+              key.ugi.getUserName() == null ? "null" : key.ugi.getUserName());
+          table.close(); // close the table
+        }
+      };
+
+      // Common table cache for primary and index tables. Key is Pair<tablePath, indexDesc>
+      // For primary table, indexDesc is null.
+      tableCache = CacheBuilder.newBuilder().
+          expireAfterAccess(tableCacheExpiryTime, TimeUnit.MINUTES).
+          maximumSize(tableCacheSize).
+          removalListener(removalListener).build(new CacheLoader<MapRDBTableCache.Key, Table>() {
+
+        @Override
+        public Table load(final MapRDBTableCache.Key key) throws Exception {
+          // getTable is already calling tableCache.get in correct user UGI context, so should be fine here.
+          // key.Left is Path. key.Right is indexDesc.
+          Table table = (key.indexDesc == null ? MapRDBImpl.getTable(key.path) : MapRDBImpl.getIndexTable(key.indexDesc));
+          logger.debug("time {} opened the table for tablePath {} tableHandle {} index {} userName {}",
+              System.nanoTime(),
+              key.path == null ? "null" : key.path,
+              table == null ? "null" : table,
+              key.indexDesc == null ? "null" : key.indexDesc.getIndexName(),
+              key.ugi.getUserName() == null ? "null" : key.ugi.getUserName());
+          return table;
+        }
+      });
+
+      logger.debug("table cache created with size {} and expiryTimeInMin {} ", tableCacheSize, tableCacheExpiryTime);
+    }
+  }
+
+
+  /**
+   * getTable given primary table path and indexDesc.
+   * returns Table for corresponding index table if indexDesc is not null.
+   * returns Table for primary table if indexDesc is null.
+   *
+   * @param tablePath primary table path
+   * @param indexDesc index table descriptor
+   */
+  public Table getTable(final Path tablePath, final IndexDesc indexDesc, final String userName) throws DrillRuntimeException {
+
+    final Table dbTableHandle;
+    final UserGroupInformation proxyUserUgi = ImpersonationUtil.createProxyUgi(userName);
+
+    try {
+      dbTableHandle = proxyUserUgi.doAs(new PrivilegedExceptionAction<Table>() {
+        public Table run() throws Exception {
+
+          if (logger.isTraceEnabled()) {
+            logger.trace("Getting MaprDB Table handle for proxy user: " + UserGroupInformation.getCurrentUser());
+          }
+
+          if (tableCachingEnabled) {
+            Table table = tableCache.get(new MapRDBTableCache.Key(tablePath, indexDesc));
+            logger.trace("time {} get the tablePath {} tableHandle {} index {} userName {} currentUser {}",
+                System.nanoTime(), tablePath == null ? "null" : tablePath,
+                table == null ? "null" : table,
+                indexDesc == null ? "null" : indexDesc.getIndexName(),
+                userName == null ? "null" : userName,
+                UserGroupInformation.getCurrentUser() == null ? "null" : UserGroupInformation.getCurrentUser());
+            return table;
+          } else {
+            return indexDesc == null ? MapRDBImpl.getTable(tablePath) : MapRDBImpl.getIndexTable(indexDesc);
+          }
+        }
+      });
+    } catch (Exception e) {
+      throw new DrillRuntimeException("Error getting table: " + tablePath.toString() + (indexDesc == null ? "" : (", " +
+          "IndexDesc: " + indexDesc.toString())), e);
+    }
+
+    return dbTableHandle;
+  }
+
+  /**
+   * getTable given primary table name.
+   * returns Table for primary table with given name.
+   *
+   * @param tableName primary table path
+   */
+  public Table getTable(String tableName, String userName) {
+    return getTable(new Path(tableName), null, userName);
+  }
+
+  /**
+   * getTable given primary table path.
+   * returns Table for primary table with given path.
+   *
+   * @param tablePath primary table path
+   */
+  public Table getTable(Path tablePath, String userName) {
+    return getTable(tablePath, null, userName);
+  }
+
+  /**
+   * getTable given primary table name and indexDesc.
+   * returns Table for corresponding index table if indexDesc is not null.
+   * returns Table for primary table if indexDesc is null.
+   *
+   * @param tableName primary table name
+   * @param indexDesc index table Descriptor
+   */
+  public Table getTable(String tableName, IndexDesc indexDesc, String userName) {
+    return getTable(new Path(tableName), indexDesc, userName);
+  }
+
+  /**
+   * closeTable
+   *
+   * @param table table to be closed.
+   */
+  public void closeTable(Table table) {
+    if (!tableCachingEnabled && table != null) {
+      table.close();
+    }
+  }
+
+  /**
+   * Key for {@link MapRDBTableCache} to store table path, {@link IndexDesc} and UGI.
+   */
+  static class Key {
+    final Path path;
+
+    final IndexDesc indexDesc;
+
+    final UserGroupInformation ugi;
+
+    Key(Path path, IndexDesc indexDesc) throws IOException {
+      this.path = path;
+      this.indexDesc = indexDesc;
+      this.ugi = UserGroupInformation.getCurrentUser();
+    }
+
+    public int hashCode() {
+
+      final int IdxDescHashCode = (indexDesc == null) ? 0 : indexDesc.getIndexFid().hashCode();
+      return (path.hashCode() + IdxDescHashCode + ugi.hashCode());
+    }
+
+    static boolean isEqual(Object a, Object b) {
+      return a == b || a != null && a.equals(b);
+    }
+    
+    public boolean equals(Object obj) {
+      if (obj == this) {
+        return true;
+      } else if (obj != null && obj instanceof MapRDBTableCache.Key) {
+        MapRDBTableCache.Key that = (MapRDBTableCache.Key) obj;
+        return isEqual(this.path, that.path)
+            && isEqual(this.indexDesc, that.indexDesc)
+            && isEqual(this.ugi, that.ugi);
+      } else {
+        return false;
+      }
+    }
+
+    public String toString() {
+      return "(Path: " + this.path.toString() +
+          ", UGI: " + this.ugi.toString() +
+          ", IndexDesc: " + (this.indexDesc == null ? "" : this.indexDesc.toString()) + ")";
+    }
+  }
+}
diff --git a/contrib/format-maprdb/src/main/java/org/apache/drill/exec/store/mapr/db/RestrictedMapRDBSubScan.java b/contrib/format-maprdb/src/main/java/org/apache/drill/exec/store/mapr/db/RestrictedMapRDBSubScan.java
new file mode 100644
index 0000000..eedbca5
--- /dev/null
+++ b/contrib/format-maprdb/src/main/java/org/apache/drill/exec/store/mapr/db/RestrictedMapRDBSubScan.java
@@ -0,0 +1,80 @@
+/*
+ * 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.store.mapr.db;
+
+import java.util.ArrayList;
+import java.util.List;
+
+import com.fasterxml.jackson.annotation.JacksonInject;
+import com.fasterxml.jackson.annotation.JsonCreator;
+import com.fasterxml.jackson.annotation.JsonProperty;
+import com.fasterxml.jackson.annotation.JsonTypeName;
+
+import org.apache.drill.common.exceptions.ExecutionSetupException;
+import org.apache.drill.common.expression.SchemaPath;
+import org.apache.drill.common.logical.StoragePluginConfig;
+import org.apache.drill.exec.physical.impl.join.RowKeyJoin;
+import org.apache.drill.exec.store.StoragePluginRegistry;
+
+/**
+ * A RestrictedMapRDBSubScan is intended for skip-scan (as opposed to sequential scan) operations
+ * where the set of rowkeys is obtained from a corresponding RowKeyJoin instance
+*/
+@JsonTypeName("maprdb-restricted-subscan")
+public class RestrictedMapRDBSubScan extends MapRDBSubScan {
+  static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(RestrictedMapRDBSubScan.class);
+
+  @JsonCreator
+  public RestrictedMapRDBSubScan(@JacksonInject StoragePluginRegistry engineRegistry,
+                       @JsonProperty("userName") String userName,
+                       @JsonProperty("formatPluginConfig") MapRDBFormatPluginConfig formatPluginConfig,
+                       @JsonProperty("storageConfig") StoragePluginConfig storageConfig,
+                       @JsonProperty("regionScanSpecList") List<RestrictedMapRDBSubScanSpec> regionScanSpecList,
+                       @JsonProperty("columns") List<SchemaPath> columns,
+                       @JsonProperty("maxRecordsToRead") int maxRecordsToRead,
+                       @JsonProperty("tableType") String tableType) throws ExecutionSetupException {
+    this(userName,
+        (MapRDBFormatPlugin) engineRegistry.getFormatPlugin(storageConfig, formatPluginConfig),
+        regionScanSpecList, columns, maxRecordsToRead, tableType);
+  }
+
+  public RestrictedMapRDBSubScan(String userName, MapRDBFormatPlugin formatPlugin,
+      List<RestrictedMapRDBSubScanSpec> maprDbSubScanSpecs, List<SchemaPath> columns, int maxRecordsToRead, String tableType) {
+    super(userName, formatPlugin, new ArrayList<MapRDBSubScanSpec>(), columns, maxRecordsToRead, tableType);
+
+    for(RestrictedMapRDBSubScanSpec restrictedSpec : maprDbSubScanSpecs) {
+      getRegionScanSpecList().add(restrictedSpec);
+    }
+
+  }
+
+  @Override
+  public void addJoinForRestrictedSubScan(RowKeyJoin rjbatch) {
+    // currently, all subscan specs are sharing the same join batch instance
+    for (MapRDBSubScanSpec s : getRegionScanSpecList()) {
+      assert (s instanceof RestrictedMapRDBSubScanSpec);
+      ((RestrictedMapRDBSubScanSpec)s).setJoinForSubScan(rjbatch);
+    }
+  }
+
+  @Override
+  public boolean isRestrictedSubScan() {
+    return true;
+  }
+
+}
diff --git a/contrib/format-maprdb/src/main/java/org/apache/drill/exec/store/mapr/db/RestrictedMapRDBSubScanSpec.java b/contrib/format-maprdb/src/main/java/org/apache/drill/exec/store/mapr/db/RestrictedMapRDBSubScanSpec.java
new file mode 100644
index 0000000..bd8a32a
--- /dev/null
+++ b/contrib/format-maprdb/src/main/java/org/apache/drill/exec/store/mapr/db/RestrictedMapRDBSubScanSpec.java
@@ -0,0 +1,219 @@
+/*
+ * 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.store.mapr.db;
+
+import com.mapr.db.impl.IdCodec;
+
+import org.apache.commons.lang3.tuple.Pair;
+import org.apache.drill.common.exceptions.DrillRuntimeException;
+import org.apache.drill.exec.physical.impl.join.RowKeyJoin;
+import org.apache.drill.exec.record.AbstractRecordBatch.BatchState;
+import org.apache.drill.exec.vector.ValueVector;
+
+import com.fasterxml.jackson.annotation.JsonIgnore;
+
+import java.nio.ByteBuffer;
+
+/**
+ * A RestrictedMapRDBSubScanSpec encapsulates a join instance which contains the ValueVectors of row keys and
+ * is associated with this sub-scan and also exposes an iterator type interface over the row key vectors.
+ */
+public class RestrictedMapRDBSubScanSpec extends MapRDBSubScanSpec {
+
+  /**
+   * The RowKeyJoin instance (specific to one minor fragment) which will supply this
+   * subscan with the set of rowkeys. For efficiency, we keep a reference to this
+   * join rather than making another copy of the rowkeys.
+   */
+  private RowKeyJoin rjbatch = null;
+
+  /**
+   * The following are needed to maintain internal state of iteration over the set
+   * of row keys
+   */
+  private ValueVector rowKeyVector = null; // the current row key value vector
+  private int currentIndex = 0;  // the current index within the row key vector
+  private int maxOccupiedIndex = -1; // max occupied index within a row key vector
+
+  public RestrictedMapRDBSubScanSpec(String tableName, String regionServer, byte[] serializedFilter, String userName) {
+    super(tableName, null, regionServer, null, null, serializedFilter, null, userName);
+  }
+  /* package */ RestrictedMapRDBSubScanSpec() {
+    // empty constructor, to be used with builder pattern;
+  }
+
+  public void setJoinForSubScan(RowKeyJoin rjbatch) {
+    this.rjbatch = rjbatch;
+  }
+
+  @JsonIgnore
+  public RowKeyJoin getJoinForSubScan() {
+    return rjbatch;
+  }
+
+  @JsonIgnore
+  private void init(Pair<ValueVector, Integer> b) {
+    this.maxOccupiedIndex = b.getRight();
+    this.rowKeyVector = b.getLeft();
+    this.currentIndex = 0;
+  }
+
+  /**
+   * Return {@code true} if a valid rowkey batch is available, {@code false} otherwise
+   */
+  @JsonIgnore
+  public boolean readyToGetRowKey() {
+    return rjbatch != null && rjbatch.hasRowKeyBatch();
+  }
+
+  /**
+   * Return {@code true} if the row key join is in the build schema phase
+   */
+  @JsonIgnore
+  public boolean isBuildSchemaPhase() {
+    return rjbatch.getBatchState() == BatchState.BUILD_SCHEMA;
+  }
+
+  /**
+   * Returns {@code true} if the iteration has more row keys.
+   * (In other words, returns {@code true} if {@link #nextRowKey} would
+   * return a non-null row key)
+   * @return {@code true} if the iteration has more row keys
+   */
+  @JsonIgnore
+  public boolean hasRowKey() {
+    if (rowKeyVector != null && currentIndex <= maxOccupiedIndex) {
+      return true;
+    }
+
+    if (rjbatch != null) {
+      Pair<ValueVector, Integer> currentBatch = rjbatch.nextRowKeyBatch();
+
+      // note that the hash table could be null initially during the BUILD_SCHEMA phase
+      if (currentBatch != null) {
+        init(currentBatch);
+        return true;
+      }
+    }
+
+    return false;
+  }
+
+  @JsonIgnore
+  public int getMaxRowKeysToBeRead() {
+    if (rjbatch != null) {
+      Pair<ValueVector, Integer> currentBatch = rjbatch.nextRowKeyBatch();
+
+      // note that the currentBatch could be null initially during the BUILD_SCHEMA phase
+      if (currentBatch != null) {
+        init(currentBatch);
+      }
+    }
+    return maxOccupiedIndex + 1;
+  }
+
+  /**
+   * Returns number of rowKeys that can be read.
+   * Number of rowKeys returned will be numRowKeysToRead at the most i.e. it
+   * will be less than numRowKeysToRead if only that many exist in the currentBatch.
+   */
+  @JsonIgnore
+  public int hasRowKeys(int numRowKeysToRead) {
+    int numKeys = 0;
+
+    // if there is pending rows from the current batch, read them first
+    // in chunks of numRowsToRead rows
+    if (rowKeyVector != null && currentIndex <= maxOccupiedIndex) {
+        numKeys = Math.min(numRowKeysToRead, maxOccupiedIndex - currentIndex + 1);
+        return numKeys;
+    }
+
+    // otherwise, get the next batch of rowkeys
+    if (rjbatch != null) {
+      Pair<ValueVector, Integer> currentBatch = rjbatch.nextRowKeyBatch();
+
+      // note that the currentBatch could be null initially during the BUILD_SCHEMA phase
+      if (currentBatch != null) {
+        init(currentBatch);
+        numKeys = Math.min(numRowKeysToRead, maxOccupiedIndex - currentIndex + 1);
+      }
+    }
+
+    return numKeys;
+  }
+
+  /**
+   * Returns ids of rowKeys to be read.
+   * Number of rowKey ids returned will be numRowKeysToRead at the most i.e. it
+   * will be less than numRowKeysToRead if only that many exist in the currentBatch.
+   */
+  @JsonIgnore
+  public ByteBuffer[] getRowKeyIdsToRead(int numRowKeysToRead) {
+
+    int numKeys = hasRowKeys(numRowKeysToRead);
+    if (numKeys == 0) return null;
+
+    int index = 0;
+    final ByteBuffer[] rowKeyIds = new ByteBuffer[numKeys];
+
+    while (index < numKeys) {
+      Object o = rowKeyVector.getAccessor().getObject(currentIndex + index);
+      rowKeyIds[index++] = IdCodec.encode(o.toString());
+    }
+
+    updateRowKeysRead(numKeys);
+    return rowKeyIds;
+  }
+
+  /**
+   * updates the index to reflect number of keys read.
+   */
+  @JsonIgnore
+  public void updateRowKeysRead(int numKeys) {
+    currentIndex += numKeys;
+  }
+
+  /**
+   * Returns the next row key in the iteration.
+   * @return the next row key in the iteration or null if no more row keys
+   */
+  @JsonIgnore
+  public String nextRowKey() {
+    if (hasRowKey()) {
+      // get the entry at the current index within this batch
+      Object o = rowKeyVector.getAccessor().getObject(currentIndex++);
+      if (o == null) {
+        throw new DrillRuntimeException("Encountered a null row key during restricted subscan !");
+      }
+
+      // this is specific to the way the hash join maintains its entries. once we have reached the max
+      // occupied index within a batch, move to the next one and reset the current index to 0
+      // TODO: we should try to abstract this out
+      if (currentIndex > maxOccupiedIndex) {
+        Pair<ValueVector, Integer> currentBatch = rjbatch.nextRowKeyBatch();
+        if (currentBatch != null) {
+          init(currentBatch);
+        }
+      }
+
+      return o.toString();
+    }
+    return null;
+  }
+
+}
diff --git a/contrib/format-maprdb/src/main/java/org/apache/drill/exec/store/mapr/db/binary/BinaryTableGroupScan.java b/contrib/format-maprdb/src/main/java/org/apache/drill/exec/store/mapr/db/binary/BinaryTableGroupScan.java
index 086ae21..a135464 100644
--- a/contrib/format-maprdb/src/main/java/org/apache/drill/exec/store/mapr/db/binary/BinaryTableGroupScan.java
+++ b/contrib/format-maprdb/src/main/java/org/apache/drill/exec/store/mapr/db/binary/BinaryTableGroupScan.java
@@ -23,6 +23,8 @@ import java.io.IOException;
 import java.util.List;
 import java.util.TreeMap;
 
+import org.apache.calcite.rel.RelNode;
+import org.apache.calcite.rex.RexNode;
 import org.apache.drill.common.exceptions.DrillRuntimeException;
 import org.apache.drill.common.exceptions.ExecutionSetupException;
 import org.apache.drill.common.expression.SchemaPath;
@@ -31,6 +33,7 @@ import org.apache.drill.exec.physical.base.PhysicalOperator;
 import org.apache.drill.exec.physical.base.ScanStats;
 import org.apache.drill.exec.physical.base.ScanStats.GroupScanProperty;
 import org.apache.drill.exec.store.AbstractStoragePlugin;
+import org.apache.drill.exec.planner.index.Statistics;
 import org.apache.drill.exec.store.StoragePluginRegistry;
 import org.apache.drill.exec.store.dfs.FileSystemConfig;
 import org.apache.drill.exec.store.hbase.DrillHBaseConstants;
@@ -130,7 +133,7 @@ public class BinaryTableGroupScan extends MapRDBGroupScan implements DrillHBaseC
         tableStats = new MapRDBTableStats(getHBaseConf(), hbaseScanSpec.getTableName());
       }
       boolean foundStartRegion = false;
-      regionsToScan = new TreeMap<>();
+      final TreeMap<TabletFragmentInfo, String> regionsToScan = new TreeMap<TabletFragmentInfo, String>();
       List<HRegionLocation> regionLocations = locator.getAllRegionLocations();
       for (HRegionLocation regionLocation : regionLocations) {
         HRegionInfo regionInfo = regionLocation.getRegionInfo();
@@ -143,6 +146,7 @@ public class BinaryTableGroupScan extends MapRDBGroupScan implements DrillHBaseC
           break;
         }
       }
+      setRegionsToScan(regionsToScan);
     } catch (Exception e) {
       throw new DrillRuntimeException("Error getting region info for table: " + hbaseScanSpec.getTableName(), e);
     }
@@ -154,11 +158,13 @@ public class BinaryTableGroupScan extends MapRDBGroupScan implements DrillHBaseC
     HBaseScanSpec spec = hbaseScanSpec;
     MapRDBSubScanSpec subScanSpec = new MapRDBSubScanSpec(
         spec.getTableName(),
-        regionsToScan.get(tfi),
+        null /* indexFid */,
+        getRegionsToScan().get(tfi),
         (!isNullOrEmpty(spec.getStartRow()) && tfi.containsRow(spec.getStartRow())) ? spec.getStartRow() : tfi.getStartKey(),
         (!isNullOrEmpty(spec.getStopRow()) && tfi.containsRow(spec.getStopRow())) ? spec.getStopRow() : tfi.getEndKey(),
         spec.getSerializedFilter(),
-        null);
+        null,
+        getUserName());
     return subScanSpec;
   }
 
@@ -191,6 +197,7 @@ public class BinaryTableGroupScan extends MapRDBGroupScan implements DrillHBaseC
     return getFormatPlugin().getHBaseConf();
   }
 
+  @Override
   @JsonIgnore
   public String getTableName() {
     return getHBaseScanSpec().getTableName();
@@ -213,4 +220,25 @@ public class BinaryTableGroupScan extends MapRDBGroupScan implements DrillHBaseC
     return hbaseScanSpec;
   }
 
+  @Override
+  public void setRowCount(RexNode condition, double count, double capRowCount) {
+    throw new UnsupportedOperationException("setRowCount() not implemented for BinaryTableGroupScan");
+  }
+
+  @Override
+  public double getRowCount(RexNode condition, RelNode scanRel) {
+    return Statistics.ROWCOUNT_UNKNOWN;
+  }
+
+  @Override
+  public Statistics getStatistics() {
+    throw new UnsupportedOperationException("getStatistics() not implemented for BinaryTableGroupScan");
+  }
+
+  @Override
+  @JsonIgnore
+  public boolean isIndexScan() {
+    return false;
+  }
+
 }
diff --git a/contrib/format-maprdb/src/main/java/org/apache/drill/exec/store/mapr/db/json/AllTextValueWriter.java b/contrib/format-maprdb/src/main/java/org/apache/drill/exec/store/mapr/db/json/AllTextValueWriter.java
new file mode 100644
index 0000000..bbb2aec
--- /dev/null
+++ b/contrib/format-maprdb/src/main/java/org/apache/drill/exec/store/mapr/db/json/AllTextValueWriter.java
@@ -0,0 +1,80 @@
+/*
+ * 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.store.mapr.db.json;
+
+import java.nio.ByteBuffer;
+
+import org.apache.drill.exec.vector.complex.impl.MapOrListWriterImpl;
+import org.apache.drill.exec.vector.complex.writer.BaseWriter.MapOrListWriter;
+import org.ojai.DocumentReader;
+
+import com.mapr.org.apache.hadoop.hbase.util.Bytes;
+
+import io.netty.buffer.DrillBuf;
+
+public class AllTextValueWriter extends OjaiValueWriter {
+
+  public AllTextValueWriter(DrillBuf buffer) {
+    super(buffer);
+  }
+
+  protected void writeTimeStamp(MapOrListWriterImpl writer, String fieldName, DocumentReader reader) {
+    writeString(writer, fieldName, reader.getTimestamp().toUTCString());
+  }
+
+  protected void writeTime(MapOrListWriterImpl writer, String fieldName, DocumentReader reader) {
+    writeString(writer, fieldName, reader.getTime().toTimeStr());
+  }
+
+  protected void writeDate(MapOrListWriterImpl writer, String fieldName, DocumentReader reader) {
+    writeString(writer, fieldName, reader.getDate().toDateStr());
+  }
+
+  protected void writeDouble(MapOrListWriterImpl writer, String fieldName, DocumentReader reader) {
+    writeString(writer, fieldName, String.valueOf(reader.getDouble()));
+  }
+
+  protected void writeFloat(MapOrListWriterImpl writer, String fieldName, DocumentReader reader) {
+    writeString(writer, fieldName, String.valueOf(reader.getFloat()));
+  }
+
+  protected void writeLong(MapOrListWriterImpl writer, String fieldName, DocumentReader reader) {
+    writeString(writer, fieldName, String.valueOf(reader.getLong()));
+  }
+
+  protected void writeInt(MapOrListWriterImpl writer, String fieldName, DocumentReader reader) {
+    writeString(writer, fieldName, String.valueOf(reader.getInt()));
+  }
+
+  protected void writeShort(MapOrListWriterImpl writer, String fieldName, DocumentReader reader) {
+    writeString(writer, fieldName, String.valueOf(reader.getShort()));
+  }
+
+  protected void writeByte(MapOrListWriterImpl writer, String fieldName, DocumentReader reader) {
+    writeString(writer, fieldName, String.valueOf(reader.getByte()));
+  }
+
+  protected void writeBoolean(MapOrListWriterImpl writer, String fieldName, DocumentReader reader) {
+    writeString(writer, fieldName, String.valueOf(reader.getBoolean()));
+  }
+
+  protected void writeBinary(MapOrListWriter writer, String fieldName, ByteBuffer buf) {
+    writeString(writer, fieldName, Bytes.toString(buf));
+  }
+
+}
diff --git a/contrib/format-maprdb/src/main/java/org/apache/drill/exec/store/mapr/db/json/CompareFunctionsProcessor.java b/contrib/format-maprdb/src/main/java/org/apache/drill/exec/store/mapr/db/json/CompareFunctionsProcessor.java
index 5dab14e..a4cb0bd 100644
--- a/contrib/format-maprdb/src/main/java/org/apache/drill/exec/store/mapr/db/json/CompareFunctionsProcessor.java
+++ b/contrib/format-maprdb/src/main/java/org/apache/drill/exec/store/mapr/db/json/CompareFunctionsProcessor.java
@@ -17,6 +17,8 @@
  */
 package org.apache.drill.exec.store.mapr.db.json;
 
+import static com.mapr.db.rowcol.DBValueBuilderImpl.KeyValueBuilder;
+
 import org.apache.drill.common.expression.FunctionCall;
 import org.apache.drill.common.expression.LogicalExpression;
 import org.apache.drill.common.expression.SchemaPath;
@@ -41,6 +43,9 @@ import org.ojai.types.OTime;
 import org.apache.drill.shaded.guava.com.google.common.collect.ImmutableMap;
 import org.apache.drill.shaded.guava.com.google.common.collect.ImmutableSet;
 import com.mapr.db.rowcol.KeyValueBuilder;
+import com.mapr.db.util.SqlHelper;
+
+import org.ojai.types.OTimestamp;
 
 class CompareFunctionsProcessor extends AbstractExprVisitor<Boolean, LogicalExpression, RuntimeException> {
 
@@ -109,7 +114,7 @@ class CompareFunctionsProcessor extends AbstractExprVisitor<Boolean, LogicalExpr
     }
 
     if (valueArg instanceof QuotedString) {
-      this.value = KeyValueBuilder.initFrom(((QuotedString) valueArg).value);
+      this.value = SqlHelper.decodeStringAsValue(((QuotedString) valueArg).value);
       this.path = path;
       return true;
     }
@@ -182,11 +187,9 @@ class CompareFunctionsProcessor extends AbstractExprVisitor<Boolean, LogicalExpr
     }
 
     if (valueArg instanceof TimeStampExpression) {
-      // disable pushdown of TimeStampExpression type until bug 22824 is fixed.
-      //
-      // this.value = KeyValueBuilder.initFrom(new OTimestamp(((TimeStampExpression)valueArg).getTimeStamp()));
-      // this.path = path;
-      // return true;
+      this.value = KeyValueBuilder.initFrom(new OTimestamp(((TimeStampExpression)valueArg).getTimeStamp()));
+      this.path = path;
+      return true;
     }
 
     return false;
diff --git a/contrib/format-maprdb/src/main/java/org/apache/drill/exec/store/mapr/db/json/DocumentReaderVectorWriter.java b/contrib/format-maprdb/src/main/java/org/apache/drill/exec/store/mapr/db/json/DocumentReaderVectorWriter.java
new file mode 100644
index 0000000..1c9dead
--- /dev/null
+++ b/contrib/format-maprdb/src/main/java/org/apache/drill/exec/store/mapr/db/json/DocumentReaderVectorWriter.java
@@ -0,0 +1,42 @@
+/*
+ * 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.store.mapr.db.json;
+
+import org.apache.drill.exec.exception.SchemaChangeException;
+import org.apache.drill.exec.vector.complex.impl.VectorContainerWriter;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import com.mapr.db.ojai.DBDocumentReaderBase;
+
+/**
+ * Base class for writing a single OJAI Document to Drill's Value Vectors.
+ */
+abstract class DocumentReaderVectorWriter {
+  protected static final Logger logger = LoggerFactory.getLogger(DocumentReaderVectorWriter.class);
+
+  protected final OjaiValueWriter valueWriter;
+
+  protected DocumentReaderVectorWriter(final OjaiValueWriter valueWriter) {
+    this.valueWriter = valueWriter;
+  }
+
+  protected abstract void writeDBDocument(final VectorContainerWriter writer, final DBDocumentReaderBase reader)
+      throws SchemaChangeException;
+
+}
diff --git a/contrib/format-maprdb/src/main/java/org/apache/drill/exec/store/mapr/db/json/FieldPathHelper.java b/contrib/format-maprdb/src/main/java/org/apache/drill/exec/store/mapr/db/json/FieldPathHelper.java
new file mode 100644
index 0000000..d8d3792
--- /dev/null
+++ b/contrib/format-maprdb/src/main/java/org/apache/drill/exec/store/mapr/db/json/FieldPathHelper.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.store.mapr.db.json;
+
+import java.util.Stack;
+
+import org.apache.drill.common.expression.PathSegment;
+import org.apache.drill.common.expression.SchemaPath;
+import org.ojai.FieldPath;
+import org.ojai.FieldSegment;
+
+public class FieldPathHelper {
+
+  /**
+   * Returns {@link SchemaPath} equivalent of the specified {@link FieldPath}.
+   */
+  public static SchemaPath fieldPath2SchemaPath(FieldPath fieldPath) {
+    Stack<FieldSegment> fieldSegments = new Stack<FieldSegment>();
+    FieldSegment seg = fieldPath.getRootSegment();
+    while (seg != null) {
+      fieldSegments.push(seg);
+      seg = seg.getChild();
+    }
+
+    PathSegment child = null;
+    while (!fieldSegments.isEmpty()) {
+      seg = fieldSegments.pop();
+      if (seg.isNamed()) {
+        child = new PathSegment.NameSegment(((FieldSegment.NameSegment)seg).getName(), child);
+      } else {
+        child = new PathSegment.ArraySegment(((FieldSegment.IndexSegment)seg).getIndex(), child);
+      }
+    }
+    return new SchemaPath((PathSegment.NameSegment)child);
+  }
+
+  /**
+   * Returns {@link FieldPath} equivalent of the specified {@link SchemaPath}.
+   */
+  public static FieldPath schemaPath2FieldPath(SchemaPath column) {
+    Stack<PathSegment> pathSegments = new Stack<PathSegment>();
+    PathSegment seg = column.getRootSegment();
+    while (seg != null) {
+      pathSegments.push(seg);
+      seg = seg.getChild();
+    }
+
+    FieldSegment child = null;
+    while (!pathSegments.isEmpty()) {
+      seg = pathSegments.pop();
+      if (seg.isNamed()) {
+        child = new FieldSegment.NameSegment(((PathSegment.NameSegment)seg).getPath(), child, false);
+      } else {
+        child = new FieldSegment.IndexSegment(((PathSegment.ArraySegment)seg).getIndex(), child);
+      }
+    }
+    return new FieldPath((FieldSegment.NameSegment) child);
+  }
+
+}
diff --git a/contrib/format-maprdb/src/main/java/org/apache/drill/exec/store/mapr/db/json/FieldTransferVectorWriter.java b/contrib/format-maprdb/src/main/java/org/apache/drill/exec/store/mapr/db/json/FieldTransferVectorWriter.java
new file mode 100644
index 0000000..67bbcb3
--- /dev/null
+++ b/contrib/format-maprdb/src/main/java/org/apache/drill/exec/store/mapr/db/json/FieldTransferVectorWriter.java
@@ -0,0 +1,49 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.drill.exec.store.mapr.db.json;
+
+import static org.apache.drill.exec.store.mapr.PluginErrorHandler.dataReadError;
+
+import org.apache.drill.exec.exception.SchemaChangeException;
+import org.apache.drill.exec.vector.complex.impl.MapOrListWriterImpl;
+import org.apache.drill.exec.vector.complex.impl.VectorContainerWriter;
+import org.ojai.DocumentReader.EventType;
+
+import com.mapr.db.ojai.DBDocumentReaderBase;
+
+/**
+ *  This implementation of DocumentReaderVectorWriter does field by field transfer the OJAI Document
+ *  to Drill Value Vectors.
+ */
+class FieldTransferVectorWriter extends DocumentReaderVectorWriter {
+
+  protected FieldTransferVectorWriter(final OjaiValueWriter valueWriter) {
+    super(valueWriter);
+  }
+
+  @Override
+  protected void writeDBDocument(VectorContainerWriter vectorWriter, DBDocumentReaderBase reader)
+      throws SchemaChangeException {
+    MapOrListWriterImpl writer = new MapOrListWriterImpl(vectorWriter.rootAsMap());
+    if (reader.next() != EventType.START_MAP) {
+      throw dataReadError(logger, "The document did not start with START_MAP!");
+    }
+    valueWriter.writeToListOrMap(writer, reader);
+  }
+
+}
diff --git a/contrib/format-maprdb/src/main/java/org/apache/drill/exec/store/mapr/db/json/IdOnlyVectorWriter.java b/contrib/format-maprdb/src/main/java/org/apache/drill/exec/store/mapr/db/json/IdOnlyVectorWriter.java
new file mode 100644
index 0000000..2c9f762
--- /dev/null
+++ b/contrib/format-maprdb/src/main/java/org/apache/drill/exec/store/mapr/db/json/IdOnlyVectorWriter.java
@@ -0,0 +1,65 @@
+/*
+ * 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.store.mapr.db.json;
+
+import static org.apache.drill.exec.store.mapr.PluginErrorHandler.schemaChangeException;
+import static org.ojai.DocumentConstants.ID_KEY;
+
+import org.apache.drill.exec.exception.SchemaChangeException;
+import org.apache.drill.exec.vector.complex.impl.VectorContainerWriter;
+import org.apache.drill.exec.vector.complex.writer.BaseWriter.MapWriter;
+import org.ojai.Value;
+
+import com.mapr.db.impl.IdCodec;
+import com.mapr.db.ojai.DBDocumentReaderBase;
+
+/**
+ *  This implementation of DocumentReaderVectorWriter writes only the "_id" field from the OJAI
+ *  Document to Drill Value Vectors. This is useful for "_id" only queries.
+ */
+class IdOnlyVectorWriter extends DocumentReaderVectorWriter {
+
+  protected IdOnlyVectorWriter(final OjaiValueWriter valueWriter) {
+    super(valueWriter);
+  }
+
+  @Override
+  public void writeDBDocument(VectorContainerWriter vectorWriter, DBDocumentReaderBase reader)
+      throws SchemaChangeException {
+    MapWriter writer = vectorWriter.rootAsMap();
+
+    Value id = reader.getId();
+    try {
+      switch(id.getType()) {
+      case STRING:
+        valueWriter.writeString(writer, ID_KEY, id.getString());
+        break;
+      case BINARY:
+        valueWriter.writeBinary(writer, ID_KEY, id.getBinary());
+        break;
+      default:
+        throw new UnsupportedOperationException(id.getType() +
+            " is not a supported type for _id field.");
+      }
+    } catch (IllegalStateException | IllegalArgumentException e) {
+      throw schemaChangeException(logger, e, "Possible schema change at _id: '%s'", IdCodec.asString(id));
+    }
+
+  }
+
+}
diff --git a/contrib/format-maprdb/src/main/java/org/apache/drill/exec/store/mapr/db/json/JsonConditionBuilder.java b/contrib/format-maprdb/src/main/java/org/apache/drill/exec/store/mapr/db/json/JsonConditionBuilder.java
index 3b64bf9..ce4e677 100644
--- a/contrib/format-maprdb/src/main/java/org/apache/drill/exec/store/mapr/db/json/JsonConditionBuilder.java
+++ b/contrib/format-maprdb/src/main/java/org/apache/drill/exec/store/mapr/db/json/JsonConditionBuilder.java
@@ -22,13 +22,13 @@ import org.apache.drill.common.expression.FunctionCall;
 import org.apache.drill.common.expression.LogicalExpression;
 import org.apache.drill.common.expression.visitors.AbstractExprVisitor;
 import org.apache.drill.exec.store.hbase.DrillHBaseConstants;
-import org.apache.drill.exec.store.mapr.db.util.FieldPathHelper;
 import org.ojai.Value;
 import org.ojai.store.QueryCondition;
 import org.ojai.store.QueryCondition.Op;
 
 import org.apache.drill.shaded.guava.com.google.common.collect.ImmutableList;
 import com.mapr.db.MapRDB;
+import com.mapr.db.impl.MapRDBImpl;
 
 public class JsonConditionBuilder extends AbstractExprVisitor<JsonScanSpec, Void, RuntimeException> implements DrillHBaseConstants {
 
@@ -88,14 +88,27 @@ public class JsonConditionBuilder extends AbstractExprVisitor<JsonScanSpec, Void
           JsonScanSpec nextScanSpec = args.get(i).accept(this, null);
           if (nodeScanSpec != null && nextScanSpec != null) {
             nodeScanSpec.mergeScanSpec(functionName, nextScanSpec);
-        } else {
-          allExpressionsConverted = false;
-          if ("booleanAnd".equals(functionName)) {
+          } else {
+            allExpressionsConverted = false;
+            if ("booleanAnd".equals(functionName)) {
               nodeScanSpec = nodeScanSpec == null ? nextScanSpec : nodeScanSpec;
             }
           }
         }
         break;
+
+      case "ojai_sizeof":
+      case "ojai_typeof":
+      case "ojai_nottypeof":
+      case "ojai_matches":
+      case "ojai_notmatches":
+      case "ojai_condition": {
+        final OjaiFunctionsProcessor processor = OjaiFunctionsProcessor.process(call);
+        if (processor != null) {
+                return new JsonScanSpec(groupScan.getTableName(), groupScan.getIndexDesc(),
+                                processor.getCondition());
+        }
+      }
       }
     }
 
@@ -159,80 +172,76 @@ public class JsonConditionBuilder extends AbstractExprVisitor<JsonScanSpec, Void
   private JsonScanSpec createJsonScanSpec(FunctionCall call,
       CompareFunctionsProcessor processor) {
     String functionName = processor.getFunctionName();
-    String fieldPath = FieldPathHelper.schemaPathToFieldPath(processor.getPath()).asPathString();
+    String fieldPath = FieldPathHelper.schemaPath2FieldPath(processor.getPath()).asPathString();
     Value fieldValue = processor.getValue();
 
     QueryCondition cond = null;
     switch (functionName) {
     case "equal":
-      cond = MapRDB.newCondition();
+      cond = MapRDBImpl.newCondition();
       setIsCondition(cond, fieldPath, Op.EQUAL, fieldValue);
-      cond.build();
       break;
 
     case "not_equal":
-      cond = MapRDB.newCondition();
+      cond = MapRDBImpl.newCondition();
       setIsCondition(cond, fieldPath, Op.NOT_EQUAL, fieldValue);
-      cond.build();
       break;
 
     case "less_than":
-      cond = MapRDB.newCondition();
+      cond = MapRDBImpl.newCondition();
       setIsCondition(cond, fieldPath, Op.LESS, fieldValue);
-      cond.build();
       break;
 
     case "less_than_or_equal_to":
-      cond = MapRDB.newCondition();
+      cond = MapRDBImpl.newCondition();
       setIsCondition(cond, fieldPath, Op.LESS_OR_EQUAL, fieldValue);
-      cond.build();
       break;
 
     case "greater_than":
-      cond = MapRDB.newCondition();
+      cond = MapRDBImpl.newCondition();
       setIsCondition(cond, fieldPath, Op.GREATER, fieldValue);
-      cond.build();
       break;
 
     case "greater_than_or_equal_to":
-      cond = MapRDB.newCondition();
+      cond = MapRDBImpl.newCondition();
       setIsCondition(cond, fieldPath, Op.GREATER_OR_EQUAL, fieldValue);
-      cond.build();
       break;
 
     case "isnull":
-      cond = MapRDB.newCondition().notExists(fieldPath).build();
+      cond = MapRDBImpl.newCondition().notExists(fieldPath);
       break;
 
     case "isnotnull":
-      cond = MapRDB.newCondition().exists(fieldPath).build();
+      cond = MapRDBImpl.newCondition().exists(fieldPath);
       break;
 
     case "istrue":
-      cond = MapRDB.newCondition().is(fieldPath, Op.EQUAL, true).build();
+      cond = MapRDBImpl.newCondition().is(fieldPath, Op.EQUAL, true);
       break;
 
     case "isnotfalse":
-      cond = MapRDB.newCondition().is(fieldPath, Op.NOT_EQUAL, false).build();
+      cond = MapRDBImpl.newCondition().is(fieldPath, Op.NOT_EQUAL, false);
       break;
 
     case "isfalse":
-      cond = MapRDB.newCondition().is(fieldPath, Op.EQUAL, false).build();
+      cond = MapRDBImpl.newCondition().is(fieldPath, Op.EQUAL, false);
       break;
 
     case "isnottrue":
-      cond = MapRDB.newCondition().is(fieldPath, Op.NOT_EQUAL, true).build();
+      cond = MapRDBImpl.newCondition().is(fieldPath, Op.NOT_EQUAL, true);
       break;
 
     case "like":
-      cond = MapRDB.newCondition().like(fieldPath, fieldValue.getString()).build();
+      cond = MapRDBImpl.newCondition().like(fieldPath, fieldValue.getString());
       break;
 
     default:
     }
 
     if (cond != null) {
-      return new JsonScanSpec(groupScan.getTableName(), cond);
+      return new JsonScanSpec(groupScan.getTableName(),
+                              groupScan.getIndexDesc(),
+                              cond.build());
     }
 
     return null;
diff --git a/contrib/format-maprdb/src/main/java/org/apache/drill/exec/store/mapr/db/json/JsonScanSpec.java b/contrib/format-maprdb/src/main/java/org/apache/drill/exec/store/mapr/db/json/JsonScanSpec.java
index 6858394..8d06f17 100644
--- a/contrib/format-maprdb/src/main/java/org/apache/drill/exec/store/mapr/db/json/JsonScanSpec.java
+++ b/contrib/format-maprdb/src/main/java/org/apache/drill/exec/store/mapr/db/json/JsonScanSpec.java
@@ -17,50 +17,78 @@
  */
 package org.apache.drill.exec.store.mapr.db.json;
 
+import java.nio.ByteBuffer;
+import java.util.List;
+
+import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.hbase.HConstants;
 import org.ojai.store.QueryCondition;
 
 import com.fasterxml.jackson.annotation.JsonCreator;
 import com.fasterxml.jackson.annotation.JsonIgnore;
 import com.fasterxml.jackson.annotation.JsonProperty;
-import com.mapr.db.MapRDB;
 import com.mapr.db.impl.ConditionImpl;
+import com.mapr.db.impl.ConditionNode.RowkeyRange;
+import com.mapr.db.impl.MapRDBImpl;
+import com.mapr.db.index.IndexDesc;
 
 public class JsonScanSpec {
-	protected String tableName;
-	protected QueryCondition condition;
-	
-	@JsonCreator
-	public JsonScanSpec(@JsonProperty("tableName") String tableName,
-	                    @JsonProperty("condition") QueryCondition condition) {
-	  this.tableName = tableName;
-	  this.condition = condition;
+  protected String tableName;
+  protected IndexDesc indexDesc;
+  protected QueryCondition condition;
+  protected byte[] startRow;
+  protected byte[] stopRow;
+
+  @JsonCreator
+  public JsonScanSpec(@JsonProperty("tableName") String tableName,
+                      @JsonProperty("indexDesc") IndexDesc indexDesc,
+                      @JsonProperty("condition") QueryCondition condition) {
+    this.tableName = tableName;
+    this.indexDesc = indexDesc;
+    this.condition = condition;
+    if (this.condition != null) {
+      List<RowkeyRange> rkRanges = ((ConditionImpl)this.condition).getRowkeyRanges();
+      if (rkRanges.size() > 0) {
+        startRow = rkRanges.get(0).getStartRow();
+        stopRow  = rkRanges.get(rkRanges.size() - 1).getStopRow();
+      } else {
+        startRow = HConstants.EMPTY_START_ROW;
+        stopRow  = HConstants.EMPTY_END_ROW;
+      }
+    }
   }
 
   public String getTableName() {
     return this.tableName;
   }
 
+  public IndexDesc getIndexDesc() {
+    return this.indexDesc;
+  }
+
+  public void setStartRow(byte []startRow) {
+    this.startRow = startRow;
+  }
+
+  public void setStopRow(byte []stopRow) {
+    this.stopRow = stopRow;
+  }
+
   public byte[] getStartRow() {
-    if (condition == null) {
-      return HConstants.EMPTY_START_ROW;
-    }
-    return ((ConditionImpl)this.condition).getRowkeyRanges().get(0).getStartRow();
+    return this.startRow;
   }
 
   public byte[] getStopRow() {
-    if (condition == null) {
-      return HConstants.EMPTY_END_ROW;
-    }
-    
-    return ((ConditionImpl)this.condition).getRowkeyRanges().get(0).getStopRow();
+    return this.stopRow;
   }
 
-  public Object getSerializedFilter() {
+  public byte[] getSerializedFilter() {
     if (this.condition != null) {
-      return ((ConditionImpl)this.condition).getDescriptor().getSerialized();
+      ByteBuffer bbuf = ((ConditionImpl)this.condition).getDescriptor().getSerialized();
+      byte[] serFilter = new byte[bbuf.limit() - bbuf.position()];
+      bbuf.get(serFilter);
+      return serFilter;
     }
-
     return null;
   }
 
@@ -73,10 +101,23 @@ public class JsonScanSpec {
     return this.condition;
   }
 
-  public void mergeScanSpec(String functionName, JsonScanSpec scanSpec) {
+  public boolean isSecondaryIndex() {
+    return (this.indexDesc != null);
+  }
+
+  @JsonIgnore
+  public Path getPrimaryTablePath() {
+    return (this.indexDesc == null) ? null : new Path(this.indexDesc.getPrimaryTablePath());
+  }
+
+  @JsonIgnore
+  public String getIndexName() {
+    return (this.indexDesc == null) ? null : this.indexDesc.getIndexName();
+  }
 
+  public void mergeScanSpec(String functionName, JsonScanSpec scanSpec) {
     if (this.condition != null && scanSpec.getCondition() != null) {
-      QueryCondition newCond = MapRDB.newCondition();
+      QueryCondition newCond = MapRDBImpl.newCondition();
       switch (functionName) {
       case "booleanAnd":
         newCond.and();
@@ -84,7 +125,7 @@ public class JsonScanSpec {
       case "booleanOr":
         newCond.or();
         break;
-        default:
+      default:
           assert(false);
       }
 
@@ -98,11 +139,13 @@ public class JsonScanSpec {
       this.condition = scanSpec.getCondition();
     }
   }
-  
+
   @Override
   public String toString() {
+    String fidInfo = (getIndexDesc() != null)? ", indexName=" + getIndexName() : "";
     return "JsonScanSpec [tableName=" + tableName
         + ", condition=" + (condition == null ? null : condition.toString())
+        + fidInfo
         + "]";
   }
 
diff --git a/contrib/format-maprdb/src/main/java/org/apache/drill/exec/store/mapr/db/json/JsonSubScanSpec.java b/contrib/format-maprdb/src/main/java/org/apache/drill/exec/store/mapr/db/json/JsonSubScanSpec.java
index 3fe0a3b..1e7eb31 100644
--- a/contrib/format-maprdb/src/main/java/org/apache/drill/exec/store/mapr/db/json/JsonSubScanSpec.java
+++ b/contrib/format-maprdb/src/main/java/org/apache/drill/exec/store/mapr/db/json/JsonSubScanSpec.java
@@ -18,75 +18,38 @@
 package org.apache.drill.exec.store.mapr.db.json;
 
 import java.nio.ByteBuffer;
-import java.util.Arrays;
 
 import org.apache.drill.exec.store.mapr.db.MapRDBSubScanSpec;
-import org.apache.hadoop.hbase.HConstants;
-import org.ojai.DocumentConstants;
-import org.ojai.Value;
 import org.ojai.store.QueryCondition;
-import org.ojai.store.QueryCondition.Op;
 
-import com.fasterxml.jackson.annotation.JsonCreator;
 import com.fasterxml.jackson.annotation.JsonIgnore;
-import com.fasterxml.jackson.annotation.JsonProperty;
-import com.mapr.db.MapRDB;
 import com.mapr.db.impl.ConditionImpl;
-import com.mapr.db.impl.IdCodec;
+import com.mapr.db.impl.MapRDBImpl;
+import com.mapr.db.index.IndexDesc;
 
+/**
+ * This class is a helper extension of {@link MapRDBSubScanSpec} class and does not
+ * get serialized or deserialized.
+ */
 public class JsonSubScanSpec extends MapRDBSubScanSpec {
 
   protected QueryCondition condition;
 
-  @JsonCreator
-  public JsonSubScanSpec(@JsonProperty("tableName") String tableName,
-                         @JsonProperty("regionServer") String regionServer,
-                         @JsonProperty("startRow") byte[] startRow,
-                         @JsonProperty("stopRow") byte[] stopRow,
-                         @JsonProperty("cond") QueryCondition cond) {
-    super(tableName, regionServer, null, null, null, null);
-
-    this.condition = MapRDB.newCondition().and();
-
-    if (cond != null) {
-      this.condition.condition(cond);
-    }
+  public JsonSubScanSpec(String tableName, IndexDesc indexDesc, String regionServer,
+                         QueryCondition scanRangeCondition, QueryCondition userCondition,
+                         byte[] startRow, byte[] stopRow, String userName) {
+    super(tableName, indexDesc, regionServer, startRow, stopRow, null, null, userName);
 
-    if (startRow != null &&
-        Arrays.equals(startRow, HConstants.EMPTY_START_ROW) == false) {
-      Value startVal = IdCodec.decode(startRow);
+    condition = MapRDBImpl.newCondition().and();
 
-      switch(startVal.getType()) {
-      case BINARY:
-        this.condition.is(DocumentConstants.ID_FIELD, Op.GREATER_OR_EQUAL, startVal.getBinary());
-        break;
-      case STRING:
-        this.condition.is(DocumentConstants.ID_FIELD, Op.GREATER_OR_EQUAL, startVal.getString());
-        break;
-      default:
-        throw new IllegalStateException("Encountered an unsupported type " + startVal.getType()
-                                        + " for _id");
-      }
+    if (userCondition != null && !userCondition.isEmpty()) {
+      condition.condition(userCondition);
     }
-
-    if (stopRow != null &&
-        Arrays.equals(stopRow, HConstants.EMPTY_END_ROW) == false) {
-      Value stopVal = IdCodec.decode(stopRow);
-
-      switch(stopVal.getType()) {
-      case BINARY:
-        this.condition.is(DocumentConstants.ID_FIELD, Op.LESS, stopVal.getBinary());
-        break;
-      case STRING:
-        this.condition.is(DocumentConstants.ID_FIELD, Op.LESS, stopVal.getString());
-        break;
-      default:
-        throw new IllegalStateException("Encountered an unsupported type " + stopVal.getType()
-                                        + " for _id");
-      }
+    if (scanRangeCondition != null && !scanRangeCondition.isEmpty()) {
+      condition.condition(scanRangeCondition);
     }
 
-    this.condition.close().build();
+    condition.close().build();
   }
 
   public void setCondition(QueryCondition cond) {
diff --git a/contrib/format-maprdb/src/main/java/org/apache/drill/exec/store/mapr/db/json/JsonTableGroupScan.java b/contrib/format-maprdb/src/main/java/org/apache/drill/exec/store/mapr/db/json/JsonTableGroupScan.java
index c0274e0..a269256 100644
--- a/contrib/format-maprdb/src/main/java/org/apache/drill/exec/store/mapr/db/json/JsonTableGroupScan.java
+++ b/contrib/format-maprdb/src/main/java/org/apache/drill/exec/store/mapr/db/json/JsonTableGroupScan.java
@@ -17,29 +17,52 @@
  */
 package org.apache.drill.exec.store.mapr.db.json;
 
-import static org.apache.drill.exec.store.mapr.db.util.CommonFns.isNullOrEmpty;
+import static org.apache.drill.exec.planner.index.Statistics.ROWCOUNT_HUGE;
+import static org.apache.drill.exec.planner.index.Statistics.ROWCOUNT_UNKNOWN;
+import static org.apache.drill.exec.planner.index.Statistics.AVG_ROWSIZE_UNKNOWN;
 
 import java.io.IOException;
+import java.util.HashMap;
 import java.util.List;
+import java.util.Map;
+import java.util.NavigableMap;
 import java.util.TreeMap;
 
+import com.mapr.db.impl.ConditionImpl;
+import com.mapr.db.impl.ConditionNode.RowkeyRange;
+
+import org.apache.calcite.rel.RelNode;
+import org.apache.calcite.rex.RexNode;
 import org.apache.drill.common.exceptions.DrillRuntimeException;
 import org.apache.drill.common.exceptions.ExecutionSetupException;
+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.GroupScan;
+import org.apache.drill.exec.physical.base.IndexGroupScan;
 import org.apache.drill.exec.physical.base.PhysicalOperator;
 import org.apache.drill.exec.physical.base.ScanStats;
 import org.apache.drill.exec.physical.base.ScanStats.GroupScanProperty;
 import org.apache.drill.exec.store.AbstractStoragePlugin;
+import org.apache.drill.exec.planner.index.Statistics;
+import org.apache.drill.exec.planner.index.MapRDBStatistics;
+import org.apache.drill.exec.planner.cost.PluginCost;
+import org.apache.drill.exec.planner.physical.PartitionFunction;
 import org.apache.drill.exec.store.StoragePluginRegistry;
 import org.apache.drill.exec.store.dfs.FileSystemConfig;
+import org.apache.drill.exec.store.dfs.FileSystemPlugin;
+import org.apache.drill.exec.store.dfs.FileSystemPlugin;
+import org.apache.drill.exec.store.mapr.PluginConstants;
 import org.apache.drill.exec.store.mapr.db.MapRDBFormatPlugin;
 import org.apache.drill.exec.store.mapr.db.MapRDBFormatPluginConfig;
 import org.apache.drill.exec.store.mapr.db.MapRDBGroupScan;
 import org.apache.drill.exec.store.mapr.db.MapRDBSubScan;
+import org.apache.drill.exec.store.mapr.db.MapRDBSubScanSpec;
+import org.apache.drill.exec.store.mapr.db.MapRDBTableStats;
 import org.apache.drill.exec.store.mapr.db.TabletFragmentInfo;
-import org.apache.hadoop.conf.Configuration;
+import org.apache.drill.exec.util.Utilities;
 import org.codehaus.jackson.annotate.JsonCreator;
+import org.ojai.store.QueryCondition;
 
 import com.fasterxml.jackson.annotation.JacksonInject;
 import com.fasterxml.jackson.annotation.JsonIgnore;
@@ -47,21 +70,46 @@ import com.fasterxml.jackson.annotation.JsonProperty;
 import com.fasterxml.jackson.annotation.JsonTypeName;
 import org.apache.drill.shaded.guava.com.google.common.base.Preconditions;
 import com.mapr.db.MapRDB;
+import com.mapr.db.MetaTable;
 import com.mapr.db.Table;
-import com.mapr.db.TabletInfo;
 import com.mapr.db.impl.TabletInfoImpl;
+import com.mapr.db.index.IndexDesc;
+import com.mapr.db.scan.ScanRange;
 
 @JsonTypeName("maprdb-json-scan")
-public class JsonTableGroupScan extends MapRDBGroupScan {
+public class JsonTableGroupScan extends MapRDBGroupScan implements IndexGroupScan {
   static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(JsonTableGroupScan.class);
 
+  public static final int STAR_COLS = 100;
   public static final String TABLE_JSON = "json";
+  /*
+   * The <forcedRowCountMap> maintains a mapping of <RexNode, Rowcount>. These RowCounts take precedence over
+   * anything computed using <MapRDBStatistics> stats. Currently, it is used for picking index plans with the
+   * index_selectivity_factor. We forcibly set the full table rows as HUGE <Statistics.ROWCOUNT_HUGE> in this
+   * map when the selectivity of the index is lower than index_selectivity_factor. During costing, the table
+   * rowCount is returned as HUGE instead of the correct <stats> rowcount. This results in the planner choosing
+   * the cheaper index plans!
+   * NOTE: Full table rowCounts are specified with the NULL condition. e.g. forcedRowCountMap<NULL, 1000>
+   */
+  protected Map<RexNode, Double> forcedRowCountMap;
+  /*
+   * This stores the statistics associated with this GroupScan. Please note that the stats must be initialized
+   * before using it to compute filter row counts based on query conditions.
+   */
+  protected MapRDBStatistics stats;
+  protected JsonScanSpec scanSpec;
+  protected double fullTableRowCount;
+  protected double fullTableEstimatedSize;
 
-  private long totalRowCount;
-  private Table table;
-  private TabletInfo[] tabletInfos;
+  /**
+   * need only read maxRecordsToRead records.
+   */
+  protected int maxRecordsToRead = -1;
 
-  private JsonScanSpec scanSpec;
+  /**
+   * Forced parallelization width
+   */
+  protected int parallelizationWidth = -1;
 
   @JsonCreator
   public JsonTableGroupScan(@JsonProperty("userName") final String userName,
@@ -80,6 +128,18 @@ public class JsonTableGroupScan extends MapRDBGroupScan {
                             MapRDBFormatPlugin formatPlugin, JsonScanSpec scanSpec, List<SchemaPath> columns) {
     super(storagePlugin, formatPlugin, columns, userName);
     this.scanSpec = scanSpec;
+    this.stats = new MapRDBStatistics();
+    this.forcedRowCountMap = new HashMap<>();
+    init();
+  }
+
+  public JsonTableGroupScan(String userName, FileSystemPlugin storagePlugin,
+                            MapRDBFormatPlugin formatPlugin, JsonScanSpec scanSpec, List<SchemaPath> columns,
+                            MapRDBStatistics stats) {
+    super(storagePlugin, formatPlugin, columns, userName);
+    this.scanSpec = scanSpec;
+    this.stats = stats;
+    this.forcedRowCountMap = new HashMap<>();
     init();
   }
 
@@ -87,16 +147,17 @@ public class JsonTableGroupScan extends MapRDBGroupScan {
    * Private constructor, used for cloning.
    * @param that The HBaseGroupScan to clone
    */
-  private JsonTableGroupScan(JsonTableGroupScan that) {
+  protected JsonTableGroupScan(JsonTableGroupScan that) {
     super(that);
     this.scanSpec = that.scanSpec;
     this.endpointFragmentMapping = that.endpointFragmentMapping;
-
-    // Reusing the table handle, tabletInfos and totalRowCount saves expensive
-    // calls to MapR DB client to get them again.
-    this.table = that.table;
-    this.tabletInfos = that.tabletInfos;
-    this.totalRowCount = that.totalRowCount;
+    this.stats = that.stats;
+    this.fullTableRowCount = that.fullTableRowCount;
+    this.fullTableEstimatedSize = that.fullTableEstimatedSize;
+    this.forcedRowCountMap = that.forcedRowCountMap;
+    this.maxRecordsToRead = that.maxRecordsToRead;
+    this.parallelizationWidth = that.parallelizationWidth;
+    init();
   }
 
   @Override
@@ -106,71 +167,122 @@ public class JsonTableGroupScan extends MapRDBGroupScan {
     return newScan;
   }
 
-  /**
-   * Create a new groupScan, which is a clone of this.
-   * Initialize scanSpec.
-   * We should recompute regionsToScan as it depends upon scanSpec.
-   * @param scanSpec
-   */
-  public JsonTableGroupScan clone(JsonScanSpec scanSpec) {
+  public GroupScan clone(JsonScanSpec scanSpec) {
     JsonTableGroupScan newScan = new JsonTableGroupScan(this);
     newScan.scanSpec = scanSpec;
-    newScan.computeRegionsToScan();
+    newScan.resetRegionsToScan(); // resetting will force recalculation
     return newScan;
   }
 
-  /**
-   * Compute regions to scan based on the scanSpec
-   */
-  private void computeRegionsToScan() {
-    boolean foundStartRegion = false;
-
-    regionsToScan = new TreeMap<TabletFragmentInfo, String>();
-    for (TabletInfo tabletInfo : tabletInfos) {
-      TabletInfoImpl tabletInfoImpl = (TabletInfoImpl) tabletInfo;
-      if (!foundStartRegion && !isNullOrEmpty(scanSpec.getStartRow()) && !tabletInfoImpl.containsRow(scanSpec.getStartRow())) {
-        continue;
-      }
-      foundStartRegion = true;
-      regionsToScan.put(new TabletFragmentInfo(tabletInfoImpl), tabletInfo.getLocations()[0]);
-      if (!isNullOrEmpty(scanSpec.getStopRow()) && tabletInfoImpl.containsRow(scanSpec.getStopRow())) {
-        break;
+  private void init() {
+    try {
+      // Get the fullTableRowCount only once i.e. if not already obtained before.
+      if (fullTableRowCount == 0) {
+        final Table t = this.formatPlugin.getJsonTableCache().getTable(
+            scanSpec.getTableName(), scanSpec.getIndexDesc(), getUserName());
+        final MetaTable metaTable = t.getMetaTable();
+        // For condition null, we get full table stats.
+        com.mapr.db.scan.ScanStats stats = metaTable.getScanStats();
+        fullTableRowCount = stats.getEstimatedNumRows();
+        fullTableEstimatedSize = stats.getEstimatedSize();
+        // MapRDB client can return invalid rowCount i.e. 0, especially right after table
+        // creation. It takes 15 minutes before table stats are obtained and cached in client.
+        // If we get 0 rowCount, fallback to getting rowCount using old admin API.
+        if (fullTableRowCount == 0) {
+          PluginCost pluginCostModel = formatPlugin.getPluginCostModel();
+          final int avgColumnSize = pluginCostModel.getAverageColumnSize(this);
+          final int numColumns = (columns == null || columns.isEmpty() || Utilities.isStarQuery(columns)) ? STAR_COLS : columns.size();
+          MapRDBTableStats tableStats = new MapRDBTableStats(formatPlugin.getFsConf(), scanSpec.getTableName());
+          fullTableRowCount = tableStats.getNumRows();
+          fullTableEstimatedSize = fullTableRowCount * numColumns * avgColumnSize;
+        }
       }
+    } catch (Exception e) {
+      throw new DrillRuntimeException("Error getting region info for table: " +
+        scanSpec.getTableName() + (scanSpec.getIndexDesc() == null ? "" : (", index: " + scanSpec.getIndexName())), e);
     }
   }
 
-  private void init() {
-    logger.debug("Getting tablet locations");
-    try {
-      Configuration conf = new Configuration();
+  protected NavigableMap<TabletFragmentInfo, String> getRegionsToScan() {
+    return getRegionsToScan(formatPlugin.getScanRangeSizeMB());
+  }
 
-      // Fetch table and tabletInfo only once and cache.
-      table = MapRDB.getTable(scanSpec.getTableName());
-      tabletInfos = table.getTabletInfos(scanSpec.getCondition());
+  protected NavigableMap<TabletFragmentInfo, String> getRegionsToScan(int scanRangeSizeMB) {
+    // If regionsToScan already computed, just return.
+    double estimatedRowCount = ROWCOUNT_UNKNOWN;
+    if (doNotAccessRegionsToScan == null) {
+      final Table t = this.formatPlugin.getJsonTableCache().getTable(
+          scanSpec.getTableName(), scanSpec.getIndexDesc(), getUserName());
+      final MetaTable metaTable = t.getMetaTable();
 
-      // Calculate totalRowCount for the table from tabletInfos estimatedRowCount.
-      // This will avoid calling expensive MapRDBTableStats API to get total rowCount, avoiding
-      // duplicate work and RPCs to MapR DB server.
-      for (TabletInfo tabletInfo : tabletInfos) {
-        totalRowCount += tabletInfo.getEstimatedNumRows();
+      QueryCondition scanSpecCondition = scanSpec.getCondition();
+      List<ScanRange> scanRanges = (scanSpecCondition == null)
+          ? metaTable.getScanRanges(scanRangeSizeMB)
+          : metaTable.getScanRanges(scanSpecCondition, scanRangeSizeMB);
+      logger.debug("getRegionsToScan() with scanSpec {}: table={}, index={}, condition={}, sizeMB={}, #ScanRanges={}",
+          System.identityHashCode(scanSpec), scanSpec.getTableName(), scanSpec.getIndexName(),
+          scanSpec.getCondition() == null ? "null" : scanSpec.getCondition(), scanRangeSizeMB,
+          scanRanges == null ? "null" : scanRanges.size());
+      final TreeMap<TabletFragmentInfo, String> regionsToScan = new TreeMap<>();
+      if (isIndexScan()) {
+        String idxIdentifier = stats.buildUniqueIndexIdentifier(scanSpec.getIndexDesc().getPrimaryTablePath(),
+            scanSpec.getIndexDesc().getIndexName());
+        if (stats.isStatsAvailable()) {
+          estimatedRowCount = stats.getRowCount(scanSpec.getCondition(), idxIdentifier);
+        }
+      } else {
+        if (stats.isStatsAvailable()) {
+          estimatedRowCount = stats.getRowCount(scanSpec.getCondition(), null);
+        }
       }
+      // If limit pushdown has occurred - factor it in the rowcount
+      if (this.maxRecordsToRead > 0) {
+        estimatedRowCount = Math.min(estimatedRowCount, this.maxRecordsToRead);
+      }
+      // If the estimated row count > 0 then scan ranges must be > 0
+      Preconditions.checkState(estimatedRowCount == ROWCOUNT_UNKNOWN
+          || estimatedRowCount == 0 || (scanRanges != null && scanRanges.size() > 0),
+          String.format("#Scan ranges should be greater than 0 since estimated rowcount=[%f]", estimatedRowCount));
+      if (scanRanges != null && scanRanges.size() > 0) {
+        // set the start-row of the scanspec as the start-row of the first scan range
+        ScanRange firstRange = scanRanges.get(0);
+        QueryCondition firstCondition = firstRange.getCondition();
+        byte[] firstStartRow = ((ConditionImpl) firstCondition).getRowkeyRanges().get(0).getStartRow();
+        scanSpec.setStartRow(firstStartRow);
 
-      computeRegionsToScan();
+        // set the stop-row of ScanSpec as the stop-row of the last scan range
+        ScanRange lastRange = scanRanges.get(scanRanges.size() - 1);
+        QueryCondition lastCondition = lastRange.getCondition();
+        List<RowkeyRange> rowkeyRanges = ((ConditionImpl) lastCondition).getRowkeyRanges();
+        byte[] lastStopRow = rowkeyRanges.get(rowkeyRanges.size() - 1).getStopRow();
+        scanSpec.setStopRow(lastStopRow);
 
-    } catch (Exception e) {
-      throw new DrillRuntimeException("Error getting region info for table: " + scanSpec.getTableName(), e);
+        for (ScanRange range : scanRanges) {
+          TabletInfoImpl tabletInfoImpl = (TabletInfoImpl) range;
+          regionsToScan.put(new TabletFragmentInfo(tabletInfoImpl), range.getLocations()[0]);
+        }
+      }
+      setRegionsToScan(regionsToScan);
     }
+    return doNotAccessRegionsToScan;
   }
 
-  protected JsonSubScanSpec getSubScanSpec(TabletFragmentInfo tfi) {
+  protected MapRDBSubScanSpec getSubScanSpec(final TabletFragmentInfo tfi) {
     // XXX/TODO check filter/Condition
-    JsonScanSpec spec = scanSpec;
+    final JsonScanSpec spec = scanSpec;
+    final String serverHostName = getRegionsToScan().get(tfi);
+    QueryCondition condition = tfi.getTabletInfoImpl().getCondition();
+    byte[] startRow = condition == null ? null : ((ConditionImpl) condition).getRowkeyRanges().get(0).getStartRow();
+    byte[] stopRow = condition == null ? null : ((ConditionImpl) condition).getRowkeyRanges().get(0).getStopRow();
     JsonSubScanSpec subScanSpec = new JsonSubScanSpec(
         spec.getTableName(),
-        regionsToScan.get(tfi),
-        (!isNullOrEmpty(spec.getStartRow()) && tfi.containsRow(spec.getStartRow())) ? spec.getStartRow() : tfi.getStartKey(),
-        (!isNullOrEmpty(spec.getStopRow()) && tfi.containsRow(spec.getStopRow())) ? spec.getStopRow() : tfi.getEndKey(),
-        spec.getCondition());
+        spec.getIndexDesc(),
+        serverHostName,
+        tfi.getTabletInfoImpl().getCondition(),
+        spec.getCondition(),
+        startRow,
+        stopRow,
+        getUserName());
     return subScanSpec;
   }
 
@@ -179,16 +291,72 @@ public class JsonTableGroupScan extends MapRDBGroupScan {
     assert minorFragmentId < endpointFragmentMapping.size() : String.format(
         "Mappings length [%d] should be greater than minor fragment id [%d] but it isn't.", endpointFragmentMapping.size(),
         minorFragmentId);
-    return new MapRDBSubScan(getUserName(), formatPlugin, endpointFragmentMapping.get(minorFragmentId), columns, TABLE_JSON);
+    return new MapRDBSubScan(getUserName(), formatPlugin, endpointFragmentMapping.get(minorFragmentId), columns, maxRecordsToRead, TABLE_JSON);
   }
 
   @Override
   public ScanStats getScanStats() {
-    //TODO: look at stats for this.
-    long rowCount = (long) ((scanSpec.getSerializedFilter() != null ? .5 : 1) * totalRowCount);
-    int avgColumnSize = 10;
-    int numColumns = (columns == null || columns.isEmpty()) ? 100 : columns.size();
-    return new ScanStats(GroupScanProperty.NO_EXACT_ROW_COUNT, rowCount, 1, avgColumnSize * numColumns * rowCount);
+    return fullTableScanStats();
+  }
+
+  private ScanStats fullTableScanStats() {
+    PluginCost pluginCostModel = formatPlugin.getPluginCostModel();
+    final int avgColumnSize = pluginCostModel.getAverageColumnSize(this);
+    final int numColumns = (columns == null || columns.isEmpty()) ? STAR_COLS : columns.size();
+    // index will be NULL for FTS
+    double rowCount = stats.getRowCount(scanSpec.getCondition(), null);
+    // rowcount based on _id predicate. If NO _id predicate present in condition, then the
+    // rowcount should be same as totalRowCount. Equality b/w the two rowcounts should not be
+    // construed as NO _id predicate since stats are approximate.
+    double leadingRowCount = stats.getLeadingRowCount(scanSpec.getCondition(), null);
+    double avgRowSize = stats.getAvgRowSize(null, true);
+    double totalRowCount = stats.getRowCount(null, null);
+    logger.debug("GroupScan {} with stats {}: rowCount={}, condition={}, totalRowCount={}, fullTableRowCount={}",
+            System.identityHashCode(this), System.identityHashCode(stats), rowCount,
+            scanSpec.getCondition()==null?"null":scanSpec.getCondition(),
+            totalRowCount, fullTableRowCount);
+    // If UNKNOWN, or DB stats sync issues(manifests as 0 rows) use defaults.
+    if (rowCount == ROWCOUNT_UNKNOWN || rowCount == 0) {
+      rowCount = (scanSpec.getSerializedFilter() != null ? .5 : 1) * fullTableRowCount;
+    }
+    // If limit pushdown has occurred - factor it in the rowcount
+    if (this.maxRecordsToRead > 0) {
+      rowCount = Math.min(rowCount, this.maxRecordsToRead);
+    }
+    if (totalRowCount == ROWCOUNT_UNKNOWN || totalRowCount == 0) {
+      logger.debug("did not get valid totalRowCount, will take this: {}", fullTableRowCount);
+      totalRowCount = fullTableRowCount;
+    }
+    if (avgRowSize == AVG_ROWSIZE_UNKNOWN || avgRowSize == 0) {
+      avgRowSize = fullTableEstimatedSize/fullTableRowCount;
+    }
+    double totalBlocks = getNumOfBlocks(totalRowCount, fullTableEstimatedSize, avgRowSize, pluginCostModel);
+    double numBlocks = Math.min(totalBlocks, getNumOfBlocks(leadingRowCount, fullTableEstimatedSize, avgRowSize, pluginCostModel));
+    double diskCost = numBlocks * pluginCostModel.getSequentialBlockReadCost(this);
+    /*
+     * Table scan cost made INFINITE in order to pick index plans. Use the MAX possible rowCount for
+     * costing purposes.
+     * NOTE: Full table rowCounts are specified with the NULL condition.
+     * e.g. forcedRowCountMap<NULL, 1000>
+     */
+    if (forcedRowCountMap.get(null) != null && //Forced full table rowcount and it is HUGE
+            forcedRowCountMap.get(null) == ROWCOUNT_HUGE ) {
+      rowCount = ROWCOUNT_HUGE;
+      diskCost = ROWCOUNT_HUGE;
+    }
+
+    logger.debug("JsonGroupScan:{} rowCount:{}, avgRowSize:{}, blocks:{}, totalBlocks:{}, diskCost:{}",
+            this.getOperatorId(), rowCount, avgRowSize, numBlocks, totalBlocks, diskCost);
+    return new ScanStats(GroupScanProperty.NO_EXACT_ROW_COUNT, rowCount, 1, diskCost);
+  }
+
+  private double getNumOfBlocks(double rowCount, double sizeFromDisk,
+                                double avgRowSize, PluginCost pluginCostModel) {
+    if (rowCount == ROWCOUNT_UNKNOWN || rowCount == 0) {
+      return Math.ceil(sizeFromDisk / pluginCostModel.getBlockSize(this));
+    } else {
+      return Math.ceil(rowCount * avgRowSize / pluginCostModel.getBlockSize(this));
+    }
   }
 
   @Override
@@ -198,11 +366,16 @@ public class JsonTableGroupScan extends MapRDBGroupScan {
     return new JsonTableGroupScan(this);
   }
 
+  @Override
   @JsonIgnore
   public String getTableName() {
     return scanSpec.getTableName();
   }
 
+  public IndexDesc getIndexDesc() {
+    return scanSpec.getIndexDesc();
+  }
+
   public boolean isDisablePushdown() {
     return !formatPluginConfig.isEnablePushdown();
   }
@@ -214,11 +387,189 @@ public class JsonTableGroupScan extends MapRDBGroupScan {
 
   @Override
   public String toString() {
-    return "JsonTableGroupScan [ScanSpec=" + scanSpec + ", columns=" + columns + "]";
+    return "JsonTableGroupScan [ScanSpec=" + scanSpec + ", columns=" + columns
+        + (maxRecordsToRead>0? ", limit=" + maxRecordsToRead : "")
+        + (getMaxParallelizationWidth()>0? ", maxwidth=" + getMaxParallelizationWidth() : "") + "]";
   }
 
   public JsonScanSpec getScanSpec() {
     return scanSpec;
   }
 
+  @Override
+  public boolean supportsSecondaryIndex() {
+    return true;
+  }
+
+  @Override
+  @JsonIgnore
+  public boolean isIndexScan() {
+    return scanSpec != null && scanSpec.isSecondaryIndex();
+  }
+
+  @Override
+  public boolean supportsRestrictedScan() {
+    return true;
+  }
+
+  /**
+   * Set the row count resulting from applying the {@link RexNode} condition. Forced row counts will take
+   * precedence over stats row counts
+   * @param condition
+   * @param count
+   * @param capRowCount
+   */
+  @Override
+  @JsonIgnore
+  public void setRowCount(RexNode condition, double count, double capRowCount) {
+    forcedRowCountMap.put(condition, count);
+  }
+
+  @Override
+  public void setStatistics(Statistics statistics) {
+    assert statistics instanceof MapRDBStatistics : String.format(
+        "Passed unexpected statistics instance. Expects MAPR-DB Statistics instance");
+    this.stats = ((MapRDBStatistics) statistics);
+  }
+
+  /**
+   * Get the row count after applying the {@link RexNode} condition
+   * @param condition, filter to apply
+   * @return row count post filtering
+   */
+  @Override
+  @JsonIgnore
+  public double getRowCount(RexNode condition, RelNode scanRel) {
+    // Do not use statistics if row count is forced. Forced rowcounts take precedence over stats
+    double rowcount;
+    if (forcedRowCountMap.get(condition) != null) {
+      return forcedRowCountMap.get(condition);
+    }
+    if (scanSpec.getIndexDesc() != null) {
+      String idxIdentifier = stats.buildUniqueIndexIdentifier(scanSpec.getIndexDesc().getPrimaryTablePath(),
+          scanSpec.getIndexName());
+      rowcount = stats.getRowCount(condition, idxIdentifier, scanRel);
+    } else {
+      rowcount = stats.getRowCount(condition, null, scanRel);
+    }
+    // Stats might NOT have the full rows (e.g. table is newly populated and DB stats APIs return it after
+    // 15 mins). Use the table rows as populated using the (expensive but accurate) Hbase API if needed.
+    if (condition == null && (rowcount == 0 || rowcount == ROWCOUNT_UNKNOWN)) {
+      rowcount = fullTableRowCount;
+      logger.debug("getRowCount: Stats not available yet! Use Admin APIs full table rowcount {}",
+          fullTableRowCount);
+    }
+    return rowcount;
+  }
+
+  @Override
+  public boolean isDistributed() {
+    // getMaxParallelizationWidth gets information about all regions to scan and is expensive.
+    // This option is meant to be used only for unit tests.
+    boolean useNumRegions = storagePlugin.getContext().getConfig().getBoolean(PluginConstants.JSON_TABLE_USE_NUM_REGIONS_FOR_DISTRIBUTION_PLANNING);
+    double fullTableSize;
+
+    if (useNumRegions) {
+      return getMaxParallelizationWidth() > 1 ? true: false;
+    }
+
+    // This function gets called multiple times during planning. To avoid performance
+    // bottleneck, estimate degree of parallelization using stats instead of actually getting information
+    // about all regions.
+    double rowCount, rowSize;
+    double scanRangeSize = storagePlugin.getContext().getConfig().getInt(PluginConstants.JSON_TABLE_SCAN_SIZE_MB) * 1024 * 1024;
+
+    if (scanSpec.getIndexDesc() != null) {
+      String idxIdentifier = stats.buildUniqueIndexIdentifier(scanSpec.getIndexDesc().getPrimaryTablePath(), scanSpec.getIndexName());
+      rowCount = stats.getRowCount(scanSpec.getCondition(), idxIdentifier);
+      rowSize = stats.getAvgRowSize(idxIdentifier, false);
+    } else {
+      rowCount = stats.getRowCount(scanSpec.getCondition(), null);
+      rowSize = stats.getAvgRowSize(null, false);
+    }
+
+    if (rowCount == ROWCOUNT_UNKNOWN || rowCount == 0 ||
+        rowSize == AVG_ROWSIZE_UNKNOWN || rowSize == 0) {
+      fullTableSize = (scanSpec.getSerializedFilter() != null ? .5 : 1) * this.fullTableEstimatedSize;
+    } else {
+      fullTableSize = rowCount * rowSize;
+    }
+
+    return (long) fullTableSize / scanRangeSize > 1 ? true : false;
+  }
+
+  @Override
+  public MapRDBStatistics getStatistics() {
+    return stats;
+  }
+
+  @Override
+  @JsonIgnore
+  public void setColumns(List<SchemaPath> columns) {
+    this.columns = columns;
+  }
+
+  @Override
+  @JsonIgnore
+  public List<SchemaPath> getColumns() {
+    return columns;
+  }
+
+  @Override
+  @JsonIgnore
+  public PartitionFunction getRangePartitionFunction(List<FieldReference> refList) {
+
+    return null;
+    //new JsonTableRangePartitionFunction(refList, scanSpec.getTableName(), this.getUserName(), this.getFormatPlugin());
+  }
+
+  /**
+   * Convert a given {@link LogicalExpression} condition into a {@link QueryCondition} condition
+   * @param condition expressed as a {@link LogicalExpression}
+   * @return {@link QueryCondition} condition equivalent to the given expression
+   */
+  @JsonIgnore
+  public QueryCondition convertToQueryCondition(LogicalExpression condition) {
+    final JsonConditionBuilder jsonConditionBuilder = new JsonConditionBuilder(this, condition);
+    final JsonScanSpec newScanSpec = jsonConditionBuilder.parseTree();
+    if (newScanSpec != null) {
+      return newScanSpec.getCondition();
+    } else {
+      return null;
+    }
+  }
+
+  /**
+   * Json table reader support limit
+   * @return
+   */
+  @Override
+  public boolean supportsLimitPushdown() {
+    if (maxRecordsToRead < 0) {
+      return true;
+    }
+    return false;//limit is already pushed. No more pushdown of limit
+  }
+
+  @Override
+  public GroupScan applyLimit(int maxRecords) {
+    maxRecordsToRead = Math.max(maxRecords, 1);
+    return this;
+  }
+
+  @Override
+  public int getMaxParallelizationWidth() {
+    if (this.parallelizationWidth > 0) {
+      return this.parallelizationWidth;
+    }
+    return super.getMaxParallelizationWidth();
+  }
+
+  @Override
+  public void setParallelizationWidth(int width) {
+    if (width > 0) {
+      this.parallelizationWidth = width;
+      logger.debug("Forced parallelization width = {}", width);
+    }
+  }
 }
diff --git a/contrib/format-maprdb/src/main/java/org/apache/drill/exec/store/mapr/db/json/MaprDBJsonRecordReader.java b/contrib/format-maprdb/src/main/java/org/apache/drill/exec/store/mapr/db/json/MaprDBJsonRecordReader.java
index 628a986..63a9381 100644
--- a/contrib/format-maprdb/src/main/java/org/apache/drill/exec/store/mapr/db/json/MaprDBJsonRecordReader.java
+++ b/contrib/format-maprdb/src/main/java/org/apache/drill/exec/store/mapr/db/json/MaprDBJsonRecordReader.java
@@ -17,18 +17,16 @@
  */
 package org.apache.drill.exec.store.mapr.db.json;
 
-import static org.ojai.DocumentConstants.ID_KEY;
-import static org.ojai.DocumentConstants.ID_FIELD;
-
-import java.nio.ByteBuffer;
-import java.util.Collection;
-import java.util.Iterator;
-import java.util.List;
-import java.util.Set;
-import java.util.Stack;
-import java.util.Collections;
-import java.util.concurrent.TimeUnit;
 
+import com.mapr.db.Table;
+import com.mapr.db.Table.TableOption;
+import com.mapr.db.exceptions.DBException;
+import com.mapr.db.impl.IdCodec;
+import com.mapr.db.impl.MapRDBImpl;
+import com.mapr.db.index.IndexDesc;
+import com.mapr.db.ojai.DBDocumentReaderBase;
+import com.mapr.db.util.ByteBufs;
+import io.netty.buffer.DrillBuf;
 import org.apache.drill.common.exceptions.ExecutionSetupException;
 import org.apache.drill.common.exceptions.UserException;
 import org.apache.drill.common.expression.PathSegment;
@@ -40,47 +38,61 @@ import org.apache.drill.exec.ops.OperatorContext;
 import org.apache.drill.exec.ops.OperatorStats;
 import org.apache.drill.exec.physical.impl.OutputMutator;
 import org.apache.drill.exec.store.AbstractRecordReader;
-import org.apache.drill.exec.store.mapr.db.MapRDBFormatPluginConfig;
+import org.apache.drill.exec.store.mapr.db.MapRDBFormatPlugin;
 import org.apache.drill.exec.store.mapr.db.MapRDBSubScanSpec;
-import org.apache.drill.exec.util.Utilities;
+import org.apache.drill.exec.util.EncodedSchemaPathSet;
 import org.apache.drill.exec.vector.BaseValueVector;
-import org.apache.drill.exec.vector.complex.impl.MapOrListWriterImpl;
 import org.apache.drill.exec.vector.complex.fn.JsonReaderUtils;
 import org.apache.drill.exec.vector.complex.impl.VectorContainerWriter;
+import org.apache.hadoop.fs.Path;
 import org.ojai.DocumentReader;
-import org.ojai.DocumentReader.EventType;
 import org.ojai.DocumentStream;
 import org.ojai.FieldPath;
 import org.ojai.FieldSegment;
-import org.ojai.Value;
 import org.ojai.store.QueryCondition;
+import org.ojai.util.FieldProjector;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
 
 import org.apache.drill.shaded.guava.com.google.common.base.Preconditions;
 import org.apache.drill.shaded.guava.com.google.common.base.Stopwatch;
+import org.apache.drill.shaded.guava.com.google.common.collect.ImmutableSet;
+import org.apache.drill.shaded.guava.com.google.common.collect.Iterables;
 import org.apache.drill.shaded.guava.com.google.common.collect.Sets;
+import org.apache.drill.shaded.guava.com.google.common.base.Predicate;
+
 import com.mapr.db.MapRDB;
-import com.mapr.db.Table;
-import com.mapr.db.Table.TableOption;
-import com.mapr.db.exceptions.DBException;
-import com.mapr.db.impl.IdCodec;
-import com.mapr.db.ojai.DBDocumentReaderBase;
-import com.mapr.db.util.ByteBufs;
 import com.mapr.org.apache.hadoop.hbase.util.Bytes;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Set;
+import java.util.Stack;
+import java.util.concurrent.TimeUnit;
 
-import io.netty.buffer.DrillBuf;
+import static org.apache.drill.exec.store.mapr.PluginConstants.DOCUMENT_SCHEMA_PATH;
+import static org.apache.drill.exec.store.mapr.PluginErrorHandler.dataReadError;
+import static org.ojai.DocumentConstants.ID_FIELD;
 
 public class MaprDBJsonRecordReader extends AbstractRecordReader {
-  private static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(MaprDBJsonRecordReader.class);
+  private static final Logger logger = LoggerFactory.getLogger(MaprDBJsonRecordReader.class);
 
-  private final long MILLISECONDS_IN_A_DAY  = (long)1000 * 60 * 60 * 24;
+  protected static final FieldPath[] ID_ONLY_PROJECTION = { ID_FIELD };
 
-  private Table table;
-  private QueryCondition condition;
-  private FieldPath[] projectedFields;
+  protected Table table;
+  protected QueryCondition condition;
+
+  /**
+   * A set of projected FieldPaths that are pushed into MapR-DB Scanner.
+   * This set is a superset of the fields returned by {@link #getColumns()} when
+   * projection pass-through is in effect. In such cases, {@link #getColumns()}
+   * returns only those fields which are required by Drill to run its operators.
+   */
+  private FieldPath[] scannedFields;
 
-  private final String tableName;
   private OperatorContext operatorContext;
-  private VectorContainerWriter vectorWriter;
+  protected VectorContainerWriter vectorWriter;
   private DBDocumentReaderBase reader;
 
   private DrillBuf buffer;
@@ -91,6 +103,10 @@ public class MaprDBJsonRecordReader extends AbstractRecordReader {
 
   private boolean includeId;
   private boolean idOnly;
+
+  private boolean projectWholeDocument;
+  private FieldProjector projector;
+
   private final boolean unionEnabled;
   private final boolean readNumbersAsDouble;
   private boolean disablePushdown;
@@ -99,15 +115,27 @@ public class MaprDBJsonRecordReader extends AbstractRecordReader {
   private final boolean disableCountOptimization;
   private final boolean nonExistentColumnsProjection;
 
-  public MaprDBJsonRecordReader(MapRDBSubScanSpec subScanSpec,
-      MapRDBFormatPluginConfig formatPluginConfig,
-      List<SchemaPath> projectedColumns, FragmentContext context) {
+  protected final MapRDBSubScanSpec subScanSpec;
+  protected final MapRDBFormatPlugin formatPlugin;
+
+  protected OjaiValueWriter valueWriter;
+  protected DocumentReaderVectorWriter documentWriter;
+  protected int maxRecordsToRead = -1;
+
+  public MaprDBJsonRecordReader(MapRDBSubScanSpec subScanSpec, MapRDBFormatPlugin formatPlugin,
+                                List<SchemaPath> projectedColumns, FragmentContext context, int maxRecords) {
+    this(subScanSpec, formatPlugin, projectedColumns, context);
+    this.maxRecordsToRead = maxRecords;
+  }
+
+  protected MaprDBJsonRecordReader(MapRDBSubScanSpec subScanSpec, MapRDBFormatPlugin formatPlugin,
+                                List<SchemaPath> projectedColumns, FragmentContext context) {
     buffer = context.getManagedBuffer();
-    projectedFields = null;
-    tableName = Preconditions.checkNotNull(subScanSpec, "MapRDB reader needs a sub-scan spec").getTableName();
-    documentReaderIterators = null;
-    includeId = false;
-    idOnly    = false;
+    final Path tablePath = new Path(Preconditions.checkNotNull(subScanSpec,
+      "MapRDB reader needs a sub-scan spec").getTableName());
+    this.subScanSpec = subScanSpec;
+    this.formatPlugin = formatPlugin;
+    final IndexDesc indexDesc = subScanSpec.getIndexDesc();
     byte[] serializedFilter = subScanSpec.getSerializedFilter();
     condition = null;
 
@@ -115,75 +143,153 @@ public class MaprDBJsonRecordReader extends AbstractRecordReader {
       condition = com.mapr.db.impl.ConditionImpl.parseFrom(ByteBufs.wrap(serializedFilter));
     }
 
-    disableCountOptimization = formatPluginConfig.disableCountOptimization();
+    disableCountOptimization = formatPlugin.getConfig().disableCountOptimization();
+    // Below call will set the scannedFields and includeId correctly
     setColumns(projectedColumns);
-    unionEnabled = context.getOptions().getBoolean(ExecConstants.ENABLE_UNION_TYPE_KEY);
-    readNumbersAsDouble = formatPluginConfig.isReadAllNumbersAsDouble();
-    allTextMode = formatPluginConfig.isAllTextMode();
-    ignoreSchemaChange = formatPluginConfig.isIgnoreSchemaChange();
-    disablePushdown = !formatPluginConfig.isEnablePushdown();
-    nonExistentColumnsProjection = formatPluginConfig.isNonExistentFieldSupport();
+    unionEnabled = context.getOptions().getOption(ExecConstants.ENABLE_UNION_TYPE);
+    readNumbersAsDouble = formatPlugin.getConfig().isReadAllNumbersAsDouble();
+    allTextMode = formatPlugin.getConfig().isAllTextMode();
+    ignoreSchemaChange = formatPlugin.getConfig().isIgnoreSchemaChange();
+    disablePushdown = !formatPlugin.getConfig().isEnablePushdown();
+    nonExistentColumnsProjection = formatPlugin.getConfig().isNonExistentFieldSupport();
+
+    // Do not use cached table handle for two reasons.
+    // cached table handles default timeout is 60 min after which those handles will become stale.
+    // Since execution can run for longer than 60 min, we want to get a new table handle and use it
+    // instead of the one from cache.
+    // Since we are setting some table options, we do not want to use shared handles.
+    //
+    // Call it here instead of setup since this will make sure it's called under correct UGI block when impersonation
+    // is enabled and table is used with and without views.
+    table = (indexDesc == null ? MapRDBImpl.getTable(tablePath) : MapRDBImpl.getIndexTable(indexDesc));
+
+    if (condition != null) {
+      logger.debug("Created record reader with query condition {}", condition.toString());
+    } else {
+      logger.debug("Created record reader with query condition NULL");
+    }
   }
 
   @Override
   protected Collection<SchemaPath> transformColumns(Collection<SchemaPath> columns) {
     Set<SchemaPath> transformed = Sets.newLinkedHashSet();
+    Set<SchemaPath> encodedSchemaPathSet = Sets.newLinkedHashSet();
+
     if (disablePushdown) {
       transformed.add(SchemaPath.STAR_COLUMN);
       includeId = true;
-      return transformed;
-    }
+    } else {
+      if (isStarQuery()) {
+        transformed.add(SchemaPath.STAR_COLUMN);
+        includeId = true;
+        if (isSkipQuery() && !disableCountOptimization) {
+          // `SELECT COUNT(*)` query
+          idOnly = true;
+          scannedFields = ID_ONLY_PROJECTION;
+        }
+      } else {
+        Set<FieldPath> scannedFieldsSet = Sets.newTreeSet();
+        Set<FieldPath> projectedFieldsSet = null;
 
-    if (isStarQuery()) {
-      transformed.add(SchemaPath.STAR_COLUMN);
-      includeId = true;
-      if (isSkipQuery()) {
-    	// `SELECT COUNT(*)` query
-    	if (!disableCountOptimization) {
-          projectedFields = new FieldPath[1];
-          projectedFields[0] = ID_FIELD;
+        for (SchemaPath column : columns) {
+          if (EncodedSchemaPathSet.isEncodedSchemaPath(column)) {
+            encodedSchemaPathSet.add(column);
+          } else {
+            transformed.add(column);
+            if (!DOCUMENT_SCHEMA_PATH.equals(column)) {
+              FieldPath fp = getFieldPathForProjection(column);
+              scannedFieldsSet.add(fp);
+            } else {
+              projectWholeDocument = true;
+            }
+          }
+        }
+        if (projectWholeDocument) {
+          // we do not want to project the fields from the encoded field path list
+          // hence make a copy of the scannedFieldsSet here for projection.
+          projectedFieldsSet = new ImmutableSet.Builder<FieldPath>()
+              .addAll(scannedFieldsSet).build();
         }
-      }
-      return transformed;
-    }
 
-    Set<FieldPath> projectedFieldsSet = Sets.newTreeSet();
-    for (SchemaPath column : columns) {
-      if (column.getRootSegment().getPath().equalsIgnoreCase(ID_KEY)) {
-        includeId = true;
-        if (!disableCountOptimization) {
-          projectedFieldsSet.add(ID_FIELD);
+        if (encodedSchemaPathSet.size() > 0) {
+          Collection<SchemaPath> decodedSchemaPaths = EncodedSchemaPathSet.decode(encodedSchemaPathSet);
+          // now we look at the fields which are part of encoded field set and either
+          // add them to scanned set or clear the scanned set if all fields were requested.
+          for (SchemaPath column : decodedSchemaPaths) {
+            if (column.equals(SchemaPath.STAR_COLUMN)) {
+              includeId = true;
+              scannedFieldsSet.clear();
+              break;
+            }
+            scannedFieldsSet.add(getFieldPathForProjection(column));
+          }
         }
-      } else {
-        projectedFieldsSet.add(getFieldPathForProjection(column));
-      }
 
-      transformed.add(column);
-    }
+        if (scannedFieldsSet.size() > 0) {
+          if (includesIdField(scannedFieldsSet)) {
+            includeId = true;
+          }
+          scannedFields = scannedFieldsSet.toArray(new FieldPath[scannedFieldsSet.size()]);
+        }
 
-    if (projectedFieldsSet.size() > 0) {
-      projectedFields = projectedFieldsSet.toArray(new FieldPath[projectedFieldsSet.size()]);
-    }
+        if (disableCountOptimization) {
+          idOnly = (scannedFields == null);
+        }
 
-    if (disableCountOptimization) {
-      idOnly = (projectedFields == null);
-    }
+        if(projectWholeDocument) {
+          projector = new FieldProjector(projectedFieldsSet);
+        }
 
+      }
+    }
     return transformed;
   }
 
+  protected FieldPath[] getScannedFields() {
+    return scannedFields;
+  }
+
+  protected boolean getIdOnly() {
+    return idOnly;
+  }
+
+  protected Table getTable() {
+    return table;
+  }
+
+  protected boolean getIgnoreSchemaChange() {
+    return ignoreSchemaChange;
+  }
+
   @Override
   public void setup(OperatorContext context, OutputMutator output) throws ExecutionSetupException {
     this.vectorWriter = new VectorContainerWriter(output, unionEnabled);
     this.operatorContext = context;
 
     try {
-      table = MapRDB.getTable(tableName);
       table.setOption(TableOption.EXCLUDEID, !includeId);
-      documentStream = table.find(condition, projectedFields);
+      documentStream = table.find(condition, scannedFields);
       documentReaderIterators = documentStream.documentReaders().iterator();
-    } catch (DBException e) {
-      throw new ExecutionSetupException(e);
+
+      if (allTextMode) {
+        valueWriter = new AllTextValueWriter(buffer);
+      } else if (readNumbersAsDouble) {
+        valueWriter = new NumbersAsDoubleValueWriter(buffer);
+      } else {
+        valueWriter = new OjaiValueWriter(buffer);
+      }
+
+      if (projectWholeDocument) {
+        documentWriter = new ProjectionPassthroughVectorWriter(valueWriter, projector, includeId);
+      } else if (isSkipQuery()) {
+        documentWriter = new RowCountVectorWriter(valueWriter);
+      } else if (idOnly) {
+        documentWriter = new IdOnlyVectorWriter(valueWriter);
+      } else {
+        documentWriter = new FieldTransferVectorWriter(valueWriter);
+      }
+    } catch (DBException ex) {
+      throw new ExecutionSetupException(ex);
     }
   }
 
@@ -198,24 +304,17 @@ public class MaprDBJsonRecordReader extends AbstractRecordReader {
     int recordCount = 0;
     reader = null;
 
-    while(recordCount < BaseValueVector.INITIAL_VALUE_ALLOCATION) {
+    int maxRecordsForThisBatch = this.maxRecordsToRead >= 0?
+        Math.min(BaseValueVector.INITIAL_VALUE_ALLOCATION, this.maxRecordsToRead) : BaseValueVector.INITIAL_VALUE_ALLOCATION;
+
+    while(recordCount < maxRecordsForThisBatch) {
       vectorWriter.setPosition(recordCount);
       try {
         reader = nextDocumentReader();
         if (reader == null) {
-          break; // no more documents for this scanner
-        } else if (isSkipQuery()) {
-          vectorWriter.rootAsMap().bit("count").writeBit(1);
+          break; // no more documents for this reader
         } else {
-          MapOrListWriterImpl writer = new MapOrListWriterImpl(vectorWriter.rootAsMap());
-          if (idOnly) {
-            writeId(writer, reader.getId());
-          } else {
-            if (reader.next() != EventType.START_MAP) {
-              throw dataReadError("The document did not start with START_MAP!");
-            }
-            writeToListOrMap(writer, reader);
-          }
+          documentWriter.writeDBDocument(vectorWriter, reader);
         }
         recordCount++;
       } catch (UserException e) {
@@ -225,11 +324,12 @@ public class MaprDBJsonRecordReader extends AbstractRecordReader {
                 reader == null ? null : IdCodec.asString(reader.getId())))
             .build(logger);
       } catch (SchemaChangeException e) {
+        String err_row = reader.getId().asJsonString();
         if (ignoreSchemaChange) {
-          logger.warn("{}. Dropping the row from result.", e.getMessage());
+          logger.warn("{}. Dropping row '{}' from result.", e.getMessage(), err_row);
           logger.debug("Stack trace:", e);
         } else {
-          throw dataReadError(e);
+          throw dataReadError(logger, e, "SchemaChangeException for row '%s'.", err_row);
         }
       }
     }
@@ -238,243 +338,14 @@ public class MaprDBJsonRecordReader extends AbstractRecordReader {
       JsonReaderUtils.ensureAtLeastOneField(vectorWriter, getColumns(), allTextMode, Collections.EMPTY_LIST);
     }
     vectorWriter.setValueCount(recordCount);
+    if (maxRecordsToRead > 0) {
+      maxRecordsToRead -= recordCount;
+    }
     logger.debug("Took {} ms to get {} records", watch.elapsed(TimeUnit.MILLISECONDS), recordCount);
     return recordCount;
   }
 
-  private void writeId(MapOrListWriterImpl writer, Value id) throws SchemaChangeException {
-    try {
-      switch(id.getType()) {
-      case STRING:
-        writeString(writer, ID_KEY, id.getString());
-        break;
-      case BINARY:
-        writeBinary(writer, ID_KEY, id.getBinary());
-        break;
-      default:
-        throw new UnsupportedOperationException(id.getType() +
-            " is not a supported type for _id field.");
-      }
-    } catch (IllegalStateException | IllegalArgumentException e) {
-      throw schemaChangeException(e, "Possible schema change at _id: '%s'", IdCodec.asString(id));
-    }
-  }
-
-  private void writeToListOrMap(MapOrListWriterImpl writer, DBDocumentReaderBase reader) throws SchemaChangeException {
-    String fieldName = null;
-    writer.start();
-    outside: while (true) {
-      EventType event = reader.next();
-      if (event == null
-          || event == EventType.END_MAP
-          || event == EventType.END_ARRAY) {
-        break outside;
-      } else if (reader.inMap()) {
-        fieldName = reader.getFieldName();
-      }
-
-      try {
-        switch (event) {
-        case NULL:
-          break; // not setting the field will leave it as null
-        case BINARY:
-          writeBinary(writer, fieldName, reader.getBinary());
-          break;
-        case BOOLEAN:
-          writeBoolean(writer, fieldName, reader);
-          break;
-        case STRING:
-          writeString(writer, fieldName, reader.getString());
-          break;
-        case BYTE:
-          writeByte(writer, fieldName, reader);
-          break;
-        case SHORT:
-          writeShort(writer, fieldName, reader);
-          break;
-        case INT:
-          writeInt(writer, fieldName, reader);
-          break;
-        case LONG:
-          writeLong(writer, fieldName, reader);
-          break;
-        case FLOAT:
-          writeFloat(writer, fieldName, reader);
-          break;
-        case DOUBLE:
-          writeDouble(writer, fieldName, reader);
-          break;
-        case DECIMAL:
-          writeDecimal(writer, fieldName, reader);
-        case DATE:
-          writeDate(writer, fieldName, reader);
-          break;
-        case TIME:
-          writeTime(writer, fieldName, reader);
-          break;
-        case TIMESTAMP:
-          writeTimeStamp(writer, fieldName, reader);
-          break;
-        case INTERVAL:
-          throw unsupportedError("Interval type is currently not supported.");
-        case START_MAP:
-          writeToListOrMap((MapOrListWriterImpl) (reader.inMap() ? writer.map(fieldName) : writer.listoftmap(fieldName)), reader);
-          break;
-        case START_ARRAY:
-          writeToListOrMap((MapOrListWriterImpl) writer.list(fieldName), reader);
-          break;
-        default:
-          throw unsupportedError("Unsupported type: %s encountered during the query.", event);
-        }
-      } catch (IllegalStateException | IllegalArgumentException e) {
-        throw schemaChangeException(e, "Possible schema change at _id: '%s', field: '%s'", IdCodec.asString(reader.getId()), fieldName);
-      }
-    }
-    writer.end();
-  }
-
-  private void writeTimeStamp(MapOrListWriterImpl writer, String fieldName, DBDocumentReaderBase reader) {
-    if (allTextMode) {
-      writeString(writer, fieldName, reader.getTimestamp().toUTCString());
-    } else {
-      ((writer.map != null) ? writer.map.timeStamp(fieldName) : writer.list.timeStamp()).writeTimeStamp(reader.getTimestampLong());
-    }
-  }
-
-  private void writeTime(MapOrListWriterImpl writer, String fieldName, DBDocumentReaderBase reader) {
-    if (allTextMode) {
-      writeString(writer, fieldName, reader.getTime().toTimeStr());
-    } else {
-      ((writer.map != null) ? writer.map.time(fieldName) : writer.list.time()).writeTime(reader.getTimeInt());
-    }
-  }
-
-  private void writeDate(MapOrListWriterImpl writer, String fieldName, DBDocumentReaderBase reader) {
-    if (allTextMode) {
-      writeString(writer, fieldName, reader.getDate().toDateStr());
-    } else {
-      long milliSecondsSinceEpoch = reader.getDateInt() * MILLISECONDS_IN_A_DAY;
-      ((writer.map != null) ? writer.map.date(fieldName) : writer.list.date()).writeDate(milliSecondsSinceEpoch);
-    }
-  }
-
-  private void writeDouble(MapOrListWriterImpl writer, String fieldName, DBDocumentReaderBase reader) {
-    if (allTextMode) {
-      writeString(writer, fieldName, String.valueOf(reader.getDouble()));
-    } else {
-      writer.float8(fieldName).writeFloat8(reader.getDouble());
-    }
-  }
-
-  private void writeDecimal(MapOrListWriterImpl writer, String fieldName, DBDocumentReaderBase reader) {
-    if (allTextMode) {
-      writeString(writer, fieldName, String.valueOf(reader.getDecimal()));
-    } else {
-      writer.varDecimal(fieldName, reader.getDecimalScale(), reader.getDecimalPrecision())
-          .writeVarDecimal(reader.getDecimal());
-    }
-  }
-
-  private void writeFloat(MapOrListWriterImpl writer, String fieldName, DBDocumentReaderBase reader) {
-    if (allTextMode) {
-      writeString(writer, fieldName, String.valueOf(reader.getFloat()));
-    } else if (readNumbersAsDouble) {
-      writer.float8(fieldName).writeFloat8(reader.getFloat());
-    } else {
-      writer.float4(fieldName).writeFloat4(reader.getFloat());
-    }
-  }
-
-  private void writeLong(MapOrListWriterImpl writer, String fieldName, DBDocumentReaderBase reader) {
-    if (allTextMode) {
-      writeString(writer, fieldName, String.valueOf(reader.getLong()));
-    } else if (readNumbersAsDouble) {
-      writer.float8(fieldName).writeFloat8(reader.getLong());
-    } else {
-      writer.bigInt(fieldName).writeBigInt(reader.getLong());
-    }
-  }
-
-  private void writeInt(MapOrListWriterImpl writer, String fieldName, DBDocumentReaderBase reader) {
-    if (allTextMode) {
-      writeString(writer, fieldName, String.valueOf(reader.getInt()));
-    } else if (readNumbersAsDouble) {
-      writer.float8(fieldName).writeFloat8(reader.getInt());
-    } else {
-      writer.integer(fieldName).writeInt(reader.getInt());
-    }
-  }
-
-  private void writeShort(MapOrListWriterImpl writer, String fieldName, DBDocumentReaderBase reader) {
-    if (allTextMode) {
-      writeString(writer, fieldName, String.valueOf(reader.getShort()));
-    } else if (readNumbersAsDouble) {
-      writer.float8(fieldName).writeFloat8(reader.getShort());
-    } else {
-      ((writer.map != null) ? writer.map.smallInt(fieldName) : writer.list.smallInt()).writeSmallInt(reader.getShort());
-    }
-  }
-
-  private void writeByte(MapOrListWriterImpl writer, String fieldName, DBDocumentReaderBase reader) {
-    if (allTextMode) {
-      writeString(writer, fieldName, String.valueOf(reader.getByte()));
-    } else if (readNumbersAsDouble) {
-      writer.float8(fieldName).writeFloat8(reader.getByte());
-    } else {
-      ((writer.map != null) ? writer.map.tinyInt(fieldName) : writer.list.tinyInt()).writeTinyInt(reader.getByte());
-    }
-  }
-
-  private void writeBoolean(MapOrListWriterImpl writer, String fieldName, DBDocumentReaderBase reader) {
-    if (allTextMode) {
-      writeString(writer, fieldName, String.valueOf(reader.getBoolean()));
-    } else {
-      writer.bit(fieldName).writeBit(reader.getBoolean() ? 1 : 0);
-    }
-  }
-
-  private void writeBinary(MapOrListWriterImpl writer, String fieldName, ByteBuffer buf) {
-    if (allTextMode) {
-      writeString(writer, fieldName, Bytes.toString(buf));
-    } else {
-      buffer = buffer.reallocIfNeeded(buf.remaining());
-      buffer.setBytes(0, buf, buf.position(), buf.remaining());
-      writer.binary(fieldName).writeVarBinary(0, buf.remaining(), buffer);
-    }
-  }
-
-  private void writeString(MapOrListWriterImpl writer, String fieldName, String value) {
-    final byte[] strBytes = Bytes.toBytes(value);
-    buffer = buffer.reallocIfNeeded(strBytes.length);
-    buffer.setBytes(0, strBytes);
-    writer.varChar(fieldName).writeVarChar(0, strBytes.length, buffer);
-  }
-
-  private UserException unsupportedError(String format, Object... args) {
-    return UserException.unsupportedError()
-        .message(String.format(format, args))
-        .build(logger);
-  }
-
-  private UserException dataReadError(Throwable t) {
-    return dataReadError(t, null);
-  }
-
-  private UserException dataReadError(String format, Object... args) {
-    return dataReadError(null, format, args);
-  }
-
-  private UserException dataReadError(Throwable t, String format, Object... args) {
-    return UserException.dataReadError(t)
-        .message(format == null ? null : String.format(format, args))
-        .build(logger);
-  }
-
-  private SchemaChangeException schemaChangeException(Throwable t, String format, Object... args) {
-    return new SchemaChangeException(format, t, args);
-  }
-
-  private DBDocumentReaderBase nextDocumentReader() {
+  protected DBDocumentReaderBase nextDocumentReader() {
     final OperatorStats operatorStats = operatorContext == null ? null : operatorContext.getStats();
     try {
       if (operatorStats != null) {
@@ -492,7 +363,7 @@ public class MaprDBJsonRecordReader extends AbstractRecordReader {
         }
       }
     } catch (DBException e) {
-      throw dataReadError(e);
+      throw dataReadError(logger, e);
     }
   }
 
@@ -505,7 +376,7 @@ public class MaprDBJsonRecordReader extends AbstractRecordReader {
    * first encountered ARRAY field and let Drill handle the projection.
    */
   private static FieldPath getFieldPathForProjection(SchemaPath column) {
-    Stack<PathSegment.NameSegment> pathSegments = new Stack<PathSegment.NameSegment>();
+    Stack<PathSegment.NameSegment> pathSegments = new Stack<>();
     PathSegment seg = column.getRootSegment();
     while (seg != null && seg.isNamed()) {
       pathSegments.push((PathSegment.NameSegment) seg);
@@ -518,6 +389,15 @@ public class MaprDBJsonRecordReader extends AbstractRecordReader {
     return new FieldPath(child);
   }
 
+  public static boolean includesIdField(Collection<FieldPath> projected) {
+    return Iterables.tryFind(projected, new Predicate<FieldPath>() {
+      @Override
+      public boolean apply(FieldPath path) {
+        return Preconditions.checkNotNull(path).equals(ID_FIELD);
+      }
+    }).isPresent();
+  }
+
   @Override
   public void close() {
     if (documentStream != null) {
diff --git a/contrib/format-maprdb/src/main/java/org/apache/drill/exec/store/mapr/db/json/NumbersAsDoubleValueWriter.java b/contrib/format-maprdb/src/main/java/org/apache/drill/exec/store/mapr/db/json/NumbersAsDoubleValueWriter.java
new file mode 100644
index 0000000..d7d38cb
--- /dev/null
+++ b/contrib/format-maprdb/src/main/java/org/apache/drill/exec/store/mapr/db/json/NumbersAsDoubleValueWriter.java
@@ -0,0 +1,51 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.drill.exec.store.mapr.db.json;
+
+import org.apache.drill.exec.vector.complex.impl.MapOrListWriterImpl;
+import org.ojai.DocumentReader;
+
+import io.netty.buffer.DrillBuf;
+
+public class NumbersAsDoubleValueWriter extends OjaiValueWriter {
+
+  public NumbersAsDoubleValueWriter(DrillBuf buffer) {
+    super(buffer);
+  }
+
+  protected void writeFloat(MapOrListWriterImpl writer, String fieldName, DocumentReader reader) {
+    writer.float8(fieldName).writeFloat8(reader.getFloat());
+  }
+
+  protected void writeLong(MapOrListWriterImpl writer, String fieldName, DocumentReader reader) {
+      writer.float8(fieldName).writeFloat8(reader.getLong());
+  }
+
+  protected void writeInt(MapOrListWriterImpl writer, String fieldName, DocumentReader reader) {
+      writer.float8(fieldName).writeFloat8(reader.getInt());
+  }
+
+  protected void writeShort(MapOrListWriterImpl writer, String fieldName, DocumentReader reader) {
+      writer.float8(fieldName).writeFloat8(reader.getShort());
+  }
+
+  protected void writeByte(MapOrListWriterImpl writer, String fieldName, DocumentReader reader) {
+      writer.float8(fieldName).writeFloat8(reader.getByte());
+  }
+
+}
diff --git a/contrib/format-maprdb/src/main/java/org/apache/drill/exec/store/mapr/db/json/OjaiFunctionsProcessor.java b/contrib/format-maprdb/src/main/java/org/apache/drill/exec/store/mapr/db/json/OjaiFunctionsProcessor.java
new file mode 100644
index 0000000..959e243
--- /dev/null
+++ b/contrib/format-maprdb/src/main/java/org/apache/drill/exec/store/mapr/db/json/OjaiFunctionsProcessor.java
@@ -0,0 +1,214 @@
+/*
+ * 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.store.mapr.db.json;
+
+import org.apache.commons.codec.binary.Base64;
+
+import org.apache.drill.common.expression.FunctionCall;
+import org.apache.drill.common.expression.LogicalExpression;
+import org.apache.drill.common.expression.SchemaPath;
+import org.apache.drill.common.expression.ValueExpressions.IntExpression;
+import org.apache.drill.common.expression.ValueExpressions.LongExpression;
+import org.apache.drill.common.expression.ValueExpressions.QuotedString;
+import org.apache.drill.common.expression.visitors.AbstractExprVisitor;
+
+import org.ojai.Value;
+import org.ojai.store.QueryCondition;
+
+import com.google.common.collect.ImmutableMap;
+import com.mapr.db.impl.ConditionImpl;
+import com.mapr.db.impl.MapRDBImpl;
+
+import java.nio.ByteBuffer;
+
+class OjaiFunctionsProcessor extends AbstractExprVisitor<Void, Void, RuntimeException> {
+  private static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(OjaiFunctionsProcessor.class);
+  private QueryCondition queryCond;
+
+  private OjaiFunctionsProcessor() {
+  }
+
+  private static String getStackTrace() {
+    final Throwable throwable = new Throwable();
+    final StackTraceElement[] ste = throwable.getStackTrace();
+    final StringBuilder sb = new StringBuilder();
+    for(int i = 1; i < ste.length; ++i) {
+      sb.append(ste[i].toString());
+      sb.append('\n');
+    }
+
+    return sb.toString();
+  }
+
+  @Override
+  public Void visitUnknown(LogicalExpression e, Void valueArg) throws RuntimeException {
+    logger.debug("visitUnknown() e class " + e.getClass());
+    logger.debug(getStackTrace());
+    return null;
+  }
+
+  private static class Ref<T> {
+    T value;
+  }
+
+  private static SchemaPath getSchemaPathArg(LogicalExpression expr) {
+    final Ref<SchemaPath> ref = new Ref<>();
+    expr.accept(new OjaiFunctionsProcessor() {
+        @Override
+        public Void visitSchemaPath(SchemaPath e, Void v) {
+          ref.value = e;
+          return null;
+        }
+     }, null);
+
+      return ref.value;
+  }
+
+  private static String getStringArg(LogicalExpression expr) {
+    final Ref<QuotedString> ref = new Ref<>();
+    expr.accept(new OjaiFunctionsProcessor() {
+        @Override
+        public Void visitQuotedStringConstant(QuotedString e, Void v) {
+          ref.value = e;
+          return null;
+        }
+     }, null);
+
+      return ref.value != null ? ref.value.getString() : null;
+  }
+
+  private static int getIntArg(LogicalExpression expr) {
+    final Ref<Integer> ref = new Ref<>();
+    expr.accept(new OjaiFunctionsProcessor() {
+        @Override
+        public Void visitIntConstant(IntExpression e, Void v) {
+          ref.value = new Integer(e.getInt());
+          return null;
+        }
+     }, null);
+
+      return ref.value != null ? ref.value.intValue() : 0;
+  }
+
+  private static long getLongArg(LogicalExpression expr) {
+    final Ref<Long> ref = new Ref<>();
+    expr.accept(new OjaiFunctionsProcessor() {
+        @Override
+        public Void visitIntConstant(IntExpression e, Void v) {
+          ref.value = new Long(e.getInt());
+          return null;
+        }
+
+        @Override
+        public Void visitLongConstant(LongExpression e, Void v) {
+          ref.value = e.getLong();
+          return null;
+        }
+     }, null);
+
+      return ref.value != null ? ref.value.longValue() : 0;
+  }
+
+  private final static ImmutableMap<String, QueryCondition.Op> STRING_TO_RELOP;
+  static {
+    ImmutableMap.Builder<String, QueryCondition.Op> builder = ImmutableMap.builder();
+    STRING_TO_RELOP = builder
+        .put("=", QueryCondition.Op.EQUAL)
+        .put("<>", QueryCondition.Op.NOT_EQUAL)
+        .put("<", QueryCondition.Op.LESS)
+        .put("<=", QueryCondition.Op.LESS_OR_EQUAL)
+        .put(">", QueryCondition.Op.GREATER)
+        .put(">=", QueryCondition.Op.GREATER_OR_EQUAL)
+        .build();
+  }
+
+  @Override
+  public Void visitFunctionCall(FunctionCall call, Void v) throws RuntimeException {
+    final String functionName = call.getName();
+    final String fieldName = FieldPathHelper.schemaPath2FieldPath(getSchemaPathArg(call.args.get(0))).asPathString();
+    switch(functionName) {
+    case "ojai_sizeof": {
+      // ojai_sizeof(field, "<rel-op>", <int-value>)
+      final String relOp = getStringArg(call.args.get(1));
+      final long size = getLongArg(call.args.get(2));
+      queryCond = MapRDBImpl.newCondition()
+          .sizeOf(fieldName, STRING_TO_RELOP.get(relOp), size)
+          .build();
+      break;
+    }
+
+    case "ojai_typeof":
+    case "ojai_nottypeof": {
+      // ojai_[not]typeof(field, <type-code>);
+      final int typeCode = getIntArg(call.args.get(1));
+      final Value.Type typeValue = Value.Type.valueOf(typeCode);
+      queryCond = MapRDBImpl.newCondition();
+      if (functionName.equals("ojai_typeof")) {
+        queryCond.typeOf(fieldName, typeValue);
+      } else {
+        queryCond.notTypeOf(fieldName, typeValue);
+      }
+      queryCond.build();
+      break;
+    }
+
+    case "ojai_matches":
+    case "ojai_notmatches": {
+      // ojai_[not]matches(field, <regex>);
+      final SchemaPath schemaPath = getSchemaPathArg(call.args.get(0));
+      final String regex = getStringArg(call.args.get(1));
+      if (functionName.equals("ojai_matches")) {
+        queryCond = MapRDBImpl.newCondition()
+            .matches(fieldName, regex);
+      } else {
+        queryCond = MapRDBImpl.newCondition()
+            .notMatches(fieldName, regex);
+      }
+      queryCond.build();
+      break;
+    }
+
+    case "ojai_condition": {
+      // ojai_condition(field, <serialized-condition>);
+      final SchemaPath schemaPath = getSchemaPathArg(call.args.get(0));
+      final String condString = getStringArg(call.args.get(1));
+      final byte[] condBytes = Base64.decodeBase64(condString);
+      final ByteBuffer condBuffer = ByteBuffer.wrap(condBytes);
+      queryCond = ConditionImpl.parseFrom(condBuffer);
+      break;
+    }
+
+    default:
+      throw new IllegalArgumentException("unrecognized functionName " + functionName);
+    } // switch(functionName)
+
+    return null;
+  }
+
+  public static OjaiFunctionsProcessor process(FunctionCall call) {
+    final OjaiFunctionsProcessor processor = new OjaiFunctionsProcessor();
+
+    call.accept(processor, null);
+    return processor;
+  }
+
+  public QueryCondition getCondition() {
+    return queryCond;
+  }
+
+}
diff --git a/contrib/format-maprdb/src/main/java/org/apache/drill/exec/store/mapr/db/json/OjaiValueWriter.java b/contrib/format-maprdb/src/main/java/org/apache/drill/exec/store/mapr/db/json/OjaiValueWriter.java
new file mode 100644
index 0000000..99b2157
--- /dev/null
+++ b/contrib/format-maprdb/src/main/java/org/apache/drill/exec/store/mapr/db/json/OjaiValueWriter.java
@@ -0,0 +1,194 @@
+/*
+ * 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.store.mapr.db.json;
+
+import static org.apache.drill.exec.store.mapr.PluginErrorHandler.schemaChangeException;
+import static org.apache.drill.exec.store.mapr.PluginErrorHandler.unsupportedError;
+
+import java.nio.ByteBuffer;
+
+import org.apache.drill.exec.exception.SchemaChangeException;
+import org.apache.drill.exec.vector.complex.impl.MapOrListWriterImpl;
+import org.apache.drill.exec.vector.complex.writer.BaseWriter.MapOrListWriter;
+import org.apache.drill.exec.vector.complex.writer.BaseWriter.MapWriter;
+import org.ojai.DocumentReader;
+import org.ojai.DocumentReader.EventType;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import com.mapr.org.apache.hadoop.hbase.util.Bytes;
+
+import io.netty.buffer.DrillBuf;
+
+public class OjaiValueWriter {
+  protected static final Logger logger = LoggerFactory.getLogger(OjaiValueWriter.class);
+
+  protected static final long MILLISECONDS_IN_A_DAY  = (long)1000 * 60 * 60 * 24;
+
+  protected DrillBuf buffer;
+
+  public OjaiValueWriter(DrillBuf buffer) {
+    this.buffer = buffer;
+  }
+
+  /*
+   * Precondition to call this function is that the DBDocumentReader has already emitted START_MAP/START_ARRAY event.
+   */
+  protected void writeToListOrMap(MapOrListWriterImpl writer, DocumentReader reader) throws SchemaChangeException {
+    String fieldName = null;
+    writer.start();
+    outside: while (true) {
+      EventType event = reader.next();
+      if (event == null
+          || event == EventType.END_MAP
+          || event == EventType.END_ARRAY) {
+        break outside;
+      } else if (reader.inMap()) {
+        fieldName = reader.getFieldName();
+      }
+
+      try {
+        switch (event) {
+        case NULL:
+          break; // not setting the field will leave it as null
+        case BINARY:
+          writeBinary(writer, fieldName, reader.getBinary());
+          break;
+        case BOOLEAN:
+          writeBoolean(writer, fieldName, reader);
+          break;
+        case STRING:
+          writeString(writer, fieldName, reader.getString());
+          break;
+        case BYTE:
+          writeByte(writer, fieldName, reader);
+          break;
+        case SHORT:
+          writeShort(writer, fieldName, reader);
+          break;
+        case INT:
+          writeInt(writer, fieldName, reader);
+          break;
+        case LONG:
+          writeLong(writer, fieldName, reader);
+          break;
+        case FLOAT:
+          writeFloat(writer, fieldName, reader);
+          break;
+        case DOUBLE:
+          writeDouble(writer, fieldName, reader);
+          break;
+        case DECIMAL:
+          throw unsupportedError(logger, "Decimal type is currently not supported.");
+        case DATE:
+          writeDate(writer, fieldName, reader);
+          break;
+        case TIME:
+          writeTime(writer, fieldName, reader);
+          break;
+        case TIMESTAMP:
+          writeTimeStamp(writer, fieldName, reader);
+          break;
+        case INTERVAL:
+          throw unsupportedError(logger, "Interval type is currently not supported.");
+        case START_MAP:
+          writeToListOrMap((MapOrListWriterImpl) (reader.inMap() ? writer.map(fieldName) : writer.listoftmap(fieldName)), reader);
+          break;
+        case START_ARRAY:
+          writeToListOrMap((MapOrListWriterImpl) writer.list(fieldName), reader);
+          break;
+        default:
+          throw unsupportedError(logger, "Unsupported type: %s encountered during the query.", event);
+        }
+      } catch (IllegalStateException | IllegalArgumentException e) {
+        throw schemaChangeException(logger, e, "Possible schema change for field: '%s'", fieldName);
+      }
+    }
+    writer.end();
+  }
+
+  protected void writeTimeStamp(MapOrListWriterImpl writer, String fieldName, DocumentReader reader) {
+    writer.timeStamp(fieldName).writeTimeStamp(reader.getTimestampLong());
+  }
+
+  protected void writeTime(MapOrListWriterImpl writer, String fieldName, DocumentReader reader) {
+    writer.time(fieldName).writeTime(reader.getTimeInt());
+  }
+
+  protected void writeDate(MapOrListWriterImpl writer, String fieldName, DocumentReader reader) {
+    long milliSecondsSinceEpoch = reader.getDateInt() * MILLISECONDS_IN_A_DAY;
+    writer.date(fieldName).writeDate(milliSecondsSinceEpoch);
+  }
+
+  protected void writeDouble(MapOrListWriterImpl writer, String fieldName, DocumentReader reader) {
+    writer.float8(fieldName).writeFloat8(reader.getDouble());
+  }
+
+  protected void writeFloat(MapOrListWriterImpl writer, String fieldName, DocumentReader reader) {
+    writer.float4(fieldName).writeFloat4(reader.getFloat());
+  }
+
+  protected void writeLong(MapOrListWriterImpl writer, String fieldName, DocumentReader reader) {
+    writer.bigInt(fieldName).writeBigInt(reader.getLong());
+  }
+
+  protected void writeInt(MapOrListWriterImpl writer, String fieldName, DocumentReader reader) {
+    writer.integer(fieldName).writeInt(reader.getInt());
+  }
+
+  protected void writeShort(MapOrListWriterImpl writer, String fieldName, DocumentReader reader) {
+    writer.smallInt(fieldName).writeSmallInt(reader.getShort());
+  }
+
+  protected void writeByte(MapOrListWriterImpl writer, String fieldName, DocumentReader reader) {
+    writer.tinyInt(fieldName).writeTinyInt(reader.getByte());
+  }
+
+  protected void writeBoolean(MapOrListWriterImpl writer, String fieldName, DocumentReader reader) {
+    writer.bit(fieldName).writeBit(reader.getBoolean() ? 1 : 0);
+  }
+
+  protected void writeBinary(MapOrListWriter writer, String fieldName, ByteBuffer buf) {
+    int bufLen = buf.remaining();
+    buffer = buffer.reallocIfNeeded(bufLen);
+    buffer.setBytes(0, buf, buf.position(), bufLen);
+    writer.varBinary(fieldName).writeVarBinary(0, bufLen, buffer);
+  }
+
+  protected void writeString(MapOrListWriter writer, String fieldName, String value) {
+    final byte[] strBytes = Bytes.toBytes(value);
+    buffer = buffer.reallocIfNeeded(strBytes.length);
+    buffer.setBytes(0, strBytes);
+    writer.varChar(fieldName).writeVarChar(0, strBytes.length, buffer);
+  }
+  
+  protected void writeBinary(MapWriter writer, String fieldName, ByteBuffer buf) {
+    int bufLen = buf.remaining();
+    buffer = buffer.reallocIfNeeded(bufLen);
+    buffer.setBytes(0, buf, buf.position(), bufLen);
+    writer.varBinary(fieldName).writeVarBinary(0, bufLen, buffer);
+  }
+  
+  protected void writeString(MapWriter writer, String fieldName, String value) {
+    final byte[] strBytes = Bytes.toBytes(value);
+    buffer = buffer.reallocIfNeeded(strBytes.length);
+    buffer.setBytes(0, strBytes);
+    writer.varChar(fieldName).writeVarChar(0, strBytes.length, buffer);
+  }
+
+}
diff --git a/contrib/format-maprdb/src/main/java/org/apache/drill/exec/store/mapr/db/json/ProjectionPassthroughVectorWriter.java b/contrib/format-maprdb/src/main/java/org/apache/drill/exec/store/mapr/db/json/ProjectionPassthroughVectorWriter.java
new file mode 100644
index 0000000..bc304fd
--- /dev/null
+++ b/contrib/format-maprdb/src/main/java/org/apache/drill/exec/store/mapr/db/json/ProjectionPassthroughVectorWriter.java
@@ -0,0 +1,83 @@
+/*
+ * 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.store.mapr.db.json;
+
+import static org.apache.drill.exec.store.mapr.PluginErrorHandler.dataReadError;
+
+import java.nio.ByteBuffer;
+import java.util.Map;
+import java.util.Map.Entry;
+
+import org.apache.drill.exec.exception.SchemaChangeException;
+import org.apache.drill.exec.vector.complex.impl.MapOrListWriterImpl;
+import org.apache.drill.exec.vector.complex.impl.VectorContainerWriter;
+import org.apache.drill.exec.vector.complex.writer.BaseWriter.MapOrListWriter;
+import org.ojai.DocumentConstants;
+import org.ojai.DocumentReader.EventType;
+import org.ojai.util.DocumentReaderWithProjection;
+import org.ojai.util.FieldProjector;
+
+import com.google.common.base.Preconditions;
+import com.mapr.db.DBConstants;
+import com.mapr.db.ojai.DBDocumentReaderBase;
+
+/**
+ *  This implementation of DocumentReaderVectorWriter writes the encoded MapR-DB OJAI Document
+ *  as binary data along with the fields required to execute Drill's operators.
+ */
+class ProjectionPassthroughVectorWriter extends DocumentReaderVectorWriter {
+
+  private final boolean includeId;
+  private final FieldProjector projector;
+
+  protected ProjectionPassthroughVectorWriter(final OjaiValueWriter valueWriter,
+      final FieldProjector projector, final boolean includeId) {
+    super(valueWriter);
+    this.includeId = includeId;
+    this.projector = Preconditions.checkNotNull(projector);
+  }
+
+  @Override
+  protected void writeDBDocument(VectorContainerWriter vectorWriter, DBDocumentReaderBase reader)
+      throws SchemaChangeException {
+    if (reader.next() != EventType.START_MAP) {
+      throw dataReadError(logger, "The document did not start with START_MAP!");
+    }
+
+    MapOrListWriterImpl writer = new MapOrListWriterImpl(vectorWriter.rootAsMap());
+    writer.start();
+    MapOrListWriter documentMapWriter = writer.map(DBConstants.DOCUMENT_FIELD);
+    documentMapWriter.start();
+
+    // write _id field data
+    if (includeId) {
+      valueWriter.writeBinary(documentMapWriter, DocumentConstants.ID_KEY, reader.getIdData());
+    }
+
+    // write rest of the data buffers
+    Map<Integer, ByteBuffer> dataMap = reader.getDataMap();
+    for (Entry<Integer, ByteBuffer> familyData : dataMap.entrySet()) {
+      valueWriter.writeBinary(documentMapWriter, String.valueOf(familyData.getKey()), familyData.getValue());
+    }
+    documentMapWriter.end();
+
+    DocumentReaderWithProjection p = new DocumentReaderWithProjection(reader, projector);
+    valueWriter.writeToListOrMap(writer, p);
+  }
+
+}
diff --git a/contrib/format-maprdb/src/main/java/org/apache/drill/exec/store/mapr/db/json/RestrictedJsonRecordReader.java b/contrib/format-maprdb/src/main/java/org/apache/drill/exec/store/mapr/db/json/RestrictedJsonRecordReader.java
new file mode 100644
index 0000000..01ff2d0
--- /dev/null
+++ b/contrib/format-maprdb/src/main/java/org/apache/drill/exec/store/mapr/db/json/RestrictedJsonRecordReader.java
@@ -0,0 +1,248 @@
+/*
+ * 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.store.mapr.db.json;
+
+import static org.apache.drill.exec.store.mapr.PluginErrorHandler.dataReadError;
+
+import java.nio.ByteBuffer;
+import java.util.List;
+import java.util.concurrent.TimeUnit;
+
+import com.mapr.db.impl.BaseJsonTable;
+import com.mapr.db.impl.MultiGet;
+
+import org.apache.drill.common.exceptions.ExecutionSetupException;
+import org.apache.drill.common.exceptions.UserException;
+import org.apache.drill.common.expression.SchemaPath;
+import org.apache.drill.exec.ExecConstants;
+import org.apache.drill.exec.exception.SchemaChangeException;
+import org.apache.drill.exec.ops.FragmentContext;
+import org.apache.drill.exec.ops.OperatorContext;
+import org.apache.drill.exec.physical.impl.OutputMutator;
+import org.apache.drill.exec.physical.impl.join.RowKeyJoin;
+import org.apache.drill.exec.record.AbstractRecordBatch;
+import org.apache.drill.exec.store.mapr.db.MapRDBFormatPlugin;
+import org.apache.drill.exec.store.mapr.db.MapRDBSubScanSpec;
+import org.apache.drill.exec.store.mapr.db.RestrictedMapRDBSubScanSpec;
+
+import com.google.common.base.Stopwatch;
+import com.mapr.db.impl.IdCodec;
+import com.mapr.db.ojai.DBDocumentReaderBase;
+
+import org.ojai.Document;
+import org.ojai.DocumentStream;
+import org.ojai.FieldPath;
+
+
+public class RestrictedJsonRecordReader extends MaprDBJsonRecordReader {
+  private static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(RestrictedJsonRecordReader.class);
+
+  private int batchSize; // batchSize for rowKey based document get
+
+  private String [] projections = null; // multiGet projections
+  public RestrictedJsonRecordReader(MapRDBSubScanSpec subScanSpec,
+                                    MapRDBFormatPlugin formatPlugin,
+                                    List<SchemaPath> projectedColumns, FragmentContext context) {
+
+    super(subScanSpec, formatPlugin, projectedColumns, context);
+    batchSize = (int)context.getOptions().getOption(ExecConstants.QUERY_ROWKEYJOIN_BATCHSIZE);
+    int idx = 0;
+    FieldPath[] scannedFields = this.getScannedFields();
+
+    // only populate projections for non-star query (for star, null is interpreted as all fields)
+    if (!this.isStarQuery() && scannedFields != null && scannedFields.length > 0) {
+      projections = new String[scannedFields.length];
+      for (FieldPath path : scannedFields) {
+        projections[idx] = path.asPathString();
+        ++idx;
+      }
+    }
+  }
+
+  public RestrictedJsonRecordReader(MapRDBSubScanSpec subScanSpec,
+                                    MapRDBFormatPlugin formatPlugin,
+                                    List<SchemaPath> projectedColumns,
+                                    FragmentContext context,
+                                    int maxRecordsToRead) {
+
+    super(subScanSpec, formatPlugin, projectedColumns, context, maxRecordsToRead);
+    batchSize = (int)context.getOptions().getOption(ExecConstants.QUERY_ROWKEYJOIN_BATCHSIZE);
+    int idx = 0;
+    FieldPath[] scannedFields = this.getScannedFields();
+
+    // only populate projections for non-star query (for star, null is interpreted as all fields)
+    if (!this.isStarQuery() && scannedFields != null && scannedFields.length > 0) {
+      projections = new String[scannedFields.length];
+      for (FieldPath path : scannedFields) {
+        projections[idx] = path.asPathString();
+        ++idx;
+      }
+    }
+  }
+
+  public void readToInitSchema() {
+    DBDocumentReaderBase reader = null;
+    vectorWriter.setPosition(0);
+
+    try (DocumentStream dstream = table.find()) {
+      reader = (DBDocumentReaderBase) dstream.iterator().next().asReader();
+      documentWriter.writeDBDocument(vectorWriter, reader);
+    } catch(UserException e) {
+      throw UserException.unsupportedError(e)
+          .addContext(String.format("Table: %s, document id: '%s'",
+              getTable().getPath(),
+              reader == null ? null : IdCodec.asString(reader.getId())))
+          .build(logger);
+    } catch (SchemaChangeException e) {
+      if (getIgnoreSchemaChange()) {
+        logger.warn("{}. Dropping the row from result.", e.getMessage());
+        logger.debug("Stack trace:", e);
+      } else {
+        throw dataReadError(logger, e);
+      }
+    }
+    finally {
+      vectorWriter.setPosition(0);
+    }
+  }
+
+  @Override
+  public void setup(OperatorContext context, OutputMutator output) throws ExecutionSetupException {
+    RestrictedMapRDBSubScanSpec rss = ((RestrictedMapRDBSubScanSpec) this.subScanSpec);
+    RowKeyJoin rjBatch = rss.getJoinForSubScan();
+    if (rjBatch == null) {
+      throw new ExecutionSetupException("RowKeyJoin Batch is not setup for Restricted MapRDB Subscan");
+    }
+
+    AbstractRecordBatch.BatchState state = rjBatch.getBatchState();
+    if ( state == AbstractRecordBatch.BatchState.BUILD_SCHEMA ||
+         state == AbstractRecordBatch.BatchState.FIRST) {
+      super.setup(context, output);
+    }
+    return;
+  }
+
+  @Override
+  public int next() {
+    Stopwatch watch = Stopwatch.createUnstarted();
+    watch.start();
+    RestrictedMapRDBSubScanSpec rss = ((RestrictedMapRDBSubScanSpec) this.subScanSpec);
+
+    vectorWriter.allocate();
+    vectorWriter.reset();
+
+    if (!rss.readyToGetRowKey()) {
+      // not ready to get rowkey, so we just load a record to initialize schema; only do this
+      // when we are in the build schema phase
+      if (rss.isBuildSchemaPhase()) {
+        readToInitSchema();
+      }
+      return 0;
+    }
+
+    final MultiGet multiGet = new MultiGet((BaseJsonTable) table, condition, false, projections);
+    int recordCount = 0;
+    DBDocumentReaderBase reader = null;
+
+    int maxRecordsForThisBatch = this.maxRecordsToRead > 0?
+        Math.min(rss.getMaxRowKeysToBeRead(), this.maxRecordsToRead) :
+            this.maxRecordsToRead == -1 ? rss.getMaxRowKeysToBeRead() : 0;
+
+    Stopwatch timer = Stopwatch.createUnstarted();
+
+    while (recordCount < maxRecordsForThisBatch) {
+      ByteBuffer rowKeyIds[] = rss.getRowKeyIdsToRead(batchSize);
+      if (rowKeyIds == null) {
+        break;
+      }
+      try {
+        timer.start();
+        final List<Document> docList = multiGet.doGet(rowKeyIds);
+        int index = 0;
+        long docsToRead = docList.size();
+        // If limit pushdown then stop once we have `limit` rows from multiget i.e. maxRecordsForThisBatch
+        if (this.maxRecordsToRead != -1) {
+          docsToRead = Math.min(docsToRead, maxRecordsForThisBatch);
+        }
+        while (index < docsToRead) {
+          vectorWriter.setPosition(recordCount);
+          reader = (DBDocumentReaderBase) docList.get(index).asReader();
+          documentWriter.writeDBDocument(vectorWriter, reader);
+          recordCount++;
+          index++;
+        }
+        timer.stop();
+      } catch (UserException e) {
+        throw UserException.unsupportedError(e).addContext(String.format("Table: %s, document id: '%s'",
+          getTable().getPath(), reader == null ? null : IdCodec.asString(reader.getId()))).build(logger);
+      } catch (SchemaChangeException e) {
+        if (getIgnoreSchemaChange()) {
+          logger.warn("{}. Dropping the row from result.", e.getMessage());
+          logger.debug("Stack trace:", e);
+        } else {
+          throw dataReadError(logger, e);
+        }
+      }
+    }
+
+    vectorWriter.setValueCount(recordCount);
+    if (maxRecordsToRead > 0) {
+      if (maxRecordsToRead - recordCount >= 0) {
+        maxRecordsToRead -= recordCount;
+      } else {
+        maxRecordsToRead = 0;
+      }
+    }
+
+    logger.debug("Took {} ms to get {} records, getrowkey {}", watch.elapsed(TimeUnit.MILLISECONDS), recordCount, timer.elapsed(TimeUnit.MILLISECONDS));
+    return recordCount;
+  }
+
+  @Override
+  public boolean hasNext() {
+    RestrictedMapRDBSubScanSpec rss = ((RestrictedMapRDBSubScanSpec) this.subScanSpec);
+
+    RowKeyJoin rjBatch = rss.getJoinForSubScan();
+    if (rjBatch == null) {
+      return false;
+    }
+
+    boolean hasMore = false;
+    AbstractRecordBatch.BatchState state = rss.getJoinForSubScan().getBatchState();
+    RowKeyJoin.RowKeyJoinState rkState = rss.getJoinForSubScan().getRowKeyJoinState();
+    if ( state == AbstractRecordBatch.BatchState.BUILD_SCHEMA ) {
+      hasMore = true;
+    } else if ( state == AbstractRecordBatch.BatchState.FIRST) {
+       if (this.maxRecordsToRead > 0) {
+         rss.getJoinForSubScan().setBatchState(AbstractRecordBatch.BatchState.NOT_FIRST);
+         rss.getJoinForSubScan().setRowKeyJoinState(RowKeyJoin.RowKeyJoinState.PROCESSING);
+         hasMore = true;
+       }
+    } else if ( rkState == RowKeyJoin.RowKeyJoinState.INITIAL) {
+      if (this.maxRecordsToRead > 0) {
+        rss.getJoinForSubScan().setRowKeyJoinState(RowKeyJoin.RowKeyJoinState.PROCESSING);
+        hasMore = true;
+      }
+    }
+
+    logger.debug("restricted reader hasMore = {}", hasMore);
+
+    return hasMore;
+  }
+
+}
diff --git a/contrib/format-maprdb/src/main/java/org/apache/drill/exec/store/mapr/db/json/RowCountVectorWriter.java b/contrib/format-maprdb/src/main/java/org/apache/drill/exec/store/mapr/db/json/RowCountVectorWriter.java
new file mode 100644
index 0000000..445bccb
--- /dev/null
+++ b/contrib/format-maprdb/src/main/java/org/apache/drill/exec/store/mapr/db/json/RowCountVectorWriter.java
@@ -0,0 +1,40 @@
+/*
+ * 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.store.mapr.db.json;
+
+import org.apache.drill.exec.exception.SchemaChangeException;
+import org.apache.drill.exec.vector.complex.impl.VectorContainerWriter;
+
+import com.mapr.db.ojai.DBDocumentReaderBase;
+
+/**
+ *  This is an optimized implementation of DocumentReaderVectorWriter that writes the row count.
+ */
+class RowCountVectorWriter extends DocumentReaderVectorWriter {
+
+  protected RowCountVectorWriter(final OjaiValueWriter valueWriter) {
+    super(valueWriter);
+  }
+
+  @Override
+  protected void writeDBDocument(VectorContainerWriter vectorWriter, DBDocumentReaderBase reader)
+      throws SchemaChangeException {
+    vectorWriter.rootAsMap().bit("count").writeBit(1);
+  }
+
+}
diff --git a/contrib/format-maprdb/src/main/java/org/apache/drill/exec/udf/mapr/db/ConditionPlaceholder.java b/contrib/format-maprdb/src/main/java/org/apache/drill/exec/udf/mapr/db/ConditionPlaceholder.java
new file mode 100644
index 0000000..3bc36b3
--- /dev/null
+++ b/contrib/format-maprdb/src/main/java/org/apache/drill/exec/udf/mapr/db/ConditionPlaceholder.java
@@ -0,0 +1,54 @@
+/*
+ * 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.udf.mapr.db;
+
+import org.apache.drill.exec.expr.DrillSimpleFunc;
+import org.apache.drill.exec.expr.annotations.FunctionTemplate;
+import org.apache.drill.exec.expr.annotations.Output;
+import org.apache.drill.exec.expr.annotations.Param;
+import org.apache.drill.exec.expr.holders.BigIntHolder;
+import org.apache.drill.exec.expr.holders.BitHolder;
+import org.apache.drill.exec.expr.holders.VarCharHolder;
+
+/**
+ * This is a placeholder for the ojai_condition() function.
+ *
+ * At this time, this function can only be used in predicates. The placeholder
+ * is here to prevent calcite from complaining; the function will get pushed down
+ * by the storage plug-in into DB. That process will go through JsonConditionBuilder.java,
+ * which will replace this function with the real OJAI equivalent to be pushed down.
+ * Therefore, there's no implementation here.
+ */
+@FunctionTemplate(
+                name="ojai_condition",
+                scope=FunctionTemplate.FunctionScope.SIMPLE,
+                nulls=FunctionTemplate.NullHandling.INTERNAL)
+public class ConditionPlaceholder implements DrillSimpleFunc {
+
+        @Param BigIntHolder /*FieldReader*/ field;
+        @Param(constant = true) VarCharHolder pattern;
+
+        @Output BitHolder output;
+
+        public void setup() {
+        }
+
+        public void eval() {
+        }
+
+}
diff --git a/contrib/format-maprdb/src/main/java/org/apache/drill/exec/udf/mapr/db/DecodeFieldPath.java b/contrib/format-maprdb/src/main/java/org/apache/drill/exec/udf/mapr/db/DecodeFieldPath.java
new file mode 100644
index 0000000..6748c4f
--- /dev/null
+++ b/contrib/format-maprdb/src/main/java/org/apache/drill/exec/udf/mapr/db/DecodeFieldPath.java
@@ -0,0 +1,65 @@
+/*
+ * 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.udf.mapr.db;
+
+import javax.inject.Inject;
+
+import org.apache.drill.exec.expr.DrillSimpleFunc;
+import org.apache.drill.exec.expr.annotations.FunctionTemplate;
+import org.apache.drill.exec.expr.annotations.FunctionTemplate.FunctionScope;
+import org.apache.drill.exec.expr.annotations.FunctionTemplate.NullHandling;
+import org.apache.drill.exec.expr.annotations.Output;
+import org.apache.drill.exec.expr.annotations.Param;
+import org.apache.drill.exec.expr.holders.VarCharHolder;
+
+import io.netty.buffer.DrillBuf;
+
+@FunctionTemplate(name = "maprdb_decode_fieldpath", scope = FunctionScope.SIMPLE, nulls = NullHandling.NULL_IF_NULL)
+public class DecodeFieldPath implements DrillSimpleFunc {
+  @Param  VarCharHolder input;
+  @Output VarCharHolder   out;
+
+  @Inject DrillBuf buffer;
+
+  @Override
+  public void setup() {
+  }
+
+  @Override
+  public void eval() {
+    String[] encodedPaths = org.apache.drill.exec.expr.fn.impl.StringFunctionHelpers.
+        toStringFromUTF8(input.start, input.end, input.buffer).split(",");
+    String[] decodedPaths = org.apache.drill.exec.util.EncodedSchemaPathSet.decode(encodedPaths);
+    java.util.Arrays.sort(decodedPaths);
+    
+    StringBuilder sb = new StringBuilder();
+    for(String decodedPath : decodedPaths) {
+      sb.append(", ").append(org.ojai.FieldPath.parseFrom(decodedPath).asPathString());
+    }
+    String outputString = "[" + sb.substring(2) + "]";
+    final byte[] strBytes = outputString.getBytes(com.google.common.base.Charsets.UTF_8);
+    buffer.setBytes(0, strBytes);
+    buffer.setIndex(0, strBytes.length);
+
+    out.start = 0;
+    out.end = strBytes.length;
+    out.buffer = buffer;
+  }
+
+}
+
diff --git a/contrib/format-maprdb/src/main/java/org/apache/drill/exec/udf/mapr/db/MatchesPlaceholder.java b/contrib/format-maprdb/src/main/java/org/apache/drill/exec/udf/mapr/db/MatchesPlaceholder.java
new file mode 100644
index 0000000..6aad44e
--- /dev/null
+++ b/contrib/format-maprdb/src/main/java/org/apache/drill/exec/udf/mapr/db/MatchesPlaceholder.java
@@ -0,0 +1,54 @@
+/*
+ * 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.udf.mapr.db;
+
+import org.apache.drill.exec.expr.DrillSimpleFunc;
+import org.apache.drill.exec.expr.annotations.FunctionTemplate;
+import org.apache.drill.exec.expr.annotations.Output;
+import org.apache.drill.exec.expr.annotations.Param;
+import org.apache.drill.exec.expr.holders.BigIntHolder;
+import org.apache.drill.exec.expr.holders.BitHolder;
+import org.apache.drill.exec.expr.holders.VarCharHolder;
+
+/**
+ * This is a placeholder for the matches() function.
+ *
+ * At this time, this function can only be used in predicates. The placeholder
+ * is here to prevent calcite from complaining; the function will get pushed down
+ * by the storage plug-in into DB. That process will go through JsonConditionBuilder.java,
+ * which will replace this function with the real OJAI equivalent to be pushed down.
+ * Therefore, there's no implementation here.
+ */
+@FunctionTemplate(
+                name="ojai_matches",
+                scope=FunctionTemplate.FunctionScope.SIMPLE,
+                nulls=FunctionTemplate.NullHandling.INTERNAL)
+public class MatchesPlaceholder implements DrillSimpleFunc {
+
+        @Param BigIntHolder /*FieldReader*/ field;
+        @Param(constant = true) VarCharHolder pattern;
+
+        @Output BitHolder output;
+
+        public void setup() {
+        }
+
+        public void eval() {
+        }
+
+}
diff --git a/contrib/format-maprdb/src/main/java/org/apache/drill/exec/udf/mapr/db/NotMatchesPlaceholder.java b/contrib/format-maprdb/src/main/java/org/apache/drill/exec/udf/mapr/db/NotMatchesPlaceholder.java
new file mode 100644
index 0000000..56baebb
--- /dev/null
+++ b/contrib/format-maprdb/src/main/java/org/apache/drill/exec/udf/mapr/db/NotMatchesPlaceholder.java
@@ -0,0 +1,54 @@
+/*
+ * 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.udf.mapr.db;
+
+import org.apache.drill.exec.expr.DrillSimpleFunc;
+import org.apache.drill.exec.expr.annotations.FunctionTemplate;
+import org.apache.drill.exec.expr.annotations.Output;
+import org.apache.drill.exec.expr.annotations.Param;
+import org.apache.drill.exec.expr.holders.BigIntHolder;
+import org.apache.drill.exec.expr.holders.BitHolder;
+import org.apache.drill.exec.expr.holders.VarCharHolder;
+
+/**
+ * This is a placeholder for the notMatches() function.
+ *
+ * At this time, this function can only be used in predicates. The placeholder
+ * is here to prevent calcite from complaining; the function will get pushed down
+ * by the storage plug-in into DB. That process will go through JsonConditionBuilder.java,
+ * which will replace this function with the real OJAI equivalent to be pushed down.
+ * Therefore, there's no implementation here.
+ */
+@FunctionTemplate(
+                name="ojai_notmatches",
+                scope=FunctionTemplate.FunctionScope.SIMPLE,
+                nulls=FunctionTemplate.NullHandling.INTERNAL)
+public class NotMatchesPlaceholder implements DrillSimpleFunc {
+
+        @Param BigIntHolder /*FieldReader*/ field;
+        @Param(constant = true) VarCharHolder pattern;
+
+        @Output BitHolder output;
+
+        public void setup() {
+        }
+
+        public void eval() {
+        }
+
+}
diff --git a/contrib/format-maprdb/src/main/java/org/apache/drill/exec/udf/mapr/db/NotTypeOfPlaceholder.java b/contrib/format-maprdb/src/main/java/org/apache/drill/exec/udf/mapr/db/NotTypeOfPlaceholder.java
new file mode 100644
index 0000000..6c01a48
--- /dev/null
+++ b/contrib/format-maprdb/src/main/java/org/apache/drill/exec/udf/mapr/db/NotTypeOfPlaceholder.java
@@ -0,0 +1,54 @@
+/*
+ * 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.udf.mapr.db;
+
+import org.apache.drill.exec.expr.DrillSimpleFunc;
+import org.apache.drill.exec.expr.annotations.FunctionTemplate;
+import org.apache.drill.exec.expr.annotations.Output;
+import org.apache.drill.exec.expr.annotations.Param;
+import org.apache.drill.exec.expr.holders.BigIntHolder;
+import org.apache.drill.exec.expr.holders.BitHolder;
+import org.apache.drill.exec.expr.holders.IntHolder;
+
+/**
+ * This is a placeholder for the nottypeof() function.
+ *
+ * At this time, this function can only be used in predicates. The placeholder
+ * is here to prevent calcite from complaining; the function will get pushed down
+ * by the storage plug-in into DB. That process will go through JsonConditionBuilder.java,
+ * which will replace this function with the real OJAI equivalent to be pushed down.
+ * Therefore, there's no implementation here.
+ */
+@FunctionTemplate(
+                name="ojai_nottypeof",
+                scope=FunctionTemplate.FunctionScope.SIMPLE,
+                nulls=FunctionTemplate.NullHandling.INTERNAL)
+public class NotTypeOfPlaceholder implements DrillSimpleFunc {
+
+        @Param BigIntHolder /*FieldReader*/ field;
+        @Param(constant = true) IntHolder typeCode;
+
+        @Output BitHolder output;
+
+        public void setup() {
+        }
+
+        public void eval() {
+        }
+
+}
diff --git a/contrib/format-maprdb/src/main/java/org/apache/drill/exec/udf/mapr/db/SizeOfPlaceholder.java b/contrib/format-maprdb/src/main/java/org/apache/drill/exec/udf/mapr/db/SizeOfPlaceholder.java
new file mode 100644
index 0000000..7d7150c
--- /dev/null
+++ b/contrib/format-maprdb/src/main/java/org/apache/drill/exec/udf/mapr/db/SizeOfPlaceholder.java
@@ -0,0 +1,55 @@
+/*
+ * 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.udf.mapr.db;
+
+import org.apache.drill.exec.expr.DrillSimpleFunc;
+import org.apache.drill.exec.expr.annotations.FunctionTemplate;
+import org.apache.drill.exec.expr.annotations.Output;
+import org.apache.drill.exec.expr.annotations.Param;
+import org.apache.drill.exec.expr.holders.BigIntHolder;
+import org.apache.drill.exec.expr.holders.BitHolder;
+import org.apache.drill.exec.expr.holders.VarCharHolder;
+
+/**
+ * This is a placeholder for the sizeof() function.
+ *
+ * At this time, this function can only be used in predicates. The placeholder
+ * is here to prevent calcite from complaining; the function will get pushed down
+ * by the storage plug-in into DB. That process will go through JsonConditionBuilder.java,
+ * which will replace this function with the real OJAI equivalent to be pushed down.
+ * Therefore, there's no implementation here.
+ */
+@FunctionTemplate(
+                name="ojai_sizeof",
+                scope=FunctionTemplate.FunctionScope.SIMPLE,
+                nulls=FunctionTemplate.NullHandling.INTERNAL)
+public class SizeOfPlaceholder implements DrillSimpleFunc {
+
+        @Param BigIntHolder /*FieldReader*/ field;
+        @Param(constant = true) VarCharHolder relOp;
+        @Param(constant = true) BigIntHolder size;
+
+        @Output BitHolder output;
+
+        public void setup() {
+        }
+
+        public void eval() {
+        }
+
+}
diff --git a/contrib/format-maprdb/src/main/java/org/apache/drill/exec/udf/mapr/db/TypeOfPlaceholder.java b/contrib/format-maprdb/src/main/java/org/apache/drill/exec/udf/mapr/db/TypeOfPlaceholder.java
new file mode 100644
index 0000000..1d1efc0
--- /dev/null
+++ b/contrib/format-maprdb/src/main/java/org/apache/drill/exec/udf/mapr/db/TypeOfPlaceholder.java
@@ -0,0 +1,54 @@
+/*
+ * 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.udf.mapr.db;
+
+import org.apache.drill.exec.expr.DrillSimpleFunc;
+import org.apache.drill.exec.expr.annotations.FunctionTemplate;
+import org.apache.drill.exec.expr.annotations.Output;
+import org.apache.drill.exec.expr.annotations.Param;
+import org.apache.drill.exec.expr.holders.BigIntHolder;
+import org.apache.drill.exec.expr.holders.BitHolder;
+import org.apache.drill.exec.expr.holders.IntHolder;
+
+/**
+ * This is a placeholder for the typeof() function.
+ *
+ * At this time, this function can only be used in predicates. The placeholder
+ * is here to prevent calcite from complaining; the function will get pushed down
+ * by the storage plug-in into DB. That process will go through JsonConditionBuilder.java,
+ * which will replace this function with the real OJAI equivalent to be pushed down.
+ * Therefore, there's no implementation here.
+ */
+@FunctionTemplate(
+                name="ojai_typeof",
+                scope=FunctionTemplate.FunctionScope.SIMPLE,
+                nulls=FunctionTemplate.NullHandling.INTERNAL)
+public class TypeOfPlaceholder implements DrillSimpleFunc {
+
+        @Param BigIntHolder /*FieldReader*/ field;
+        @Param(constant = true) IntHolder typeCode;
+
+        @Output BitHolder output;
+
+        public void setup() {
+        }
+
+        public void eval() {
+        }
+
+}
diff --git a/contrib/format-maprdb/src/main/resources/drill-module.conf b/contrib/format-maprdb/src/main/resources/drill-module.conf
index 8d42355..a4270c7 100644
--- a/contrib/format-maprdb/src/main/resources/drill-module.conf
+++ b/contrib/format-maprdb/src/main/resources/drill-module.conf
@@ -17,4 +17,22 @@
 //  This file can also include any supplementary configuration information.
 //  This file is in HOCON format, see https://github.com/typesafehub/config/blob/master/HOCON.md for more information.
 
-# This file currently does not contain any configuration
\ No newline at end of file
+format-maprdb: {
+  json: {
+    mediaType: SSD,
+    scanSizeMB: 128,
+    restrictedScanSizeMB: 4096,
+    useNumRegionsForDistribution: false,
+    tableCache: {
+      enabled: true,
+      size: 1000,
+      expireTimeInMinutes: 60
+    },
+    pluginCost: {
+      blockSize: 8192
+    }
+  }
+}
+
+// this picks up the UDFs
+drill.classpath.scanning.packages += "org.apache.drill.exec.udf.mapr.db"
diff --git a/contrib/format-maprdb/src/test/java/com/mapr/drill/maprdb/tests/MaprDBTestsSuite.java b/contrib/format-maprdb/src/test/java/com/mapr/drill/maprdb/tests/MaprDBTestsSuite.java
index 0c81a18..edd5ab4 100644
--- a/contrib/format-maprdb/src/test/java/com/mapr/drill/maprdb/tests/MaprDBTestsSuite.java
+++ b/contrib/format-maprdb/src/test/java/com/mapr/drill/maprdb/tests/MaprDBTestsSuite.java
@@ -17,12 +17,12 @@
  */
 package com.mapr.drill.maprdb.tests;
 
-import java.io.IOException;
 import java.io.InputStream;
 import java.lang.management.ManagementFactory;
 import java.util.concurrent.atomic.AtomicInteger;
 
 import org.apache.drill.exec.server.DrillbitContext;
+import org.apache.drill.exec.store.StoragePluginRegistry;
 import org.apache.drill.exec.store.dfs.FileSystemConfig;
 import org.apache.drill.hbase.HBaseTestsSuite;
 import org.apache.hadoop.conf.Configuration;
@@ -31,25 +31,23 @@ import org.junit.BeforeClass;
 import org.junit.runner.RunWith;
 import org.junit.runners.Suite;
 import org.junit.runners.Suite.SuiteClasses;
-import org.ojai.Document;
-import org.ojai.DocumentStream;
-import org.ojai.json.Json;
 
 import com.mapr.db.Admin;
 import com.mapr.db.MapRDB;
-import com.mapr.db.Table;
 import com.mapr.drill.maprdb.tests.binary.TestMapRDBFilterPushDown;
 import com.mapr.drill.maprdb.tests.binary.TestMapRDBSimple;
+import com.mapr.drill.maprdb.tests.json.TestScanRanges;
 import com.mapr.drill.maprdb.tests.json.TestSimpleJson;
 
 @RunWith(Suite.class)
 @SuiteClasses({
   TestMapRDBSimple.class,
   TestMapRDBFilterPushDown.class,
-  TestSimpleJson.class
+  TestSimpleJson.class,
+  TestScanRanges.class
 })
 public class MaprDBTestsSuite {
-  private static final String TMP_BUSINESS_TABLE = "/tmp/business";
+  public static final int INDEX_FLUSH_TIMEOUT = 60000;
 
   private static final boolean IS_DEBUG = ManagementFactory.getRuntimeMXBean().getInputArguments().toString().indexOf("-agentlib:jdwp") > 0;
 
@@ -65,13 +63,13 @@ public class MaprDBTestsSuite {
         if (initCount.get() == 0) {
           HBaseTestsSuite.configure(false /*manageHBaseCluster*/, true /*createTables*/);
           HBaseTestsSuite.initCluster();
-          createJsonTables();
 
           // Sleep to allow table data to be flushed to tables.
           // Without this, the row count stats to return 0,
           // causing the planner to reject optimized plans.
           Thread.sleep(5000);
 
+          admin = MapRDB.newAdmin();
           conf = HBaseTestsSuite.getConf();
           initCount.incrementAndGet(); // must increment while inside the synchronized block
           return;
@@ -87,17 +85,19 @@ public class MaprDBTestsSuite {
     synchronized (MaprDBTestsSuite.class) {
       if (initCount.decrementAndGet() == 0) {
         HBaseTestsSuite.tearDownCluster();
-        deleteJsonTables();
+        admin.close();
       }
     }
   }
 
-  private static volatile boolean pluginCreated;
+  private static volatile boolean pluginsUpdated;
 
   public static Configuration createPluginAndGetConf(DrillbitContext ctx) throws Exception {
-    if (!pluginCreated) {
+    if (!pluginsUpdated) {
       synchronized (MaprDBTestsSuite.class) {
-        if (!pluginCreated) {
+        if (!pluginsUpdated) {
+          StoragePluginRegistry pluginRegistry = ctx.getStorage();
+
           String pluginConfStr = "{" +
               "  \"type\": \"file\"," +
               "  \"enabled\": true," +
@@ -108,6 +108,11 @@ public class MaprDBTestsSuite {
               "      \"writable\": false," +
               "      \"defaultInputFormat\": \"maprdb\"" +
               "    }," +
+              "    \"tmp\": {" +
+              "      \"location\": \"/tmp\"," +
+              "      \"writable\": true," +
+              "      \"defaultInputFormat\": \"parquet\"" +
+              "    }," +
               "    \"root\": {" +
               "      \"location\": \"/\"," +
               "      \"writable\": false," +
@@ -121,6 +126,9 @@ public class MaprDBTestsSuite {
               "      \"readAllNumbersAsDouble\": false," +
               "      \"enablePushdown\": true" +
               "    }," +
+              "   \"parquet\": {" +
+              "      \"type\": \"parquet\"" +
+              "    }," +
               "   \"streams\": {" +
               "      \"type\": \"streams\"" +
               "    }" +
@@ -129,7 +137,7 @@ public class MaprDBTestsSuite {
 
           FileSystemConfig pluginConfig = ctx.getLpPersistence().getMapper().readValue(pluginConfStr, FileSystemConfig.class);
           // create the plugin with "hbase" name so that we can run HBase unit tests against them
-          ctx.getStorage().createOrUpdate("hbase", pluginConfig, true);
+          pluginRegistry.createOrUpdate("hbase", pluginConfig, true);
         }
       }
     }
@@ -140,33 +148,12 @@ public class MaprDBTestsSuite {
     return IS_DEBUG;
   }
 
-  public static InputStream getJsonStream(String resourceName) {
-    return MaprDBTestsSuite.class.getClassLoader().getResourceAsStream(resourceName);
-  }
-
-  public static void createJsonTables() throws IOException {
-    admin = MapRDB.newAdmin();
-    if (admin.tableExists(TMP_BUSINESS_TABLE)) {
-      admin.deleteTable(TMP_BUSINESS_TABLE);
-    }
-
-    try (Table table = admin.createTable(TMP_BUSINESS_TABLE);
-         InputStream in = getJsonStream("json/business.json");
-         DocumentStream stream = Json.newDocumentStream(in)) {
-      for (Document document : stream) {
-        table.insert(document, "business_id");
-      }
-      table.flush();
-    }
+  public static Admin getAdmin() {
+    return admin;
   }
 
-  public static void deleteJsonTables() {
-    if (admin != null) {
-      if (admin.tableExists(TMP_BUSINESS_TABLE)) {
-        admin.deleteTable(TMP_BUSINESS_TABLE);
-      }
-      admin.close();
-    }
+  public static InputStream getJsonStream(String resourceName) {
+    return MaprDBTestsSuite.class.getResourceAsStream(resourceName);
   }
 
 }
diff --git a/contrib/format-maprdb/src/test/java/com/mapr/drill/maprdb/tests/json/BaseJsonTest.java b/contrib/format-maprdb/src/test/java/com/mapr/drill/maprdb/tests/json/BaseJsonTest.java
index 550fb73..2d6ae06 100644
--- a/contrib/format-maprdb/src/test/java/com/mapr/drill/maprdb/tests/json/BaseJsonTest.java
+++ b/contrib/format-maprdb/src/test/java/com/mapr/drill/maprdb/tests/json/BaseJsonTest.java
@@ -29,6 +29,19 @@ import org.junit.BeforeClass;
 import com.mapr.drill.maprdb.tests.MaprDBTestsSuite;
 
 public class BaseJsonTest extends BaseTestQuery {
+  protected static final String SCHEMA = "hbase.root";
+  
+  protected String format(final String sql) {
+    return String.format(sql, SCHEMA, getTablePath());
+  }
+
+  protected String getTablePath() {
+    throw new RuntimeException("unimplemented");
+  }
+
+  public static String format(final String sql, final String tablePath) {
+    return String.format(sql, SCHEMA, tablePath);
+  }
 
   @BeforeClass
   public static void setupDefaultTestCluster() throws Exception {
diff --git a/contrib/format-maprdb/src/test/java/com/mapr/drill/maprdb/tests/json/TestEncodedFieldPaths.java b/contrib/format-maprdb/src/test/java/com/mapr/drill/maprdb/tests/json/TestEncodedFieldPaths.java
new file mode 100644
index 0000000..dcbc440
--- /dev/null
+++ b/contrib/format-maprdb/src/test/java/com/mapr/drill/maprdb/tests/json/TestEncodedFieldPaths.java
@@ -0,0 +1,128 @@
+/*
+ * 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 com.mapr.drill.maprdb.tests.json;
+
+import static com.mapr.drill.maprdb.tests.MaprDBTestsSuite.INDEX_FLUSH_TIMEOUT;
+
+import java.io.InputStream;
+
+import org.apache.drill.PlanTestBase;
+import org.apache.drill.exec.util.EncodedSchemaPathSet;
+import org.junit.AfterClass;
+import org.junit.BeforeClass;
+import org.junit.Test;
+import org.ojai.DocumentStream;
+import org.ojai.json.Json;
+
+import com.google.common.collect.ImmutableMap;
+import com.mapr.db.Table;
+import com.mapr.db.tests.utils.DBTests;
+
+public class TestEncodedFieldPaths extends BaseJsonTest {
+
+  private static final String TABLE_NAME = "encoded_fields_userdata_table";
+  private static final String INDEX_NAME = "encoded_fields_userdata_index";
+  private static final String JSON_FILE_URL = "/com/mapr/drill/json/encoded_fields_userdata.json";
+
+  private static boolean tableCreated = false;
+  private static String tablePath;
+
+  @BeforeClass
+  public static void setup_TestEncodedFieldPaths() throws Exception {
+    try (Table table = DBTests.createOrReplaceTable(TABLE_NAME, ImmutableMap.of("codes", "codes"))) {
+      tableCreated = true;
+      tablePath = table.getPath().toUri().getPath();
+
+      DBTests.createIndex(TABLE_NAME, INDEX_NAME, new String[] {"age"}, new String[] {"name.last", "data.salary"});
+      DBTests.admin().getTableIndexes(table.getPath(), true);
+
+      try (final InputStream in = TestEncodedFieldPaths.class.getResourceAsStream(JSON_FILE_URL);
+          final DocumentStream stream = Json.newDocumentStream(in);) {
+        table.insertOrReplace(stream);
+        table.flush();
+      }
+
+      // wait for the indexes to sync
+      DBTests.waitForRowCount(table.getPath(), 5, INDEX_FLUSH_TIMEOUT);
+      DBTests.waitForIndexFlush(table.getPath(), INDEX_FLUSH_TIMEOUT);
+    } finally {
+      test("ALTER SESSION SET `planner.disable_full_table_scan` = true");
+    }
+  }
+
+  @AfterClass
+  public static void cleanup_TestEncodedFieldPaths() throws Exception {
+    test("ALTER SESSION SET `planner.disable_full_table_scan` = false");
+    if (tableCreated) {
+      DBTests.deleteTables(TABLE_NAME);
+    }
+  }
+
+  @Test
+  public void test_encoded_fields_with_non_covering_index() throws Exception {
+    final String sql = String.format(
+        "SELECT\n"
+      + "  t.`%s`,t.`$$document`\n"
+      + "FROM\n"
+      + "  hbase.root.`%s` t\n"
+      + "WHERE (t.`age` > 20)\n"
+      + "ORDER BY t.`_id` ASC",
+      EncodedSchemaPathSet.encode("_id", "codes")[0],
+      tablePath);
+
+    setColumnWidths(new int[] {20, 60});
+    runSQLAndVerifyCount(sql, 3);
+
+
+    // plan test
+    final String[] expectedPlan = {"JsonTableGroupScan.*indexName=encoded_fields_userdata_index.*" + // scan on index
+                                   "columns=\\[`_id`, `age`\\]", 
+                                   "RestrictedJsonTableGroupScan.*" + // restricted scan on the table with encoded name
+                                   "columns=\\[`age`, `\\$\\$ENC00L5UWIADDN5SGK4Y`, `\\$\\$document`, `_id`\\]",
+                                   "RowKeyJoin"};                             // join on row_key
+    final String[] excludedPlan = {};
+
+    PlanTestBase.testPlanMatchingPatterns(sql, expectedPlan, excludedPlan);
+  }
+
+  @Test
+  public void test_encoded_fields_with_covering_index() throws Exception {
+    final String sql = String.format(
+        "SELECT\n"
+            + "  t.`%s`,t.`$$document`\n"
+            + "FROM\n"
+            + "  hbase.root.`%s` t\n"
+            + "WHERE (t.`age` > 10)\n"
+            + "ORDER BY t.`_id` ASC",
+            EncodedSchemaPathSet.encode("name.last", "data.salary")[0],
+            tablePath);
+
+    setColumnWidths(new int[] {20, 60});
+    runSQLAndVerifyCount(sql, 4);
+
+
+    // plan test
+    final String[] expectedPlan = {"JsonTableGroupScan.*indexName=encoded_fields_userdata_index.*",           // scan on index
+        "columns=\\[`age`, `\\$\\$ENC00NZQW2ZJONRQXG5AAMRQXIYJOONQWYYLSPE`, `\\$\\$document`, `_id`\\]"};
+    final String[] excludedPlan = {"RestrictedJsonTableGroupScan",                      // restricted scan on the table
+                                   "RowKeyJoin"};                                       // join on row_key
+
+    PlanTestBase.testPlanMatchingPatterns(sql, expectedPlan, excludedPlan);
+  }
+
+}
diff --git a/contrib/format-maprdb/src/test/java/com/mapr/drill/maprdb/tests/json/TestFieldPathHelper.java b/contrib/format-maprdb/src/test/java/com/mapr/drill/maprdb/tests/json/TestFieldPathHelper.java
new file mode 100644
index 0000000..914ad96
--- /dev/null
+++ b/contrib/format-maprdb/src/test/java/com/mapr/drill/maprdb/tests/json/TestFieldPathHelper.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 com.mapr.drill.maprdb.tests.json;
+
+import static org.junit.Assert.assertEquals;
+
+import org.apache.drill.common.expression.SchemaPath;
+import org.apache.drill.exec.store.mapr.db.json.FieldPathHelper;
+import org.junit.Test;
+import org.ojai.FieldPath;
+
+public class TestFieldPathHelper {
+
+  @Test
+  public void simeTests() {
+    String[] pathStrs = {"a", "a.b", "a.b.c", "a[1].b[2].c", "a[0][1][2][3].b"};
+    FieldPath[] fieldPaths = new FieldPath[pathStrs.length];
+    SchemaPath[] schemaPaths = new SchemaPath[pathStrs.length];
+
+    // build
+    for (int i = 0; i < pathStrs.length; i++) {
+      String path = pathStrs[i];
+      fieldPaths[i] = FieldPath.parseFrom(path);
+      schemaPaths[i] = SchemaPath.parseFromString(path);
+    }
+
+    //verify
+    for (int i = 0; i < pathStrs.length; i++) {
+      FieldPath fp = FieldPathHelper.schemaPath2FieldPath(schemaPaths[i]);
+      assertEquals(fieldPaths[i], fp);
+
+      SchemaPath sp = FieldPathHelper.fieldPath2SchemaPath(fieldPaths[i]);
+      assertEquals(schemaPaths[i], sp);
+    }
+  }
+
+}
diff --git a/contrib/format-maprdb/src/test/java/com/mapr/drill/maprdb/tests/json/TestScanRanges.java b/contrib/format-maprdb/src/test/java/com/mapr/drill/maprdb/tests/json/TestScanRanges.java
new file mode 100644
index 0000000..3ec27d9
--- /dev/null
+++ b/contrib/format-maprdb/src/test/java/com/mapr/drill/maprdb/tests/json/TestScanRanges.java
@@ -0,0 +1,158 @@
+/*
+ * 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 com.mapr.drill.maprdb.tests.json;
+
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertNotNull;
+import static org.junit.Assert.assertTrue;
+
+import java.io.InputStream;
+import java.util.List;
+
+import org.apache.drill.exec.proto.UserBitShared;
+import org.apache.drill.exec.proto.UserBitShared.QueryProfile;
+import org.apache.drill.exec.proto.UserBitShared.QueryType;
+import org.apache.drill.exec.proto.helper.QueryIdHelper;
+import org.apache.drill.exec.rpc.user.AwaitableUserResultsListener;
+import org.apache.drill.exec.store.sys.PersistentStore;
+import org.apache.drill.exec.work.foreman.QueryManager;
+import org.junit.AfterClass;
+import org.junit.BeforeClass;
+import org.junit.Test;
+import org.junit.experimental.categories.Category;
+import org.ojai.Document;
+import org.ojai.DocumentStream;
+import org.ojai.json.Json;
+
+import com.google.common.collect.Lists;
+import com.mapr.db.Table;
+import com.mapr.db.tests.utils.DBTests;
+import com.mapr.drill.maprdb.tests.MaprDBTestsSuite;
+import com.mapr.tests.annotations.ClusterTest;
+
+@Category(ClusterTest.class)
+public class TestScanRanges extends BaseJsonTest {
+  private static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(TestScanRanges.class);
+
+  private static final int TOTAL_ROW_COUNT = 1000000;
+  private static final String TABLE_NAME = "large_table_TestScanRanges";
+  private static final String JSON_FILE_URL = "/com/mapr/drill/json/business.json";
+
+  private static boolean tableCreated = false;
+  private static String tablePath;
+  protected String getTablePath() {
+    return tablePath;
+  }
+
+  @BeforeClass
+  public static void setup_TestSimpleJson() throws Exception {
+    // We create a large table with auto-split set to disabled.
+    // Without intra-tablet partitioning, this test should run with only one minor fragment
+    try (Table table = DBTests.createOrReplaceTable(TABLE_NAME, false /*autoSplit*/);
+         InputStream in = MaprDBTestsSuite.getJsonStream(JSON_FILE_URL);
+         DocumentStream stream = Json.newDocumentStream(in)) {
+      tableCreated = true;
+      tablePath = table.getPath().toUri().getPath();
+
+      List<Document> docs = Lists.newArrayList(stream);
+      for (char ch = 'A'; ch <= 'T'; ch++) {
+        for (int rowIndex = 0; rowIndex < 5000; rowIndex++) {
+          for (int i = 0; i < docs.size(); i++) {
+            final Document document = docs.get(i);
+            final String id = String.format("%c%010d%03d", ch, rowIndex, i);
+            document.set("documentId", rowIndex);
+            table.insertOrReplace(id, document);
+          }
+        }
+      }
+      table.flush();
+      DBTests.waitForRowCount(table.getPath(), TOTAL_ROW_COUNT);
+
+      setSessionOption("planner.width.max_per_node", "5");
+   }
+  }
+
+  @AfterClass
+  public static void cleanup_TestEncodedFieldPaths() throws Exception {
+    if (tableCreated) {
+      DBTests.deleteTables(TABLE_NAME);
+    }
+  }
+
+  @Test
+  public void test_scan_ranges() throws Exception {
+    final PersistentStore<UserBitShared.QueryProfile> completed = getDrillbitContext().getProfileStoreContext().getCompletedProfileStore();
+
+    setColumnWidths(new int[] {25, 40, 25, 45});
+    final String sql = format("SELECT\n"
+        + "  *\n"
+        + "FROM\n"
+        + "  %s.`%s` business");
+
+    final SilentListener resultListener = new SilentListener();
+    final AwaitableUserResultsListener listener = new AwaitableUserResultsListener(resultListener);
+    testWithListener(QueryType.SQL, sql, listener);
+    listener.await();
+
+    assertEquals(TOTAL_ROW_COUNT, resultListener.getRowCount());
+    String queryId = QueryIdHelper.getQueryId(resultListener.getQueryId());
+
+    QueryProfile profile = completed.get(queryId);
+    String profileString = String.valueOf(profile);
+    logger.debug(profileString);
+    assertNotNull(profile);
+    assertTrue(profile.getTotalFragments() >= 5); // should at least as many as
+  }
+
+  @Test
+  public void test_scan_ranges_with_filter_on_id() throws Exception {
+    setColumnWidths(new int[] {25, 25, 25});
+    final String sql = format("SELECT\n"
+        + "  _id, business_id, city\n"
+        + "FROM\n"
+        + "  %s.`%s` business\n"
+        + "WHERE\n"
+        + " _id > 'M' AND _id < 'Q'");
+
+    final SilentListener resultListener = new SilentListener();
+    final AwaitableUserResultsListener listener = new AwaitableUserResultsListener(resultListener);
+    testWithListener(QueryType.SQL, sql, listener);
+    listener.await();
+
+    assertEquals(200000, resultListener.getRowCount());
+  }
+
+  @Test
+  public void test_scan_ranges_with_filter_on_non_id_field() throws Exception {
+    setColumnWidths(new int[] {25, 25, 25});
+    final String sql = format("SELECT\n"
+        + "  _id, business_id, documentId\n"
+        + "FROM\n"
+        + "  %s.`%s` business\n"
+        + "WHERE\n"
+        + " documentId >= 100 AND documentId < 150");
+
+    final SilentListener resultListener = new SilentListener();
+    final AwaitableUserResultsListener listener = new AwaitableUserResultsListener(resultListener);
+    testWithListener(QueryType.SQL, sql, listener);
+    listener.await();
+
+    assertEquals(10000, resultListener.getRowCount());
+  }
+
+}
diff --git a/contrib/format-maprdb/src/test/java/com/mapr/drill/maprdb/tests/json/TestSimpleJson.java b/contrib/format-maprdb/src/test/java/com/mapr/drill/maprdb/tests/json/TestSimpleJson.java
index 26f54b8..4dbff5d 100644
--- a/contrib/format-maprdb/src/test/java/com/mapr/drill/maprdb/tests/json/TestSimpleJson.java
+++ b/contrib/format-maprdb/src/test/java/com/mapr/drill/maprdb/tests/json/TestSimpleJson.java
@@ -21,6 +21,8 @@ import static org.junit.Assert.assertEquals;
 import static org.junit.Assert.assertNotNull;
 import static org.junit.Assert.assertNull;
 
+import java.io.InputStream;
+
 import org.apache.drill.PlanTestBase;
 import org.apache.drill.SingleRowListener;
 import org.apache.drill.exec.exception.SchemaChangeException;
@@ -28,64 +30,102 @@ import org.apache.drill.exec.proto.UserBitShared.QueryType;
 import org.apache.drill.exec.record.RecordBatchLoader;
 import org.apache.drill.exec.rpc.user.QueryDataBatch;
 import org.apache.drill.exec.util.VectorUtil;
+import org.junit.AfterClass;
+import org.junit.BeforeClass;
+import org.junit.Ignore;
 import org.junit.Test;
 import org.junit.experimental.categories.Category;
 import org.ojai.Document;
+import org.ojai.DocumentStream;
+import org.ojai.json.Json;
 
-import com.mapr.db.MapRDB;
+import com.mapr.db.Table;
+import com.mapr.db.impl.MapRDBImpl;
+import com.mapr.db.tests.utils.DBTests;
+import com.mapr.drill.maprdb.tests.MaprDBTestsSuite;
 import com.mapr.tests.annotations.ClusterTest;
 
 @Category(ClusterTest.class)
 public class TestSimpleJson extends BaseJsonTest {
 
+  private static final String TABLE_NAME = "business";
+  private static final String JSON_FILE_URL = "/com/mapr/drill/json/business.json";
+
+  private static boolean tableCreated = false;
+  private static String tablePath;
+  protected String getTablePath() {
+    return tablePath;
+  }
+
+  @BeforeClass
+  public static void setup_TestSimpleJson() throws Exception {
+    try (Table table = DBTests.createOrReplaceTable(TABLE_NAME);
+         InputStream in = MaprDBTestsSuite.getJsonStream(JSON_FILE_URL);
+         DocumentStream stream = Json.newDocumentStream(in)) {
+      tableCreated = true;
+      tablePath = table.getPath().toUri().getPath();
+
+      for (Document document : stream) {
+       table.insert(document, "business_id");
+     }
+     table.flush();
+   }
+  }
+
+  @AfterClass
+  public static void cleanup_TestEncodedFieldPaths() throws Exception {
+    if (tableCreated) {
+      DBTests.deleteTables(TABLE_NAME);
+    }
+  }
+
   @Test
   public void testSelectStar() throws Exception {
-    final String sql = "SELECT\n"
+    final String sql = format("SELECT\n"
         + "  *\n"
         + "FROM\n"
-        + "  hbase.`business` business";
+        + "  %s.`%s` business");
     runSQLAndVerifyCount(sql, 10);
   }
 
   @Test
   public void testSelectId() throws Exception {
     setColumnWidths(new int[] {23});
-    final String sql = "SELECT\n"
+    final String sql = format("SELECT\n"
         + "  _id\n"
         + "FROM\n"
-        + "  hbase.`business` business";
+        + "  %s.`%s` business");
     runSQLAndVerifyCount(sql, 10);
   }
 
   @Test
   public void testSelectNonExistentColumns() throws Exception {
     setColumnWidths(new int[] {23});
-    final String sql = "SELECT\n"
+    final String sql = format("SELECT\n"
             + "  something\n"
             + "FROM\n"
-            + "  hbase.business business limit 5";
+            + "  %s.`%s` business limit 5");
     runSQLAndVerifyCount(sql, 5);
   }
 
   @Test
   public void testKVGen() throws Exception {
     setColumnWidths(new int[] {21, 10, 6});
-    final String sql = "select _id, t.parking[0].`key` K, t.parking[0].`value` V from"
-        + " (select _id, kvgen(b.attributes.Parking) as parking from hbase.business b)"
-        + " as t where t.parking[0].`key` = 'garage' AND t.parking[0].`value` = true";
+    final String sql = format("select _id, t.parking[0].`key` K, t.parking[0].`value` V from"
+        + " (select _id, kvgen(b.attributes.Parking) as parking from %s.`%s` b)"
+        + " as t where t.parking[0].`key` = 'garage' AND t.parking[0].`value` = true");
     runSQLAndVerifyCount(sql, 1);
   }
 
   @Test
   public void testPushdownDisabled() throws Exception {
     setColumnWidths(new int[] {25, 40, 40, 40});
-    final String sql = "SELECT\n"
+    final String sql = format("SELECT\n"
         + "  _id, name, categories, full_address\n"
         + "FROM\n"
-        + "  table(hbase.`business`(type => 'maprdb', enablePushdown => false)) business\n"
+        + "  table(%s.`%s`(type => 'maprdb', enablePushdown => false)) business\n"
         + "WHERE\n"
-        + " name <> 'Sprint'"
-        ;
+        + " name <> 'Sprint'");
     runSQLAndVerifyCount(sql, 9);
 
     final String[] expectedPlan = {"condition=null", "columns=\\[`\\*`\\]"};
@@ -97,15 +137,14 @@ public class TestSimpleJson extends BaseJsonTest {
   @Test
   public void testPushdownStringEqual() throws Exception {
     setColumnWidths(new int[] {25, 40, 40, 40});
-    final String sql = "SELECT\n"
+    final String sql = format("SELECT\n"
         + "  _id, name, business.hours.Monday.`open`, categories[1], years[2], full_address\n"
         + "FROM\n"
-        + "  hbase.`business` business\n"
+        + "  %s.`%s` business\n"
         + "WHERE\n"
-        + " name = 'Sprint'"
-        ;
+        + " name = 'Sprint'");
 
-    final Document queryResult = MapRDB.newDocument();
+    final Document queryResult = MapRDBImpl.newDocument();
     SingleRowListener listener = new SingleRowListener() {
       @Override
       protected void rowArrived(QueryDataBatch result) {
@@ -141,13 +180,12 @@ public class TestSimpleJson extends BaseJsonTest {
   @Test
   public void testPushdownStringLike() throws Exception {
     setColumnWidths(new int[] {25, 40, 40, 40});
-    final String sql = "SELECT\n"
+    final String sql = format("SELECT\n"
         + "  _id, name, categories, full_address\n"
         + "FROM\n"
-        + "  hbase.`business` business\n"
+        + "  %s.`%s` business\n"
         + "WHERE\n"
-        + " name LIKE 'S%'"
-        ;
+        + " name LIKE 'S%%'");
     runSQLAndVerifyCount(sql, 3);
 
     final String[] expectedPlan = {"condition=\\(name MATCHES \"\\^\\\\\\\\QS\\\\\\\\E\\.\\*\\$\"\\)"};
@@ -159,13 +197,12 @@ public class TestSimpleJson extends BaseJsonTest {
   @Test
   public void testPushdownStringNotEqual() throws Exception {
     setColumnWidths(new int[] {25, 40, 40, 40});
-    final String sql = "SELECT\n"
+    final String sql = format("SELECT\n"
         + "  _id, name, categories, full_address\n"
         + "FROM\n"
-        + "  hbase.`business` business\n"
+        + "  %s.`%s` business\n"
         + "WHERE\n"
-        + " name <> 'Sprint'"
-        ;
+        + " name <> 'Sprint'");
     runSQLAndVerifyCount(sql, 9);
 
     final String[] expectedPlan = {"condition=\\(name != \"Sprint\"\\)", "columns=\\[`name`, `_id`, `categories`, `full_address`\\]"};
@@ -177,13 +214,12 @@ public class TestSimpleJson extends BaseJsonTest {
   @Test
   public void testPushdownLongEqual() throws Exception {
     setColumnWidths(new int[] {25, 40, 40, 40});
-    final String sql = "SELECT\n"
+    final String sql = format("SELECT\n"
         + "  _id, name, categories, full_address\n"
         + "FROM\n"
-        + "  hbase.`business` business\n"
+        + "  %s.`%s` business\n"
         + "WHERE\n"
-        + " zip = 85260"
-        ;
+        + " zip = 85260");
     runSQLAndVerifyCount(sql, 1);
 
     final String[] expectedPlan = {"condition=\\(zip = \\{\"\\$numberLong\":85260\\}\\)"};
@@ -195,15 +231,14 @@ public class TestSimpleJson extends BaseJsonTest {
   @Test
   public void testCompositePredicate() throws Exception {
     setColumnWidths(new int[] {25, 40, 40, 40});
-    final String sql = "SELECT\n"
+    final String sql = format("SELECT\n"
         + "  _id, name, categories, full_address\n"
         + "FROM\n"
-        + "  hbase.`business` business\n"
+        + "  %s.`%s` business\n"
         + "WHERE\n"
         + " zip = 85260\n"
         + " OR\n"
-        + " city = 'Las Vegas'"
-        ;
+        + " city = 'Las Vegas'");
     runSQLAndVerifyCount(sql, 4);
 
     final String[] expectedPlan = {"condition=\\(\\(zip = \\{\"\\$numberLong\":85260\\}\\) or \\(city = \"Las Vegas\"\\)\\)"};
@@ -215,13 +250,12 @@ public class TestSimpleJson extends BaseJsonTest {
   @Test
   public void testPruneScanRange() throws Exception {
     setColumnWidths(new int[] {25, 40, 40, 40});
-    final String sql = "SELECT\n"
+    final String sql = format("SELECT\n"
         + "  _id, name, categories, full_address\n"
         + "FROM\n"
-        + "  hbase.`business` business\n"
+        + "  %s.`%s` business\n"
         + "WHERE\n"
-        + " _id = 'jFTZmywe7StuZ2hEjxyA'"
-        ;
+        + " _id = 'jFTZmywe7StuZ2hEjxyA'");
     runSQLAndVerifyCount(sql, 1);
 
     final String[] expectedPlan = {"condition=\\(_id = \"jFTZmywe7StuZ2hEjxyA\"\\)"};
@@ -231,17 +265,17 @@ public class TestSimpleJson extends BaseJsonTest {
   }
 
   @Test
+  @Ignore("Bug 27981")
   public void testPruneScanRangeAndPushDownCondition() throws Exception {
     // XXX/TODO:
     setColumnWidths(new int[] {25, 40, 40, 40});
-    final String sql = "SELECT\n"
+    final String sql = format("SELECT\n"
         + "  _id, name, categories, full_address\n"
         + "FROM\n"
-        + "  hbase.`business` business\n"
+        + "  %s.`%s` business\n"
         + "WHERE\n"
         + " _id = 'jFTZmywe7StuZ2hEjxyA' AND\n"
-        + " name = 'Subway'"
-        ;
+        + " name = 'Subway'");
     runSQLAndVerifyCount(sql, 1);
 
     final String[] expectedPlan = {"condition=\\(\\(_id = \"jFTZmywe7StuZ2hEjxyA\"\\) and \\(name = \"Subway\"\\)\\)"};
@@ -253,12 +287,12 @@ public class TestSimpleJson extends BaseJsonTest {
   @Test
   public void testPushDownOnSubField1() throws Exception {
     setColumnWidths(new int[] {25, 120, 20});
-    final String sql = "SELECT\n"
+    final String sql = format("SELECT\n"
         + "  _id, name, b.attributes.Ambience.touristy attributes\n"
         + "FROM\n"
-        + "  hbase.`business` b\n"
+        + "  %s.`%s` b\n"
         + "WHERE\n"
-        + " b.attributes.Ambience.casual = false";
+        + " b.attributes.Ambience.casual = false");
     runSQLAndVerifyCount(sql, 1);
 
     final String[] expectedPlan = {"condition=\\(attributes.Ambience.casual = false\\)"};
@@ -270,13 +304,12 @@ public class TestSimpleJson extends BaseJsonTest {
   @Test
   public void testPushDownOnSubField2() throws Exception {
     setColumnWidths(new int[] {25, 40, 40, 40});
-    final String sql = "SELECT\n"
+    final String sql = format("SELECT\n"
         + "  _id, name, b.attributes.Attire attributes\n"
         + "FROM\n"
-        + "  hbase.`business` b\n"
+        + "  %s.`%s` b\n"
         + "WHERE\n"
-        + " b.attributes.Attire = 'casual'"
-        ;
+        + " b.attributes.Attire = 'casual'");
     runSQLAndVerifyCount(sql, 4);
 
     final String[] expectedPlan = {"condition=\\(attributes.Attire = \"casual\"\\)"};
@@ -288,13 +321,12 @@ public class TestSimpleJson extends BaseJsonTest {
   public void testPushDownIsNull() throws Exception {
     setColumnWidths(new int[] {25, 40, 40, 40});
 
-    final String sql = "SELECT\n"
+    final String sql = format("SELECT\n"
         + "  _id, name, attributes\n"
         + "FROM\n"
-        + "  hbase.`business` business\n"
+        + "  %s.`%s` business\n"
         + "WHERE\n"
-        + " business.attributes.Ambience.casual IS NULL"
-        ;
+        + " business.attributes.Ambience.casual IS NULL");
     runSQLAndVerifyCount(sql, 7);
 
     final String[] expectedPlan = {"condition=\\(attributes.Ambience.casual = null\\)"};
@@ -307,13 +339,12 @@ public class TestSimpleJson extends BaseJsonTest {
   public void testPushDownIsNotNull() throws Exception {
     setColumnWidths(new int[] {25, 75, 75, 50});
 
-    final String sql = "SELECT\n"
+    final String sql = format("SELECT\n"
         + "  _id, name, b.attributes.Parking\n"
         + "FROM\n"
-        + "  hbase.`business` b\n"
+        + "  %s.`%s` b\n"
         + "WHERE\n"
-        + " b.attributes.Ambience.casual IS NOT NULL"
-        ;
+        + " b.attributes.Ambience.casual IS NOT NULL");
     runSQLAndVerifyCount(sql, 3);
 
     final String[] expectedPlan = {"condition=\\(attributes.Ambience.casual != null\\)"};
@@ -325,13 +356,12 @@ public class TestSimpleJson extends BaseJsonTest {
   @Test
   public void testPushDownOnSubField3() throws Exception {
     setColumnWidths(new int[] {25, 40, 40, 40});
-    final String sql = "SELECT\n"
+    final String sql = format("SELECT\n"
         + "  _id, name, b.attributes.`Accepts Credit Cards` attributes\n"
         + "FROM\n"
-        + "  hbase.`business` b\n"
+        + "  %s.`%s` b\n"
         + "WHERE\n"
-        + " b.attributes.`Accepts Credit Cards` IS NULL"
-        ;
+        + " b.attributes.`Accepts Credit Cards` IS NULL");
     runSQLAndVerifyCount(sql, 3);
 
     final String[] expectedPlan = {"condition=\\(attributes.Accepts Credit Cards = null\\)"};
@@ -342,13 +372,12 @@ public class TestSimpleJson extends BaseJsonTest {
 
   @Test
   public void testPushDownLong() throws Exception {
-    final String sql = "SELECT\n"
+    final String sql = format("SELECT\n"
         + "  *\n"
         + "FROM\n"
-        + "  hbase.`business` business\n"
+        + "  %s.`%s` business\n"
         + "WHERE\n"
-        + " stars > 4.0"
-        ;
+        + " stars > 4.0");
     runSQLAndVerifyCount(sql, 2);
 
     final String[] expectedPlan = {"condition=\\(stars > 4\\)"};
@@ -359,14 +388,13 @@ public class TestSimpleJson extends BaseJsonTest {
 
   @Test
   public void testPushDownSubField4() throws Exception {
-    final String sql = "SELECT\n"
+    final String sql = format("SELECT\n"
         + "  *\n"
         + "FROM\n"
-        + "  hbase.`business` business\n"
+        + "  %s.`%s` business\n"
         + "WHERE\n"
         + " business.attributes.`Good For`.lunch = true AND"
-        + " stars > 4.1"
-        ;
+        + " stars > 4.1");
     runSQLAndVerifyCount(sql, 1);
 
     final String[] expectedPlan = {"condition=\\(\\(attributes.Good For.lunch = true\\) and \\(stars > 4.1\\)\\)"};
@@ -378,13 +406,12 @@ public class TestSimpleJson extends BaseJsonTest {
 
   @Test
   public void testPushDownSubField5() throws Exception {
-    final String sql = "SELECT\n"
+    final String sql = format("SELECT\n"
         + "  *\n"
         + "FROM\n"
-        + "  hbase.`business` business\n"
+        + "  %s.`%s` business\n"
         + "WHERE\n"
-        + " business.hours.Tuesday.`open` < TIME '10:30:00'"
-        ;
+        + " business.hours.Tuesday.`open` < TIME '10:30:00'");
     runSQLAndVerifyCount(sql, 1);
 
     final String[] expectedPlan = {"condition=\\(hours.Tuesday.open < \\{\"\\$time\":\"10:30:00\"\\}\\)"};
@@ -395,13 +422,12 @@ public class TestSimpleJson extends BaseJsonTest {
 
   @Test
   public void testPushDownSubField6() throws Exception {
-    final String sql = "SELECT\n"
+    final String sql = format("SELECT\n"
         + "  *\n"
         + "FROM\n"
-        + "  hbase.`business` business\n"
+        + "  %s.`%s` business\n"
         + "WHERE\n"
-        + " business.hours.Sunday.`close` > TIME '20:30:00'"
-        ;
+        + " business.hours.Sunday.`close` > TIME '20:30:00'");
     runSQLAndVerifyCount(sql, 3);
 
     final String[] expectedPlan = {"condition=\\(hours.Sunday.close > \\{\"\\$time\":\"20:30:00\"\\}\\)"};
@@ -413,13 +439,12 @@ public class TestSimpleJson extends BaseJsonTest {
   @Test
   public void testPushDownSubField7() throws Exception {
     setColumnWidths(new int[] {25, 40, 25, 45});
-    final String sql = "SELECT\n"
+    final String sql = format("SELECT\n"
         + "  _id, name, start_date, last_update\n"
         + "FROM\n"
-        + "  hbase.`business` business\n"
+        + "  %s.`%s` business\n"
         + "WHERE\n"
-        + " business.`start_date` = DATE '2012-07-14'"
-        ;
+        + " business.`start_date` = DATE '2012-07-14'");
     runSQLAndVerifyCount(sql, 1);
 
     final String[] expectedPlan = {"condition=\\(start_date = \\{\"\\$dateDay\":\"2012-07-14\"\\}\\)"};
@@ -431,19 +456,34 @@ public class TestSimpleJson extends BaseJsonTest {
   @Test
   public void testPushDownSubField8() throws Exception {
     setColumnWidths(new int[] {25, 40, 25, 45});
-    final String sql = "SELECT\n"
+    final String sql = format("SELECT\n"
         + "  _id, name, start_date, last_update\n"
         + "FROM\n"
-        + "  hbase.`business` business\n"
+        + "  %s.`%s` business\n"
         + "WHERE\n"
-        + " business.`last_update` = TIMESTAMP '2012-10-20 07:42:46'"
-        ;
+        + " business.`last_update` = TIMESTAMP '2012-10-20 07:42:46'");
     runSQLAndVerifyCount(sql, 1);
 
-    final String[] expectedPlan = {"condition=null"};
-    final String[] excludedPlan = {"condition=\\(last_update = \\{\"\\$date\":\"2012-10-20T07:42:46.000Z\"\\}\\)"};
+    final String[] expectedPlan = {"condition=\\(last_update = \\{\"\\$date\":\"2012-10-20T07:42:46.000Z\"\\}\\)"};
+    final String[] excludedPlan = {};
 
     PlanTestBase.testPlanMatchingPatterns(sql, expectedPlan, excludedPlan);
   }
 
+  @Test
+  public void testLimit() throws Exception {
+    final String sql = format("SELECT\n"
+        + "  _id, name, start_date, last_update\n"
+        + "FROM\n"
+        + "  %s.`%s` business\n"
+        + "limit 1"
+    );
+
+    final String[] expectedPlan = {"JsonTableGroupScan.*limit=1"};
+    final String[] excludedPlan = {};
+
+    PlanTestBase.testPlanMatchingPatterns(sql, expectedPlan, excludedPlan);
+    runSQLAndVerifyCount(sql, 1);
+  }
+
 }
diff --git a/contrib/format-maprdb/src/test/resources/json/business.json b/contrib/format-maprdb/src/test/resources/com/mapr/drill/json/business.json
similarity index 100%
rename from contrib/format-maprdb/src/test/resources/json/business.json
rename to contrib/format-maprdb/src/test/resources/com/mapr/drill/json/business.json
diff --git a/contrib/format-maprdb/src/test/resources/com/mapr/drill/json/encoded_fields_userdata.json b/contrib/format-maprdb/src/test/resources/com/mapr/drill/json/encoded_fields_userdata.json
new file mode 100644
index 0000000..3295ba5
--- /dev/null
+++ b/contrib/format-maprdb/src/test/resources/com/mapr/drill/json/encoded_fields_userdata.json
@@ -0,0 +1,5 @@
+{"_id":"user001", "age":43, "name": {"first":"Sam", "last":"Harris"}, "codes": [1, "x@#ss2", 9.0], "data": {"salary": {"$numberLong": 125000}}}
+{"_id":"user002", "age":12, "name": {"first":"Leon", "last":"Russel"}, "codes": ["JA32S"], "data": {"salary": "170200"}}
+{"_id":"user003", "age":"87", "name": {"first":"David", "last":"Bowie"}, "codes": [236.35], "data": {"salary": {"$numberLong": 185200}}}
+{"_id":"user004", "age":56, "name": {"first":"Bob", "last":"Dylan"}, "codes": [{"$date": "1989-07-4"}, 0], "data": {"salary": {"$numberLong": 136900}}}
+{"_id":"user005", "age":54, "name": {"first":"David", "last":"Ackert"}, "codes": [{"a": 25, "b": "yas"}], "data": {"salary": {"$numberLong": 112850}}}
diff --git a/contrib/format-maprdb/src/test/resources/hbase-site.xml b/contrib/format-maprdb/src/test/resources/core-site.xml
similarity index 90%
rename from contrib/format-maprdb/src/test/resources/hbase-site.xml
rename to contrib/format-maprdb/src/test/resources/core-site.xml
index ec26f70..7ea9704 100644
--- a/contrib/format-maprdb/src/test/resources/hbase-site.xml
+++ b/contrib/format-maprdb/src/test/resources/core-site.xml
@@ -25,4 +25,9 @@
     <value>*:/tmp/</value>
   </property>
 
+  <property>
+    <name>fs.mapr.bailout.on.library.mismatch</name>
+    <value>false</value>
+  </property>
+
 </configuration>
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/base/AbstractGroupScan.java b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/base/AbstractGroupScan.java
index cddf3fe..a4eb369 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/base/AbstractGroupScan.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/base/AbstractGroupScan.java
@@ -72,6 +72,12 @@ public abstract class AbstractGroupScan extends AbstractBase implements GroupSca
 
   @Override
   @JsonIgnore
+  public boolean isDistributed() {
+    return getMaxParallelizationWidth() > 1 ? true : false;
+  }
+
+  @Override
+  @JsonIgnore
   public int getMinParallelizationWidth() {
     return 1;
   }
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/base/GroupScan.java b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/base/GroupScan.java
index 655e3a9..263ef05 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/base/GroupScan.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/base/GroupScan.java
@@ -55,6 +55,9 @@ public interface GroupScan extends Scan, HasAffinity{
   @JsonIgnore
   int getMaxParallelizationWidth();
 
+  @JsonIgnore
+  boolean isDistributed();
+
   /**
    * At minimum, the GroupScan requires these many fragments to run.
    * Currently, this is used in {@link org.apache.drill.exec.planner.fragment.SimpleParallelizer}
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 c474f2e..e1cc661 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,6 +18,9 @@
 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;
@@ -27,18 +30,28 @@ 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 {
 
@@ -94,6 +107,121 @@ 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> {
@@ -123,6 +251,69 @@ 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);
@@ -148,4 +339,44 @@ 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/test/java/org/apache/drill/test/BaseTestQuery.java b/exec/java-exec/src/test/java/org/apache/drill/test/BaseTestQuery.java
index e85b2c7..71aa240 100644
--- a/exec/java-exec/src/test/java/org/apache/drill/test/BaseTestQuery.java
+++ b/exec/java-exec/src/test/java/org/apache/drill/test/BaseTestQuery.java
@@ -480,6 +480,7 @@ public class BaseTestQuery extends ExecTest {
 
   public static class SilentListener implements UserResultsListener {
     private final AtomicInteger count = new AtomicInteger();
+    private QueryId queryId;
 
     @Override
     public void submissionFailed(UserException ex) {
@@ -501,8 +502,17 @@ public class BaseTestQuery extends ExecTest {
     }
 
     @Override
-    public void queryIdArrived(QueryId queryId) {}
+    public void queryIdArrived(QueryId queryId) {
+      this.queryId = queryId;
+    }
+
+    public QueryId getQueryId() {
+      return queryId;
+    }
 
+    public int getRowCount() {
+      return count.get();
+    }
   }
 
   protected void setColumnWidth(int columnWidth) {
diff --git a/pom.xml b/pom.xml
index dbd8caf..a75e251 100644
--- a/pom.xml
+++ b/pom.xml
@@ -53,8 +53,8 @@
     <jackson.version>2.9.5</jackson.version>
     <jackson.databind.version>2.9.5</jackson.databind.version>
     <zookeeper.version>3.4.12</zookeeper.version>
-    <mapr.release.version>5.2.1-mapr</mapr.release.version>
-    <ojai.version>1.1</ojai.version>
+    <mapr.release.version>6.0.1-mapr</mapr.release.version>
+    <ojai.version>2.0.1-mapr-1804</ojai.version>
     <kerby.version>1.0.0-RC2</kerby.version>
     <findbugs.version>3.0.0</findbugs.version>
     <netty.tcnative.classifier />


[drill] 01/08: DRILL-6381: (Part 1) Secondary Index framework

Posted by am...@apache.org.
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 0abcbe3f36bf6c0a2b5fe07a778d201ead8dd2ce
Author: rebase <bu...@mapr.com>
AuthorDate: Mon Feb 12 14:10:56 2018 -0800

    DRILL-6381: (Part 1) Secondary Index framework
    
      1. Secondary Index planning interfaces and abstract classes like DBGroupScan, DbSubScan, IndexDecriptor etc.
      2. Statistics and Cost model interfaces/classes: PluginCost, Statistics, StatisticsPayload, AbstractIndexStatistics
      3. ScanBatch and RecordReader to support repeatable scan
      4. Secondary Index execution related interfaces: RangePartitionSender, RowKeyJoin, PartitionFunction
      5. MD-3979: Query using cast index plan fails with NPE
    
    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/impl/ScanBatch.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/logical/DrillTable.java
    	protocol/src/main/java/org/apache/drill/exec/proto/UserBitShared.java
    	protocol/src/main/java/org/apache/drill/exec/proto/beans/CoreOperatorType.java
    	protocol/src/main/protobuf/UserBitShared.proto
---
 .../java/org/apache/drill/exec/ExecConstants.java  |   2 +
 .../exec/physical/base/AbstractDbGroupScan.java    |  95 +++++++
 .../exec/physical/base/AbstractDbSubScan.java      |  37 +++
 .../physical/base/AbstractPhysicalVisitor.java     |  12 +
 .../drill/exec/physical/base/DbGroupScan.java      | 129 +++++++++
 .../apache/drill/exec/physical/base/DbSubScan.java |  43 +++
 .../drill/exec/physical/base/IndexGroupScan.java   |  76 ++++++
 .../drill/exec/physical/base/PhysicalVisitor.java  |   4 +
 ...{RangeSender.java => RangePartitionSender.java} |  52 ++--
 .../apache/drill/exec/physical/impl/ScanBatch.java |  15 +-
 .../drill/exec/physical/impl/join/RowKeyJoin.java  |  79 ++++++
 .../drill/exec/planner/common/DrillRelOptUtil.java |  17 ++
 .../apache/drill/exec/planner/cost/PluginCost.java |  79 ++++++
 .../planner/index/AbstractIndexCollection.java     |  96 +++++++
 .../planner/index/AbstractIndexDescriptor.java     |  74 ++++++
 .../planner/index/AbstractIndexStatistics.java     |  51 ++++
 .../drill/exec/planner/index/CollationContext.java |  37 +++
 .../exec/planner/index/DrillIndexCollection.java   |  75 ++++++
 .../exec/planner/index/DrillIndexDefinition.java   | 278 ++++++++++++++++++++
 .../exec/planner/index/DrillIndexDescriptor.java   | 110 ++++++++
 .../exec/planner/index/FunctionalIndexInfo.java    |  85 ++++++
 .../drill/exec/planner/index/IndexCallContext.java |  76 ++++++
 .../drill/exec/planner/index/IndexCollection.java  |  99 +++++++
 .../drill/exec/planner/index/IndexDefinition.java  | 105 ++++++++
 .../drill/exec/planner/index/IndexDescriptor.java  |  68 +++++
 .../drill/exec/planner/index/IndexDiscover.java    |  23 ++
 .../exec/planner/index/IndexDiscoverBase.java      | 110 ++++++++
 .../exec/planner/index/IndexDiscoverable.java      |  37 +++
 .../drill/exec/planner/index/IndexGroup.java       |  63 +++++
 .../drill/exec/planner/index/IndexProperties.java  |  64 +++++
 .../drill/exec/planner/index/IndexStatistics.java  |  36 +++
 .../exec/planner/index/IndexableExprMarker.java    | 262 +++++++++++++++++++
 .../index/InvalidIndexDefinitionException.java     |  27 ++
 .../drill/exec/planner/index/Statistics.java       |  66 +++++
 .../exec/planner/index/StatisticsPayload.java      |  24 ++
 .../drill/exec/planner/logical/DrillTable.java     |   4 +
 .../exec/planner/physical/PartitionFunction.java   |  56 ++++
 .../drill/exec/record/AbstractRecordBatch.java     |   2 +-
 .../drill/exec/store/AbstractRecordReader.java     |   5 +
 .../org/apache/drill/exec/store/RecordReader.java  |   8 +
 .../drill/exec/util/EncodedSchemaPathSet.java      | 291 +++++++++++++++++++++
 .../org/apache/drill/exec/proto/UserBitShared.java |  75 +++---
 .../drill/exec/proto/beans/CoreOperatorType.java   |   8 +-
 protocol/src/main/protobuf/UserBitShared.proto     |   3 +-
 44 files changed, 2893 insertions(+), 65 deletions(-)

diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/ExecConstants.java b/exec/java-exec/src/main/java/org/apache/drill/exec/ExecConstants.java
index 21e16eb..cb0fc5c 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/ExecConstants.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/ExecConstants.java
@@ -807,6 +807,8 @@ public final class ExecConstants {
    */
   public static final String ENABLE_ITERATOR_VALIDATION = "drill.exec.debug.validate_iterators";
 
+  public static final String QUERY_ROWKEYJOIN_BATCHSIZE_KEY = "exec.query.rowkeyjoin_batchsize";
+  public static final PositiveLongValidator QUERY_ROWKEYJOIN_BATCHSIZE = new PositiveLongValidator(QUERY_ROWKEYJOIN_BATCHSIZE_KEY, Long.MAX_VALUE, null);
   /**
    * When iterator validation is enabled, additionally validates the vectors in
    * each batch passed to each iterator.
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/base/AbstractDbGroupScan.java b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/base/AbstractDbGroupScan.java
new file mode 100644
index 0000000..42e4bb9
--- /dev/null
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/base/AbstractDbGroupScan.java
@@ -0,0 +1,95 @@
+/*
+ * 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.base;
+
+import java.util.List;
+
+import org.apache.drill.common.expression.FieldReference;
+import org.apache.drill.common.expression.SchemaPath;
+import org.apache.drill.common.logical.StoragePluginConfig;
+import org.apache.calcite.rel.RelNode;
+import org.apache.drill.exec.planner.index.IndexCollection;
+import org.apache.drill.exec.planner.cost.PluginCost;
+import org.apache.drill.exec.planner.physical.PartitionFunction;
+import org.apache.drill.exec.store.AbstractStoragePlugin;
+
+public abstract class AbstractDbGroupScan extends AbstractGroupScan implements DbGroupScan {
+  static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(AbstractDbGroupScan.class);
+
+  private static final String ROW_KEY = "_id";
+  private static final SchemaPath ROW_KEY_PATH = SchemaPath.getSimplePath(ROW_KEY);
+
+  public AbstractDbGroupScan(String userName) {
+    super(userName);
+  }
+
+  public AbstractDbGroupScan(AbstractDbGroupScan that) {
+    super(that);
+  }
+
+  public abstract AbstractStoragePlugin getStoragePlugin();
+
+  public abstract StoragePluginConfig getStorageConfig();
+
+  public abstract List<SchemaPath> getColumns();
+
+  @Override
+  public boolean supportsSecondaryIndex() {
+    return false;
+  }
+
+  @Override
+  public IndexCollection getSecondaryIndexCollection(RelNode scanrel) {
+    return null;
+  }
+
+  @Override
+  public boolean supportsRestrictedScan() {
+    return false;
+  }
+
+  @Override
+  public boolean isRestrictedScan() {
+    return false;
+  }
+
+  @Override
+  public DbGroupScan getRestrictedScan(List<SchemaPath> columns) {
+    return null;
+  }
+
+  @Override
+  public String getRowKeyName() {
+    return ROW_KEY;
+  }
+
+  @Override
+  public SchemaPath getRowKeyPath() {
+    return ROW_KEY_PATH;
+  }
+
+  @Override
+  public PartitionFunction getRangePartitionFunction(List<FieldReference> refList) {
+    throw new UnsupportedOperationException();
+  }
+
+  @Override
+  public PluginCost getPluginCostModel() {
+    return null;
+  }
+}
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/base/AbstractDbSubScan.java b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/base/AbstractDbSubScan.java
new file mode 100644
index 0000000..caa5831
--- /dev/null
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/base/AbstractDbSubScan.java
@@ -0,0 +1,37 @@
+/*
+ * 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.base;
+
+import org.apache.drill.exec.physical.impl.join.RowKeyJoin;
+
+public abstract class AbstractDbSubScan extends AbstractSubScan implements DbSubScan {
+
+  public AbstractDbSubScan(String userName) {
+    super(userName);
+  }
+
+  public boolean isRestrictedSubScan() {
+    return false;
+  }
+
+  @Override
+  public void addJoinForRestrictedSubScan(RowKeyJoin batch) {
+    throw new UnsupportedOperationException();
+  }
+
+}
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 340c303..ca82ca6 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
@@ -22,6 +22,7 @@ import org.apache.drill.exec.physical.config.Filter;
 import org.apache.drill.exec.physical.config.FlattenPOP;
 import org.apache.drill.exec.physical.config.HashAggregate;
 import org.apache.drill.exec.physical.config.HashPartitionSender;
+import org.apache.drill.exec.physical.config.HashToRandomExchange;
 import org.apache.drill.exec.physical.config.IteratorValidator;
 import org.apache.drill.exec.physical.config.LateralJoinPOP;
 import org.apache.drill.exec.physical.config.Limit;
@@ -29,6 +30,7 @@ import org.apache.drill.exec.physical.config.MergingReceiverPOP;
 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.Screen;
 import org.apache.drill.exec.physical.config.SingleSender;
 import org.apache.drill.exec.physical.config.Sort;
@@ -157,6 +159,16 @@ public abstract class AbstractPhysicalVisitor<T, X, E extends Throwable> impleme
   }
 
   @Override
+  public T visitHashPartitionSender(HashToRandomExchange op, X value) throws E {
+    return visitExchange(op, value);
+  }
+
+  @Override
+  public T visitRangePartitionSender(RangePartitionSender op, X value) throws E {
+    return visitSender(op, value);
+  }
+
+  @Override
   public T visitBroadcastSender(BroadcastSender op, X value) throws E {
     return visitSender(op, value);
   }
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/base/DbGroupScan.java b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/base/DbGroupScan.java
new file mode 100644
index 0000000..e16fba1
--- /dev/null
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/base/DbGroupScan.java
@@ -0,0 +1,129 @@
+/*
+ * 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.base;
+
+import org.apache.calcite.rel.RelNode;
+import org.apache.drill.common.expression.FieldReference;
+import org.apache.calcite.rex.RexNode;
+import org.apache.drill.common.expression.SchemaPath;
+import org.apache.drill.exec.planner.index.IndexCollection;
+import com.fasterxml.jackson.annotation.JsonIgnore;
+import org.apache.drill.exec.planner.cost.PluginCost;
+import org.apache.drill.exec.planner.physical.PartitionFunction;
+import org.apache.drill.exec.planner.index.Statistics;
+
+import java.util.List;
+
+/**
+ * A DbGroupScan operator represents the scan associated with a database. The underlying
+ * database may support secondary indexes, so there are interface methods for indexes.
+ */
+public interface DbGroupScan extends GroupScan {
+
+
+  @JsonIgnore
+  public boolean supportsSecondaryIndex();
+
+  /**
+   * Get the index collection associated with this table if any
+   */
+  @JsonIgnore
+  public IndexCollection getSecondaryIndexCollection(RelNode scan);
+
+  /**
+   * Set the artificial row count after applying the {@link RexNode} condition
+   * @param condition
+   * @param count
+   * @param capRowCount
+   */
+  @JsonIgnore
+  public void setRowCount(RexNode condition, double count, double capRowCount);
+
+  /**
+   * Get the row count after applying the {@link RexNode} condition
+   * @param condition, filter to apply
+   * @param scanRel, the current scan rel
+   * @return row count post filtering
+   */
+  @JsonIgnore
+  public double getRowCount(RexNode condition, RelNode scanRel);
+
+  /**
+   * Get the statistics for this {@link DbGroupScan}
+   * @return the {@link Statistics} for this Scan
+   */
+  @JsonIgnore
+  public Statistics getStatistics();
+
+  public List<SchemaPath> getColumns();
+
+  public void setCostFactor(double sel);
+
+  @JsonIgnore
+  boolean isIndexScan();
+
+  /**
+   * Whether this DbGroupScan supports creating a restricted (skip) scan
+   * @return true if restricted scan is supported, false otherwise
+   */
+  @JsonIgnore
+  boolean supportsRestrictedScan();
+
+  /**
+   * Whether this DbGroupScan is itself a restricted scan
+   * @return true if this DbGroupScan is itself a restricted scan, false otherwise
+   */
+  @JsonIgnore
+  boolean isRestrictedScan();
+
+  /**
+   * If this DbGroupScan supports restricted scan, create a restricted scan from this DbGroupScan.
+   * @param columns
+   * @return a non-null DbGroupScan if restricted scan is supported, null otherwise
+   */
+  @JsonIgnore
+  DbGroupScan getRestrictedScan(List<SchemaPath> columns);
+
+  @JsonIgnore
+  String getRowKeyName();
+
+  @JsonIgnore
+  String getIndexHint();
+
+  @JsonIgnore
+  SchemaPath getRowKeyPath();
+
+  /**
+   * Get a partition function instance for range based partitioning
+   * @param refList a list of FieldReference exprs that are participating in the range partitioning
+   * @return instance of a partitioning function
+   */
+  @JsonIgnore
+  PartitionFunction getRangePartitionFunction(List<FieldReference> refList);
+
+  /**
+   * Get the format plugin cost model. The cost model will provide cost factors such as seq. scan cost,
+   * random scan cost, block size.
+   * @return a PluginCost cost model
+   */
+  @JsonIgnore
+  PluginCost getPluginCostModel();
+
+  @JsonIgnore
+  boolean isFilterPushedDown();
+}
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/base/DbSubScan.java b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/base/DbSubScan.java
new file mode 100644
index 0000000..874468d
--- /dev/null
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/base/DbSubScan.java
@@ -0,0 +1,43 @@
+/*
+ * 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.base;
+
+import org.apache.drill.exec.physical.impl.join.RowKeyJoin;
+
+import com.fasterxml.jackson.annotation.JsonIgnore;
+
+
+public interface DbSubScan extends SubScan {
+
+  /**
+   * Whether this subscan is a restricted (skip) subscan
+   * @return true if this subscan is a restricted subscan, false otherwise
+   */
+  @JsonIgnore
+  boolean isRestrictedSubScan();
+
+  /**
+   * For a restricted sub-scan, this method allows associating a (hash)join instance.  A subscan within a minor
+   * fragment must have a corresponding (hash)join batch instance from which it will retrieve its set of
+   * rowkeys to perform the restricted scan.
+   * @param batch
+   */
+  @JsonIgnore
+  void addJoinForRestrictedSubScan(RowKeyJoin batch);
+
+}
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/base/IndexGroupScan.java b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/base/IndexGroupScan.java
new file mode 100644
index 0000000..1047e82
--- /dev/null
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/base/IndexGroupScan.java
@@ -0,0 +1,76 @@
+/*
+ * 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.base;
+
+import com.fasterxml.jackson.annotation.JsonIgnore;
+
+import org.apache.calcite.rel.RelNode;
+import org.apache.calcite.rex.RexNode;
+import org.apache.drill.common.expression.SchemaPath;
+import org.apache.drill.exec.planner.index.Statistics;
+
+
+import java.util.List;
+
+/**
+ * An IndexGroupScan operator represents the scan associated with an Index.
+ */
+public interface IndexGroupScan extends GroupScan {
+
+  /**
+   * Get the column ordinal of the rowkey column from the output schema of the IndexGroupScan
+   * @return
+   */
+  @JsonIgnore
+  public int getRowKeyOrdinal();
+
+  /**
+   * Set the artificial row count after applying the {@link RexNode} condition
+   * Mainly used for debugging
+   * @param condition
+   * @param count
+   * @param capRowCount
+   */
+  @JsonIgnore
+  public void setRowCount(RexNode condition, double count, double capRowCount);
+
+  /**
+   * Get the row count after applying the {@link RexNode} condition
+   * @param condition, filter to apply
+   * @return row count post filtering
+   */
+  @JsonIgnore
+  public double getRowCount(RexNode condition, RelNode scanRel);
+
+  /**
+   * Set the statistics for {@link IndexGroupScan}
+   * @param statistics
+   */
+  @JsonIgnore
+  public void setStatistics(Statistics statistics);
+
+  @JsonIgnore
+  public void setColumns(List<SchemaPath> columns);
+
+  @JsonIgnore
+  public List<SchemaPath> getColumns();
+
+  @JsonIgnore
+  public void setParallelizationWidth(int width);
+
+}
\ No newline at end of file
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 f2e53eb..1bb1545 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
@@ -22,6 +22,7 @@ import org.apache.drill.exec.physical.config.Filter;
 import org.apache.drill.exec.physical.config.FlattenPOP;
 import org.apache.drill.exec.physical.config.HashAggregate;
 import org.apache.drill.exec.physical.config.HashPartitionSender;
+import org.apache.drill.exec.physical.config.HashToRandomExchange;
 import org.apache.drill.exec.physical.config.IteratorValidator;
 import org.apache.drill.exec.physical.config.LateralJoinPOP;
 import org.apache.drill.exec.physical.config.Limit;
@@ -29,6 +30,7 @@ import org.apache.drill.exec.physical.config.MergingReceiverPOP;
 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.Screen;
 import org.apache.drill.exec.physical.config.SingleSender;
 import org.apache.drill.exec.physical.config.Sort;
@@ -73,6 +75,8 @@ public interface PhysicalVisitor<RETURN, EXTRA, EXCEP extends Throwable> {
   public RETURN visitOrderedPartitionSender(OrderedPartitionSender op, EXTRA value) throws EXCEP;
   public RETURN visitUnorderedReceiver(UnorderedReceiver op, EXTRA value) throws EXCEP;
   public RETURN visitMergingReceiver(MergingReceiverPOP op, EXTRA value) throws EXCEP;
+  public RETURN visitHashPartitionSender(HashToRandomExchange op, EXTRA value) throws EXCEP;
+  public RETURN visitRangePartitionSender(RangePartitionSender op, EXTRA value) throws EXCEP;
   public RETURN visitBroadcastSender(BroadcastSender op, EXTRA value) throws EXCEP;
   public RETURN visitScreen(Screen op, EXTRA value) throws EXCEP;
   public RETURN visitSingleSender(SingleSender op, EXTRA value) throws EXCEP;
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/config/RangeSender.java b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/config/RangePartitionSender.java
similarity index 50%
rename from exec/java-exec/src/main/java/org/apache/drill/exec/physical/config/RangeSender.java
rename to exec/java-exec/src/main/java/org/apache/drill/exec/physical/config/RangePartitionSender.java
index 88c3be0..0c0852a 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/config/RangeSender.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/config/RangePartitionSender.java
@@ -17,57 +17,57 @@
  */
 package org.apache.drill.exec.physical.config;
 
-import java.util.Collections;
 import java.util.List;
 
 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.proto.CoordinationProtos.DrillbitEndpoint;
+import org.apache.drill.exec.physical.base.PhysicalVisitor;
+import org.apache.drill.exec.planner.physical.PartitionFunction;
 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;
 
-@JsonTypeName("range-sender")
-public class RangeSender extends AbstractSender{
+@JsonTypeName("range-partition-sender")
+public class RangePartitionSender extends AbstractSender{
 
-  static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(RangeSender.class);
+  static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(RangePartitionSender.class);
 
-  List<EndpointPartition> partitions;
+  // The number of records in the outgoing batch. This is overriding the default value in Partitioner
+  public static final int RANGE_PARTITION_OUTGOING_BATCH_SIZE = (1 << 12) - 1;
+
+  @JsonProperty("partitionFunction")
+  private PartitionFunction partitionFunction;
 
   @JsonCreator
-  public RangeSender(@JsonProperty("receiver-major-fragment") int oppositeMajorFragmentId, @JsonProperty("child") PhysicalOperator child, @JsonProperty("partitions") List<EndpointPartition> partitions) {
-    super(oppositeMajorFragmentId, child, Collections.<MinorFragmentEndpoint>emptyList());
-    this.partitions = partitions;
+  public RangePartitionSender(@JsonProperty("receiver-major-fragment") int oppositeMajorFragmentId,
+                              @JsonProperty("child") PhysicalOperator child,
+                              @JsonProperty("destinations") List<MinorFragmentEndpoint> endpoints,
+                              @JsonProperty("partitionFunction") PartitionFunction partitionFunction) {
+    super(oppositeMajorFragmentId, child, endpoints);
+    this.partitionFunction = partitionFunction;
   }
 
   @Override
   protected PhysicalOperator getNewWithChild(PhysicalOperator child) {
-    return new RangeSender(oppositeMajorFragmentId, child, partitions);
+    return new RangePartitionSender(oppositeMajorFragmentId, child, destinations, partitionFunction);
   }
 
-  public static class EndpointPartition{
-    private final PartitionRange range;
-    private final DrillbitEndpoint endpoint;
+  @JsonProperty("partitionFunction")
+  public PartitionFunction getPartitionFunction() {
+    return partitionFunction;
+  }
 
-    @JsonCreator
-    public EndpointPartition(@JsonProperty("range") PartitionRange range, @JsonProperty("endpoint") DrillbitEndpoint endpoint) {
-      super();
-      this.range = range;
-      this.endpoint = endpoint;
-    }
-    public PartitionRange getRange() {
-      return range;
-    }
-    public DrillbitEndpoint getEndpoint() {
-      return endpoint;
-    }
+  @Override
+  public <T, X, E extends Throwable> T accept(PhysicalVisitor<T, X, E> physicalVisitor, X value) throws E {
+    return physicalVisitor.visitRangePartitionSender(this, value);
   }
 
   @Override
   public int getOperatorType() {
-    return CoreOperatorType.RANGE_SENDER_VALUE;
+    return CoreOperatorType.RANGE_PARTITION_SENDER_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 dc8dd0f..5ccf1c0 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
@@ -83,10 +83,14 @@ public class ScanBatch implements CloseableRecordBatch {
   private final List<Map<String, String>> implicitColumnList;
   private String currentReaderClassName;
   private final RecordBatchStatsContext batchStatsContext;
+
   // Represents last outcome of next(). If an Exception is thrown
   // during the method's execution a value IterOutcome.STOP will be assigned.
   private IterOutcome lastOutcome;
 
+  private List<RecordReader> readerList = null; // needed for repeatable scanners
+  private boolean isRepeatableScan = false;     // needed for repeatable scanners
+
   /**
    *
    * @param context
@@ -137,6 +141,15 @@ public class ScanBatch implements CloseableRecordBatch {
         readers, Collections.<Map<String, String>> emptyList());
   }
 
+  public ScanBatch(PhysicalOperator subScanConfig, FragmentContext context,
+                   List<RecordReader> readerList, boolean isRepeatableScan)
+      throws ExecutionSetupException {
+    this(context, context.newOperatorContext(subScanConfig),
+        readerList, Collections.<Map<String, String>> emptyList());
+    this.readerList = readerList;
+    this.isRepeatableScan = isRepeatableScan;
+  }
+
   @Override
   public FragmentContext getContext() {
     return context;
@@ -255,7 +268,7 @@ public class ScanBatch implements CloseableRecordBatch {
       return false;
     }
     currentReader = readers.next();
-    if (readers.hasNext()) {
+    if (!isRepeatableScan && readers.hasNext()) {
       readers.remove();
     }
     implicitValues = implicitColumns.hasNext() ? implicitColumns.next() : null;
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/join/RowKeyJoin.java b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/join/RowKeyJoin.java
new file mode 100644
index 0000000..7b4dfca
--- /dev/null
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/join/RowKeyJoin.java
@@ -0,0 +1,79 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.drill.exec.physical.impl.join;
+
+import org.apache.commons.lang3.tuple.Pair;
+import org.apache.drill.exec.record.AbstractRecordBatch.BatchState;
+import org.apache.drill.exec.vector.ValueVector;
+
+/**
+ * Interface for a row key join
+ */
+public interface RowKeyJoin {
+
+  /**
+   * Enum for RowKeyJoin internal state.
+   * Possible states are {INITIAL, PROCESSING, DONE}
+   *
+   * Initially RowKeyJoin will be at INITIAL state. Then the state will be transitioned
+   * by the RestrictedJsonRecordReader to PROCESSING as soon as it processes the rows
+   * related to RowKeys. Then RowKeyJoin algorithm sets to INITIAL state when leftStream has no data.
+   * Basically RowKeyJoin calls leftStream multiple times depending upon the rightStream, hence
+   * this transition from PROCESSING to INITIAL. If there is no data from rightStream or OutOfMemory
+   * condition then the state is transitioned to DONE.
+   */
+  public enum RowKeyJoinState {
+    INITIAL, PROCESSING, DONE;
+  }
+
+  /**
+   * Is the next batch of row keys ready to be returned
+   * @return True if ready, false if not
+   */
+  public boolean hasRowKeyBatch();
+
+  /**
+   * Get the next batch of row keys
+   * @return a Pair whose left element is the ValueVector containing the row keys, right
+   *    element is the number of row keys in this batch
+   */
+  public Pair<ValueVector, Integer> nextRowKeyBatch();
+
+
+  /**
+   * Get the current BatchState (this is useful when performing row key join)
+   */
+  public BatchState getBatchState();
+
+  /**
+   * Set the BatchState (this is useful when performing row key join)
+   * @param newState
+   */
+  public void setBatchState(BatchState newState);
+
+  /**
+   * Set the RowKeyJoinState (this is useful for maintaining state for row key join algorithm)
+   * @param newState
+   */
+  public void setRowKeyJoinState(RowKeyJoinState newState);
+
+  /**
+   * Get the current RowKeyJoinState.
+   */
+  public RowKeyJoinState getRowKeyJoinState();
+}
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 96c3112..b39328e 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
@@ -396,4 +396,21 @@ public abstract class DrillRelOptUtil {
       }
     }
   }
+
+  public static boolean isProjectFlatten(RelNode project) {
+
+    assert project instanceof Project : "Rel is NOT an instance of project!";
+
+    for (RexNode rex : project.getChildExps()) {
+      RexNode newExpr = rex;
+      if (rex instanceof RexCall) {
+        RexCall function = (RexCall) rex;
+        String functionName = function.getOperator().getName();
+        if (functionName.equalsIgnoreCase("flatten") ) {
+          return true;
+        }
+      }
+    }
+    return false;
+  }
 }
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/cost/PluginCost.java b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/cost/PluginCost.java
new file mode 100644
index 0000000..d765162
--- /dev/null
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/cost/PluginCost.java
@@ -0,0 +1,79 @@
+package org.apache.drill.exec.planner.cost;
+
+/*
+ * 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.
+ */
+
+
+import org.apache.drill.exec.physical.base.GroupScan;
+
+/**
+ * PluginCost describes the cost factors to be used when costing for the specific storage/format plugin
+ */
+public interface PluginCost {
+  org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(PluginCost.class);
+
+  /**
+   * An interface to check if a parameter provided by user is valid or not.
+   * @param <T> Type of the parameter.
+   */
+  interface CheckValid<T> {
+    boolean isValid(T paramValue);
+  }
+
+  /**
+   * Class which checks whether the provided parameter value is greater than
+   * or equals to a minimum limit.
+   */
+  class greaterThanEquals implements CheckValid<Integer> {
+    private final Integer atleastEqualsTo;
+    public greaterThanEquals(Integer atleast) {
+      atleastEqualsTo = atleast;
+    }
+
+    @Override
+    public boolean isValid(Integer paramValue) {
+      if (paramValue >= atleastEqualsTo &&
+          paramValue <= Integer.MAX_VALUE) {
+        return true;
+      } else {
+        logger.warn("Setting default value as the supplied parameter value is less than {}", paramValue);
+        return false;
+      }
+    }
+  }
+
+  /**
+   * @return the average column size in bytes
+   */
+  int getAverageColumnSize(GroupScan scan);
+
+  /**
+   * @return the block size in bytes
+   */
+  int getBlockSize(GroupScan scan);
+
+  /**
+   * @return the sequential block read cost
+   */
+  int getSequentialBlockReadCost(GroupScan scan);
+
+  /**
+   * @return the random block read cost
+   */
+  int getRandomBlockReadCost(GroupScan scan);
+}
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/index/AbstractIndexCollection.java b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/index/AbstractIndexCollection.java
new file mode 100644
index 0000000..9894b32
--- /dev/null
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/index/AbstractIndexCollection.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.planner.index;
+
+import java.util.Iterator;
+import java.util.List;
+
+import com.fasterxml.jackson.annotation.JsonProperty;
+import org.apache.drill.shaded.guava.com.google.common.collect.Lists;
+
+import org.apache.calcite.rex.RexNode;
+import org.apache.drill.common.expression.SchemaPath;
+
+/**
+ * Abstract base class for Index collection (collection of Index descriptors)
+ *
+ */
+public abstract class AbstractIndexCollection implements IndexCollection, Iterable<IndexDescriptor> {
+
+  static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(AbstractIndexCollection.class);
+  /**
+   * A set of indexes for a particular table
+   */
+  @JsonProperty
+  protected List<IndexDescriptor> indexes;
+
+  public AbstractIndexCollection() {
+    indexes = Lists.newArrayList();
+  }
+
+  @Override
+  public boolean addIndex(IndexDescriptor index) {
+    return indexes.add(index);
+  }
+
+  @Override
+  public boolean removeIndex(IndexDescriptor index) {
+    return indexes.remove(index);
+  }
+
+  @Override
+  public void clearAll() {
+    indexes.clear();
+  }
+
+  @Override
+  public boolean supportsIndexSelection() {
+    return false;
+  }
+
+  @Override
+  public double getRows(RexNode indexCondition) {
+    throw new UnsupportedOperationException("getRows() not supported for this index collection.");
+  }
+
+  @Override
+  public boolean supportsRowCountStats() {
+    return false;
+  }
+
+  @Override
+  public boolean supportsFullTextSearch() {
+    return false;
+  }
+
+  @Override
+  public boolean isColumnIndexed(SchemaPath path) {
+    for (IndexDescriptor index : indexes) {
+      if (index.getIndexColumnOrdinal(path) >= 0) {
+        return true;
+      }
+    }
+    return false;
+  }
+
+  @Override
+  public Iterator<IndexDescriptor> iterator() {
+    return indexes.iterator();
+  }
+
+}
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/index/AbstractIndexDescriptor.java b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/index/AbstractIndexDescriptor.java
new file mode 100644
index 0000000..f908ead
--- /dev/null
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/index/AbstractIndexDescriptor.java
@@ -0,0 +1,74 @@
+/*
+ * 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 org.apache.calcite.plan.RelOptCost;
+import org.apache.calcite.plan.RelOptPlanner;
+import org.apache.calcite.rel.RelFieldCollation.NullDirection;
+import org.apache.calcite.rel.RelNode;
+import org.apache.calcite.rex.RexNode;
+import org.apache.drill.common.expression.LogicalExpression;
+import org.apache.drill.exec.physical.base.GroupScan;
+import org.apache.drill.exec.physical.base.IndexGroupScan;
+
+/**
+ * Abstract base class for an Index descriptor
+ *
+ */
+public abstract class AbstractIndexDescriptor extends DrillIndexDefinition implements IndexDescriptor {
+  static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(AbstractIndexDescriptor .class);
+
+  public AbstractIndexDescriptor(List<LogicalExpression> indexCols,
+                                 CollationContext indexCollationContext,
+                                 List<LogicalExpression> nonIndexCols,
+                                 List<LogicalExpression> rowKeyColumns,
+                                 String indexName,
+                                 String tableName,
+                                 IndexType type,
+                                 NullDirection nullsDirection) {
+    super(indexCols, indexCollationContext, nonIndexCols, rowKeyColumns, indexName, tableName, type, nullsDirection);
+  }
+
+  @Override
+  public double getRows(RelNode scan, RexNode indexCondition) {
+    throw new UnsupportedOperationException("getRows() not supported for this index.");
+  }
+
+  @Override
+  public boolean supportsRowCountStats() {
+    return false;
+  }
+
+  @Override
+  public IndexGroupScan getIndexGroupScan() {
+    throw new UnsupportedOperationException("Group scan not supported for this index.");
+  }
+
+  @Override
+  public boolean supportsFullTextSearch() {
+    return false;
+  }
+
+  @Override
+  public RelOptCost getCost(IndexProperties indexProps, RelOptPlanner planner,
+      int numProjectedFields, GroupScan primaryGroupScan) {
+    throw new UnsupportedOperationException("getCost() not supported for this index.");
+  }
+}
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/index/AbstractIndexStatistics.java b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/index/AbstractIndexStatistics.java
new file mode 100644
index 0000000..dfc0897
--- /dev/null
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/index/AbstractIndexStatistics.java
@@ -0,0 +1,51 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.drill.exec.planner.index;
+
+import org.apache.calcite.rel.RelCollation;
+import org.apache.calcite.rel.RelDistribution;
+import org.apache.calcite.rel.RelNode;
+import org.apache.calcite.rex.RexNode;
+import org.apache.drill.exec.planner.logical.DrillTable;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.util.List;
+
+public abstract class AbstractIndexStatistics implements IndexStatistics {
+
+    protected static final Logger logger = LoggerFactory.getLogger(AbstractIndexStatistics.class);
+    protected final RelNode input;
+    protected final RexNode condition;
+    protected final DrillTable table;
+
+    public AbstractIndexStatistics(RelNode input, RexNode condition, DrillTable table) {
+            this.input = input;
+            this.condition = condition;
+            this.table = table;
+    }
+    public abstract double getRowCount();
+
+    public List<RelCollation> getCollations() {
+        throw new UnsupportedOperationException();
+    }
+
+    public RelDistribution getDistribution() {
+        throw new UnsupportedOperationException();
+    }
+}
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/index/CollationContext.java b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/index/CollationContext.java
new file mode 100644
index 0000000..8260bee
--- /dev/null
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/index/CollationContext.java
@@ -0,0 +1,37 @@
+/*
+ * 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.calcite.rel.RelFieldCollation;
+import org.apache.drill.common.expression.LogicalExpression;
+
+import java.util.List;
+import java.util.Map;
+
+public class CollationContext {
+
+  public final Map<LogicalExpression, RelFieldCollation> collationMap;
+  public final List<RelFieldCollation> relFieldCollations;
+
+  public CollationContext(Map<LogicalExpression, RelFieldCollation> collationMap,
+      List<RelFieldCollation> relFieldCollations) {
+    this.collationMap = collationMap;
+    this.relFieldCollations = relFieldCollations;
+  }
+
+}
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/index/DrillIndexCollection.java b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/index/DrillIndexCollection.java
new file mode 100644
index 0000000..0ea3d83
--- /dev/null
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/index/DrillIndexCollection.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.calcite.rex.RexNode;
+import org.apache.drill.exec.physical.base.IndexGroupScan;
+import org.apache.calcite.rel.RelNode;
+
+import java.util.Set;
+
+public class DrillIndexCollection extends AbstractIndexCollection {
+  private final RelNode scan;  // physical scan rel corresponding to the primary table
+
+  public DrillIndexCollection(RelNode scanRel,
+                               Set<DrillIndexDescriptor> indexes) {
+    this.scan = scanRel;
+    for (IndexDescriptor index : indexes) {
+      super.addIndex(index);
+    }
+  }
+
+  private IndexDescriptor getIndexDescriptor() {
+
+    //XXX need a policy to pick the indexDesc to use instead of picking the first one.
+    return this.indexes.iterator().next();
+  }
+
+  @Override
+  public boolean supportsIndexSelection() {
+    return true;
+  }
+
+  @Override
+  public boolean supportsRowCountStats() {
+    return true;
+  }
+
+  @Override
+  public boolean supportsFullTextSearch() {
+    return true;
+  }
+
+  @Override
+  public double getRows(RexNode indexCondition) {
+
+    return getIndexDescriptor().getRows(scan, indexCondition);
+  }
+
+  @Override
+  public IndexGroupScan getGroupScan() {
+    return getIndexDescriptor().getIndexGroupScan();
+  }
+
+  @Override
+  public IndexCollectionType getIndexCollectionType() {
+    return IndexCollection.IndexCollectionType.EXTERNAL_SECONDARY_INDEX_COLLECTION;
+  }
+
+}
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/index/DrillIndexDefinition.java b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/index/DrillIndexDefinition.java
new file mode 100644
index 0000000..03c2a44
--- /dev/null
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/index/DrillIndexDefinition.java
@@ -0,0 +1,278 @@
+/*
+ * 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 com.fasterxml.jackson.annotation.JsonIgnore;
+import com.fasterxml.jackson.annotation.JsonProperty;
+import org.apache.drill.shaded.guava.com.google.common.collect.Sets;
+
+import org.apache.calcite.rel.RelCollation;
+import org.apache.calcite.rel.RelCollations;
+import org.apache.calcite.rel.RelFieldCollation;
+import org.apache.calcite.rel.RelFieldCollation.NullDirection;
+import org.apache.drill.common.expression.CastExpression;
+import org.apache.drill.common.expression.LogicalExpression;
+import org.apache.drill.common.expression.SchemaPath;
+
+import java.util.Collection;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+
+public class DrillIndexDefinition implements IndexDefinition {
+  /**
+   * The indexColumns is the list of column(s) on which this index is created. If there is more than 1 column,
+   * the order of the columns is important: index on {a, b} is not the same as index on {b, a}
+   * NOTE: the indexed column could be of type columnfamily.column
+   */
+  @JsonProperty
+  protected final List<LogicalExpression> indexColumns;
+
+  /**
+   * nonIndexColumns: the list of columns that are included in the index as 'covering'
+   * columns but are not themselves indexed.  These are useful for covering indexes where the
+   * query request can be satisfied directly by the index and avoid accessing the table altogether.
+   */
+  @JsonProperty
+  protected final List<LogicalExpression> nonIndexColumns;
+
+  @JsonIgnore
+  protected final Set<LogicalExpression> allIndexColumns;
+
+  @JsonProperty
+  protected final List<LogicalExpression> rowKeyColumns;
+
+  @JsonProperty
+  protected final CollationContext indexCollationContext;
+
+  /**
+   * indexName: name of the index that should be unique within the scope of a table
+   */
+  @JsonProperty
+  protected final String indexName;
+
+  protected final String tableName;
+
+  @JsonProperty
+  protected final IndexDescriptor.IndexType indexType;
+
+  @JsonProperty
+  protected final NullDirection nullsDirection;
+
+  public DrillIndexDefinition(List<LogicalExpression> indexCols,
+                              CollationContext indexCollationContext,
+                              List<LogicalExpression> nonIndexCols,
+                              List<LogicalExpression> rowKeyColumns,
+                              String indexName,
+                              String tableName,
+                              IndexType type,
+                              NullDirection nullsDirection) {
+    this.indexColumns = indexCols;
+    this.nonIndexColumns = nonIndexCols;
+    this.rowKeyColumns = rowKeyColumns;
+    this.indexName = indexName;
+    this.tableName = tableName;
+    this.indexType = type;
+    this.allIndexColumns = Sets.newHashSet(indexColumns);
+    this.allIndexColumns.addAll(nonIndexColumns);
+    this.indexCollationContext = indexCollationContext;
+    this.nullsDirection = nullsDirection;
+
+  }
+
+  @Override
+  public int getIndexColumnOrdinal(LogicalExpression path) {
+    int id = indexColumns.indexOf(path);
+    return id;
+  }
+
+  @Override
+  public boolean isCoveringIndex(List<LogicalExpression> columns) {
+    return allIndexColumns.containsAll(columns);
+  }
+
+  @Override
+  public boolean allColumnsIndexed(Collection<LogicalExpression> columns) {
+    return columnsInIndexFields(columns, indexColumns);
+  }
+
+  @Override
+  public boolean someColumnsIndexed(Collection<LogicalExpression> columns) {
+    return someColumnsInIndexFields(columns, indexColumns);
+  }
+
+  public boolean pathExactIn(SchemaPath path, Collection<LogicalExpression> exprs) {
+    for (LogicalExpression expr : exprs) {
+      if (expr instanceof SchemaPath) {
+        if (((SchemaPath) expr).toExpr().equals(path.toExpr())) {
+          return true;
+        }
+      }
+    }
+
+    return false;
+  }
+
+  boolean castIsCompatible(CastExpression castExpr, Collection<LogicalExpression> indexFields) {
+    for(LogicalExpression indexExpr : indexFields) {
+      if(indexExpr.getClass() != castExpr.getClass()) {
+        continue;
+      }
+      CastExpression indexCastExpr = (CastExpression)indexExpr;
+      //we compare input using equals because we know we are comparing SchemaPath,
+      //if we extend to support other expression, make sure the equals of that expression
+      //is implemented properly, otherwise it will fall to identity comparison
+      if ( !castExpr.getInput().equals(indexCastExpr.getInput()) ) {
+          continue;
+      }
+
+      if( castExpr.getMajorType().getMinorType() != indexCastExpr.getMajorType().getMinorType()) {
+        continue;
+      }
+      return true;
+    }
+    return false;
+  }
+
+  protected boolean columnsInIndexFields(Collection<LogicalExpression> columns, Collection<LogicalExpression> indexFields) {
+    //we need to do extra check, so we could allow the case when query condition expression is not identical with indexed fields
+    //and they still could use the index either by implicit cast or the difference is allowed, e.g. width of varchar
+    for (LogicalExpression col : columns) {
+      if (col instanceof CastExpression) {
+        if (!castIsCompatible((CastExpression) col, indexFields)) {
+          return false;
+        }
+      }
+      else {
+        if (!pathExactIn((SchemaPath)col, indexFields)) {
+          return false;
+        }
+      }
+    }
+    return true;//indexFields.containsAll(columns);
+  }
+
+  protected boolean someColumnsInIndexFields(Collection<LogicalExpression> columns,
+      Collection<LogicalExpression> indexFields) {
+
+    //we need to do extra check, so we could allow the case when query condition expression is not identical with indexed fields
+    //and they still could use the index either by implicit cast or the difference is allowed, e.g. width of varchar
+    for (LogicalExpression col : columns) {
+      if (col instanceof CastExpression) {
+        if (castIsCompatible((CastExpression) col, indexFields)) {
+          return true;
+        }
+      }
+      else {
+        if (pathExactIn((SchemaPath)col, indexFields)) {
+          return true;
+        }
+      }
+    }
+    return false;
+  }
+
+  @Override
+  public String toString() {
+    String columnsDesc = " Index columns: " + indexColumns.toString() + " Non-Index columns: " + nonIndexColumns.toString();
+    String desc = "Table: " + tableName + " Index: " + indexName + columnsDesc;
+    return desc;
+  }
+
+  @Override
+  public boolean equals(Object o) {
+    if (this == o) {
+      return true;
+    }
+    if (o == null) {
+      return false;
+    }
+    DrillIndexDefinition index1 = (DrillIndexDefinition) o;
+    return tableName.equals(index1.tableName)
+        && indexName.equals(index1.indexName)
+        && indexType.equals(index1.indexType)
+        && indexColumns.equals(index1.indexColumns);
+  }
+
+  @Override
+  public int hashCode() {
+    final int prime = 31;
+    final String fullName = tableName + indexName;
+    int result = 1;
+    result = prime * result + fullName.hashCode();
+    result = prime * result + indexType.hashCode();
+
+    return result;
+  }
+
+  @Override
+  @JsonProperty
+  public String getIndexName() {
+    return indexName;
+  }
+
+  @Override
+  public String getTableName() {
+    return tableName;
+  }
+
+  @Override
+  @JsonProperty
+  public IndexDescriptor.IndexType getIndexType() {
+    return indexType;
+  }
+
+  @Override
+  @JsonProperty
+  public List<LogicalExpression> getRowKeyColumns() {
+    return this.rowKeyColumns;
+  }
+
+  @Override
+  @JsonProperty
+  public List<LogicalExpression> getIndexColumns() {
+    return this.indexColumns;
+  }
+
+  @Override
+  @JsonProperty
+  public List<LogicalExpression> getNonIndexColumns() {
+    return this.nonIndexColumns;
+  }
+
+  @Override
+  @JsonIgnore
+  public RelCollation getCollation() {
+    if (indexCollationContext != null) {
+      return RelCollations.of(indexCollationContext.relFieldCollations);
+    }
+    return null;
+  }
+
+  @Override
+  @JsonIgnore
+  public Map<LogicalExpression, RelFieldCollation> getCollationMap() {
+    return indexCollationContext.collationMap;
+  }
+
+  @Override
+  @JsonIgnore
+  public NullDirection getNullsOrderingDirection() {
+    return nullsDirection;
+  }
+}
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/index/DrillIndexDescriptor.java b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/index/DrillIndexDescriptor.java
new file mode 100644
index 0000000..4da62c2
--- /dev/null
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/index/DrillIndexDescriptor.java
@@ -0,0 +1,110 @@
+/*
+ * 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.calcite.rel.RelFieldCollation.NullDirection;
+import org.apache.calcite.rel.RelNode;
+import org.apache.calcite.rex.RexNode;
+import org.apache.drill.common.expression.LogicalExpression;
+import org.apache.drill.exec.physical.base.GroupScan;
+import org.apache.drill.exec.physical.base.IndexGroupScan;
+import org.apache.drill.exec.planner.cost.PluginCost;
+import org.apache.drill.exec.planner.logical.DrillTable;
+
+import java.io.IOException;
+import java.util.List;
+
+public class DrillIndexDescriptor extends AbstractIndexDescriptor {
+
+  /**
+   * The name of Drill's Storage Plugin on which the Index was stored
+   */
+  private String storage;
+
+  private DrillTable table;
+
+  public DrillIndexDescriptor(List<LogicalExpression> indexCols,
+                              CollationContext indexCollationContext,
+                              List<LogicalExpression> nonIndexCols,
+                              List<LogicalExpression> rowKeyColumns,
+                              String indexName,
+                              String tableName,
+                              IndexType type,
+                              NullDirection nullsDirection) {
+    super(indexCols, indexCollationContext, nonIndexCols, rowKeyColumns, indexName, tableName, type, nullsDirection);
+  }
+
+  public DrillIndexDescriptor(DrillIndexDefinition def) {
+    this(def.indexColumns, def.indexCollationContext, def.nonIndexColumns, def.rowKeyColumns, def.indexName,
+        def.getTableName(), def.getIndexType(), def.nullsDirection);
+  }
+
+  @Override
+  public double getRows(RelNode scan, RexNode indexCondition) {
+    //TODO: real implementation is to use Drill's stats implementation. for now return fake value 1.0
+    return 1.0;
+  }
+
+  @Override
+  public IndexGroupScan getIndexGroupScan() {
+    try {
+      final DrillTable idxTable = getDrillTable();
+      GroupScan scan = idxTable.getGroupScan();
+
+      if (!(scan instanceof IndexGroupScan)){
+        logger.error("The Groupscan from table {} is not an IndexGroupScan", idxTable.toString());
+        return null;
+      }
+      return (IndexGroupScan)scan;
+    }
+    catch(IOException e) {
+      logger.error("Error in getIndexGroupScan ", e);
+    }
+    return null;
+  }
+
+  public void attach(String storageName, DrillTable inTable) {
+    storage = storageName;
+    setDrillTable(inTable);
+  }
+
+  public void setStorageName(String storageName) {
+    storage = storageName;
+  }
+
+  public String getStorageName() {
+    return storage;
+  }
+
+  public void setDrillTable(DrillTable table) {
+    this.table = table;
+  }
+
+  public DrillTable getDrillTable() {
+    return this.table;
+  }
+
+  public FunctionalIndexInfo getFunctionalInfo() {
+    return null;
+  }
+
+  @Override
+  public PluginCost getPluginCostModel() {
+    return null;
+  }
+}
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/index/FunctionalIndexInfo.java b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/index/FunctionalIndexInfo.java
new file mode 100644
index 0000000..a12dcc6
--- /dev/null
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/index/FunctionalIndexInfo.java
@@ -0,0 +1,85 @@
+/*
+ * 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.expression.LogicalExpression;
+import org.apache.drill.common.expression.SchemaPath;
+
+import java.util.Map;
+import java.util.Set;
+
+/**
+ * FunctionalIndexInfo is to collect Functional fields in IndexDescriptor, derive information needed for index plan,
+ * e.g. convert and rewrite filter, columns, and rowtype on index scan that involve functional index.
+ * In case different store might have different way to rename expression in index table, we allow storage plugin
+ */
+public interface FunctionalIndexInfo {
+
+  /**
+   * @return if this index has functional indexed field, return true
+   */
+  boolean hasFunctional();
+
+  /**
+   * @return the IndexDescriptor this IndexInfo built from
+   */
+  IndexDescriptor getIndexDesc();
+
+  /**
+   * getNewPath: for an original path, return new rename '$N' path, notice there could be multiple renamed paths
+   * if the there are multiple functional indexes refer original path.
+   * @param path
+   * @return
+   */
+  SchemaPath getNewPath(SchemaPath path);
+
+  /**
+   * return a plain field path if the incoming index expression 'expr' is replaced to be a plain field
+   * @param expr suppose to be an indexed expression
+   * @return the renamed schemapath in index table for the indexed expression
+   */
+  SchemaPath getNewPathFromExpr(LogicalExpression expr);
+
+  /**
+   * @return the map of indexed expression --> the involved schema paths in a indexed expression
+   */
+  Map<LogicalExpression, Set<SchemaPath>> getPathsInFunctionExpr();
+
+  /**
+   * @return the map between indexed expression and to-be-converted target expression for scan in index
+   * e.g. cast(a.b as int) -> '$0'
+   */
+  Map<LogicalExpression, LogicalExpression> getExprMap();
+
+  /**
+   * @return the set of all new field names for indexed functions in index
+   */
+  Set<SchemaPath> allNewSchemaPaths();
+
+  /**
+   * @return the set of all schemaPath exist in functional index fields
+   */
+  Set<SchemaPath> allPathsInFunction();
+
+  /**
+   * Whether this implementation( may be different per storage) support rewrite rewriting varchar equality expression,
+   * e.g. cast(a.b as varchar(2)) = 'ca'  to LIKE expression: cast(a.b as varchar(2) LIKE 'ca%'
+   */
+  boolean supportEqualCharConvertToLike();
+
+}
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/index/IndexCallContext.java b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/index/IndexCallContext.java
new file mode 100644
index 0000000..65788cb
--- /dev/null
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/index/IndexCallContext.java
@@ -0,0 +1,76 @@
+/*
+ * 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.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.DbGroupScan;
+import org.apache.drill.exec.planner.physical.DrillDistributionTrait.DistributionField;
+import org.apache.drill.exec.planner.common.DrillScanRelBase;
+import org.apache.drill.exec.planner.common.DrillProjectRelBase;
+import java.util.List;
+import java.util.Set;
+
+public interface IndexCallContext {
+  DrillScanRelBase getScan();
+
+  DbGroupScan getGroupScan();
+
+  List<RelCollation> getCollationList();
+
+  RelCollation getCollation();
+
+  boolean hasLowerProject();
+
+  boolean hasUpperProject();
+
+  RelOptRuleCall getCall();
+
+  Set<LogicalExpression> getLeftOutPathsInFunctions();
+
+  RelNode getFilter();
+
+  IndexableExprMarker getOrigMarker();
+
+  List<LogicalExpression> getSortExprs();
+
+  DrillProjectRelBase getLowerProject();
+
+  DrillProjectRelBase getUpperProject();
+
+  void setLeftOutPathsInFunctions(Set<LogicalExpression> exprs);
+
+  List<SchemaPath> getScanColumns();
+
+  RexNode getFilterCondition();
+
+  RexNode getOrigCondition();
+
+  Sort getSort();
+
+  void createSortExprs();
+
+  RelNode getExchange();
+
+  List<DistributionField> getDistributionFields();
+}
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/index/IndexCollection.java b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/index/IndexCollection.java
new file mode 100644
index 0000000..9b4d170
--- /dev/null
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/index/IndexCollection.java
@@ -0,0 +1,99 @@
+/*
+ * 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.calcite.rex.RexNode;
+import org.apache.drill.common.expression.SchemaPath;
+import org.apache.drill.exec.physical.base.IndexGroupScan;
+
+// Interface used to describe an index collection
+public interface IndexCollection extends Iterable<IndexDescriptor> {
+  /**
+   * Types of an index collections: NATIVE_SECONDARY_INDEX_COLLECTION, EXTERNAL_SECONDARY_INDEX_COLLECTION
+   */
+  public static enum IndexCollectionType {
+    NATIVE_SECONDARY_INDEX_COLLECTION,
+    EXTERNAL_SECONDARY_INDEX_COLLECTION
+  };
+
+  /**
+   * Add a new index to the collection. Return True if index was successfully added; False otherwise
+   */
+  public boolean addIndex(IndexDescriptor index);
+
+  /**
+   * Remove an index (identified by table name and index name) from the collection.
+   * Return True if index was successfully removed; False otherwise
+   */
+  public boolean removeIndex(IndexDescriptor index);
+
+  /**
+   * Clears all entries from this index collection
+   */
+  public void clearAll();
+
+  /**
+   * Get the type of this index based on {@link IndexCollectionType}
+   * @return one of the values in {@link IndexCollectionType}
+   */
+  public IndexCollectionType getIndexCollectionType();
+
+  /**
+   * Whether or not this index collection supports index selection (selecting an
+   * appropriate index out of multiple candidates). Typically, external index collections
+   * such as Elasticsearch already have this capability while native secondary index collection
+   * may not have - in such cases, Drill needs to do the index selection.
+   */
+  public boolean supportsIndexSelection();
+
+  /**
+   * Get the estimated row count for a single index condition
+   * @param indexCondition The index condition (e.g index_col1 < 10 AND index_col2 = 'abc')
+   * @return The estimated row count
+   */
+  public double getRows(RexNode indexCondition);
+
+  /**
+   * Whether or not the index supports getting row count statistics
+   * @return True if index supports getting row count, False otherwise
+   */
+  public boolean supportsRowCountStats();
+
+  /**
+   * Whether or not the index supports full-text search (to allow pushing down such filters)
+   * @return True if index supports full-text search, False otherwise
+   */
+  public boolean supportsFullTextSearch();
+
+  /**
+   * If this IndexCollection exposes a single GroupScan, return the GroupScan instance. For external indexes
+   * such as Elasticsearch, we may have a single GroupScan representing all the indexes contained
+   * within that collection.  On the other hand, for native indexes, each separate index would
+   * have its own GroupScan.
+   * @return GroupScan for this IndexCollection if available, otherwise null
+   */
+  public IndexGroupScan getGroupScan();
+
+  /**
+   * Check if the field name is the leading key of any of the indexes in this collection
+   * @param path
+   * @return True if an appropriate index is found, False otherwise
+   */
+  public boolean isColumnIndexed(SchemaPath path);
+
+}
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/index/IndexDefinition.java b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/index/IndexDefinition.java
new file mode 100644
index 0000000..995d23c
--- /dev/null
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/index/IndexDefinition.java
@@ -0,0 +1,105 @@
+/*
+ * 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.calcite.rel.RelCollation;
+import org.apache.calcite.rel.RelFieldCollation;
+import org.apache.calcite.rel.RelFieldCollation.NullDirection;
+import org.apache.drill.common.expression.LogicalExpression;
+
+import java.util.Collection;
+import java.util.List;
+import java.util.Map;
+
+// Interface used to define an index,
+public interface IndexDefinition {
+  /**
+   * Types of an index: PRIMARY_KEY_INDEX, NATIVE_SECONDARY_INDEX, EXTERNAL_SECONDARY_INDEX
+   */
+  static enum IndexType {
+    PRIMARY_KEY_INDEX,
+    NATIVE_SECONDARY_INDEX,
+    EXTERNAL_SECONDARY_INDEX
+  };
+
+  /**
+   * Check to see if the field name is an index column and if so return the ordinal position in the index
+   * @param path The field path you want to compare to index column names.
+   * @return Return ordinal of the indexed column if valid, otherwise return -1
+   */
+  int getIndexColumnOrdinal(LogicalExpression path);
+
+  /**
+   * Get the name of the index
+   */
+  String getIndexName();
+
+  /**
+   * Check if this index 'covers' all the columns specified in the supplied list of columns
+   * @param columns
+   * @return True for covering index, False for non-covering
+   */
+  boolean isCoveringIndex(List<LogicalExpression> columns);
+
+  /**
+   * Check if this index have all the columns specified in the supplied list of columns indexed
+   * @param columns
+   * @return True if all fields are indexed, False for some or all fields is not indexed
+   */
+  boolean allColumnsIndexed(Collection<LogicalExpression> columns);
+
+  /**
+   * Check if this index has some columns specified in the supplied list of columns indexed
+   * @param columns
+   * @return True if some fields are indexed, False if none of the fields are indexed
+   */
+  boolean someColumnsIndexed(Collection<LogicalExpression> columns);
+
+  /**
+   * Get the list of columns (typically 1 column) that constitute the row key (primary key)
+   * @return
+   */
+  List<LogicalExpression> getRowKeyColumns();
+
+  /**
+   * Get the name of the table this index is associated with
+   */
+  String getTableName();
+
+  /**
+   * Get the type of this index based on {@link IndexType}
+   * @return one of the values in {@link IndexType}
+   */
+  IndexType getIndexType();
+
+
+  List<LogicalExpression> getIndexColumns();
+
+  List<LogicalExpression> getNonIndexColumns();
+
+  RelCollation getCollation();
+
+  Map<LogicalExpression, RelFieldCollation> getCollationMap();
+
+  /**
+   * Get the nulls ordering of this index
+   * @return True, if nulls first. False otherwise
+   */
+  NullDirection getNullsOrderingDirection();
+
+}
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/index/IndexDescriptor.java b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/index/IndexDescriptor.java
new file mode 100644
index 0000000..f355285
--- /dev/null
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/index/IndexDescriptor.java
@@ -0,0 +1,68 @@
+/*
+ * 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.calcite.plan.RelOptCost;
+import org.apache.calcite.plan.RelOptPlanner;
+import org.apache.calcite.rel.RelNode;
+import org.apache.calcite.rex.RexNode;
+import org.apache.drill.exec.physical.base.GroupScan;
+import org.apache.drill.exec.physical.base.IndexGroupScan;
+import org.apache.drill.exec.planner.cost.PluginCost;
+
+
+/**
+ * IndexDefinition + functions to access materialized index(index table/scan, etc)
+ */
+
+public interface IndexDescriptor extends IndexDefinition {
+
+  /**
+   * Get the estimated row count for a single index condition
+   * @param input The rel node corresponding to the primary table
+   * @param indexCondition The index condition (e.g index_col1 < 10 AND index_col2 = 'abc')
+   * @return The estimated row count
+   */
+  double getRows(RelNode input, RexNode indexCondition);
+
+  /**
+   * Whether or not the index supports getting row count statistics
+   * @return True if index supports getting row count, False otherwise
+   */
+  boolean supportsRowCountStats();
+
+  /**
+   * Get an instance of the group scan associated with this index descriptor
+   * @return An instance of group scan for this index
+   */
+  IndexGroupScan getIndexGroupScan();
+
+  /**
+   * Whether or not the index supports full-text search (to allow pushing down such filters)
+   * @return True if index supports full-text search, False otherwise
+   */
+  boolean supportsFullTextSearch();
+
+  FunctionalIndexInfo getFunctionalInfo();
+
+  public RelOptCost getCost(IndexProperties indexProps, RelOptPlanner planner,
+      int numProjectedFields, GroupScan primaryGroupScan);
+
+  public PluginCost getPluginCostModel();
+
+}
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/index/IndexDiscover.java b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/index/IndexDiscover.java
new file mode 100644
index 0000000..309083b
--- /dev/null
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/index/IndexDiscover.java
@@ -0,0 +1,23 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.drill.exec.planner.index;
+
+
+public interface IndexDiscover {
+    IndexCollection getTableIndex(String tableName);
+}
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/index/IndexDiscoverBase.java b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/index/IndexDiscoverBase.java
new file mode 100644
index 0000000..fde2a32
--- /dev/null
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/index/IndexDiscoverBase.java
@@ -0,0 +1,110 @@
+/*
+ * 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.exec.physical.base.AbstractDbGroupScan;
+import org.apache.drill.exec.planner.common.DrillScanRelBase;
+import org.apache.drill.exec.planner.logical.DrillTable;
+import org.apache.drill.exec.planner.physical.ScanPrel;
+import org.apache.calcite.rel.RelNode;
+import java.util.Collection;
+import java.util.HashSet;
+import java.util.Set;
+
+/**
+ * IndexDiscoverBase is the layer to read index configurations of tables on storage plugins,
+ * then based on the properties it collected, get the StoragePlugin from StoragePluginRegistry,
+ * together with indexes information, build an IndexCollection
+ */
+public abstract class IndexDiscoverBase implements IndexDiscover {
+  static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(IndexDiscoverBase.class);
+
+  private AbstractDbGroupScan scan; // group scan corresponding to the primary table
+  private RelNode scanRel;   // physical scan rel corresponding to the primary table
+
+  public IndexDiscoverBase(AbstractDbGroupScan inScan, DrillScanRelBase inScanPrel) {
+    scan = inScan;
+    scanRel = inScanPrel;
+  }
+
+  public IndexDiscoverBase(AbstractDbGroupScan inScan, ScanPrel inScanPrel) {
+    scan = inScan;
+    scanRel = inScanPrel;
+  }
+
+  public AbstractDbGroupScan getOriginalScan() {
+    return scan;
+  }
+
+  public RelNode getOriginalScanRel() {
+    return scanRel;
+  }
+
+  public IndexCollection getTableIndex(String tableName, String storageName, Collection<DrillIndexDefinition>  indexDefs ) {
+    Set<DrillIndexDescriptor> idxSet = new HashSet<>();
+    for (DrillIndexDefinition def : indexDefs) {
+      DrillIndexDescriptor indexDescriptor = new DrillIndexDescriptor(def);
+      materializeIndex(storageName, indexDescriptor);
+    }
+    return new DrillIndexCollection(getOriginalScanRel(), idxSet);
+  }
+
+  public void materializeIndex(String storageName, DrillIndexDescriptor index) {
+    index.setStorageName(storageName);
+    index.setDrillTable(buildDrillTable(index));
+  }
+
+  /**
+   * When there is storageName in IndexDescriptor, get a DrillTable instance based on the
+   * StorageName and other informaiton in idxDesc that helps identifying the table.
+   * @param idxDesc
+   * @return
+   */
+  public DrillTable getExternalDrillTable(IndexDescriptor idxDesc) {
+    //XX: get table object for this index, index storage plugin should provide interface to get the DrillTable object
+    return null;
+  }
+
+  /**
+   * Abstract function getDrillTable will be implemented the IndexDiscover within storage plugin(e.g. HBase, MaprDB)
+   * since the implementations of AbstractStoragePlugin, IndexDescriptor and DrillTable in that storage plugin may have
+   * the implement details.
+   * @param idxDesc
+
+   * @return
+   */
+  public DrillTable buildDrillTable(IndexDescriptor idxDesc) {
+    if(idxDesc.getIndexType() == IndexDescriptor.IndexType.EXTERNAL_SECONDARY_INDEX) {
+      return getExternalDrillTable(idxDesc);
+    }
+    else {
+      return getNativeDrillTable(idxDesc);
+    }
+  }
+
+  /**
+   * When it is native index(index provided by native storage plugin),
+   * the actual IndexDiscover should provide the implementation to get the DrillTable object of index,
+   * Otherwise, we call IndexDiscoverable interface exposed from external storage plugin's SchemaFactory
+   * to get the desired DrillTable.
+   * @param idxDesc
+   * @return
+   */
+  public abstract DrillTable getNativeDrillTable(IndexDescriptor idxDesc);
+
+}
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/index/IndexDiscoverable.java b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/index/IndexDiscoverable.java
new file mode 100644
index 0000000..dbf5edc
--- /dev/null
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/index/IndexDiscoverable.java
@@ -0,0 +1,37 @@
+/*
+ * 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.exec.planner.logical.DrillTable;
+
+
+/**
+ * SchemaFactory of a storage plugin that can used to store index tables should expose this interface to allow
+ * IndexDiscovers discovering the index table without adding dependency to the storage plugin.
+ */
+public interface IndexDiscoverable {
+
+  /**
+   * return the found DrillTable with path (e.g. names={"elasticsearch", "staffidx", "stjson"})
+   * @param discover
+   * @param desc
+   * @return
+   */
+    DrillTable findTable(IndexDiscover discover, DrillIndexDescriptor desc);
+
+}
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/index/IndexGroup.java b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/index/IndexGroup.java
new file mode 100644
index 0000000..ea34ea5
--- /dev/null
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/index/IndexGroup.java
@@ -0,0 +1,63 @@
+/*
+ * 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 java.util.List;
+
+/**
+ * Encapsulates one or more IndexProperties representing (non)covering or intersecting indexes. The encapsulated
+ * IndexProperties are used to rank the index in comparison with other IndexGroups.
+ */
+public class IndexGroup {
+  private List<IndexProperties> indexProps;
+
+  public IndexGroup() {
+    indexProps = Lists.newArrayList();
+  }
+
+  public boolean isIntersectIndex() {
+    if (indexProps.size() > 1) {
+      return true;
+    } else {
+      return false;
+    }
+  }
+
+  public int numIndexes() {
+    return indexProps.size();
+  }
+
+  public void addIndexProp(IndexProperties prop) {
+    indexProps.add(prop);
+  }
+
+  public void addIndexProp(List<IndexProperties> prop) {
+    indexProps.addAll(prop);
+  }
+
+  public boolean removeIndexProp(IndexProperties prop) {
+    return indexProps.remove(prop);
+  }
+
+  public List<IndexProperties> getIndexProps() {
+    return indexProps;
+  }
+}
+
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
new file mode 100644
index 0000000..cfdd6d0
--- /dev/null
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/index/IndexProperties.java
@@ -0,0 +1,64 @@
+/*
+ * 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.calcite.plan.RelOptCost;
+import org.apache.calcite.plan.RelOptPlanner;
+import org.apache.calcite.rex.RexNode;
+import org.apache.drill.common.expression.LogicalExpression;
+import org.apache.drill.exec.planner.common.DrillScanRelBase;
+
+import java.util.Map;
+
+/**
+ * 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 interface IndexProperties  {
+
+  void setProperties(Map<LogicalExpression, RexNode> prefixMap,
+                            boolean satisfiesCollation,
+                            RexNode indexColumnsRemainderFilter,
+                            Statistics stats);
+
+  double getLeadingSelectivity();
+
+  double getRemainderSelectivity();
+
+  boolean isCovering();
+
+  double getTotalRows();
+
+  IndexDescriptor getIndexDesc();
+
+  DrillScanRelBase getPrimaryTableScan();
+
+  RexNode getTotalRemainderFilter();
+
+  boolean satisfiesCollation();
+
+  void setSatisfiesCollation(boolean satisfiesCollation);
+
+  RelOptCost getSelfCost(RelOptPlanner planner);
+
+  int numLeadingFilters();
+
+  double getAvgRowSize();
+}
+
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/index/IndexStatistics.java b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/index/IndexStatistics.java
new file mode 100644
index 0000000..e716369
--- /dev/null
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/index/IndexStatistics.java
@@ -0,0 +1,36 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.drill.exec.planner.index;
+
+import org.apache.calcite.rel.RelCollation;
+import org.apache.calcite.rel.RelDistribution;
+
+
+import java.util.List;
+
+public interface IndexStatistics {
+    /** Returns the approximate number of rows in the table. */
+    double getRowCount();
+
+    /** Returns the collections of columns on which this table is sorted. */
+    List<RelCollation> getCollations();
+
+    /** Returns the distribution of the data in query result table. */
+    RelDistribution getDistribution();
+
+}
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
new file mode 100644
index 0000000..a1a6fc8
--- /dev/null
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/index/IndexableExprMarker.java
@@ -0,0 +1,262 @@
+/*
+ * 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.drill.shaded.guava.com.google.common.collect.Sets;
+import org.apache.calcite.rel.RelNode;
+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.type.SqlTypeName;
+import org.apache.drill.common.expression.LogicalExpression;
+import org.apache.drill.exec.planner.logical.DrillOptiq;
+import org.apache.drill.exec.planner.logical.DrillParseContext;
+import org.apache.drill.exec.planner.physical.PrelUtil;
+
+import java.util.Map;
+import java.util.Set;
+
+/**
+ * The filter expressions that could be indexed
+ * Other than SchemaPaths, which represent columns of a table and could be indexed,
+ * we consider only function expressions, and specifically, CAST function.
+ * To judge if an expression is indexable, we check these:
+ * 1, this expression should be one operand of a comparison operator, one of SqlKind.COMPARISON:
+ *      IN, EQUALS, NOT_EQUALS, LESS_THAN, GREATER_THAN, GREATER_THAN_OR_EQUAL, LESS_THAN_OR_EQUAL
+ * 2, the expression tree should contain at least one inputRef (which means this expression is a
+ *     computation on top of at least one column), and if we have more than one indexable expressions
+ *     are found from operands of comparison operator, we should not take any expression as indexable.
+ *
+ * 3, (LIMIT to one level function) the expression is a function call, and no nested function call underneath, except ITEM
+ * 4, (LIMIT to CAST), the function call is a CAST
+ */
+public class IndexableExprMarker extends RexVisitorImpl<Boolean> {
+
+  //map of rexNode->converted LogicalExpression
+  final Map<RexNode, LogicalExpression> desiredExpressions = Maps.newHashMap();
+
+  //the expressions in equality comparison
+  final Map<RexNode, LogicalExpression> equalityExpressions = Maps.newHashMap();
+
+  //the expression found in non-equality comparison
+  final Map<RexNode, LogicalExpression> notInEquality = Maps.newHashMap();
+
+  //for =(cast(a.b as VARCHAR(len)), 'abcd'), if the 'len' is less than the max length of casted field on index table,
+  // we want to rewrite it to LIKE(cast(a.b as VARCHAR(len)), 'abcd%')
+  //map equalOnCastChar: key is the equal operator, value is the operand (cast(a.b as VARCHAR(10)),
+  final Map<RexNode, LogicalExpression> equalOnCastChar = Maps.newHashMap();
+
+  final private RelNode inputRel;
+
+  //flag current recursive call state: whether we are on a direct operand of comparison operator
+  boolean directCompareOp = false;
+
+  RexCall contextCall = null;
+
+  DrillParseContext parserContext;
+
+  public IndexableExprMarker(RelNode inputRel) {
+    super(true);
+    this.inputRel = inputRel;
+    parserContext = new DrillParseContext(PrelUtil.getPlannerSettings(inputRel.getCluster()));
+  }
+
+  public Map<RexNode, LogicalExpression> getIndexableExpression() {
+    return ImmutableMap.copyOf(desiredExpressions);
+  }
+
+  public Map<RexNode, LogicalExpression> getEqualOnCastChar() {
+    return ImmutableMap.copyOf(equalOnCastChar);
+  }
+
+  /**
+   * return the expressions that were only in equality condition _and_ only once. ( a.b = 'value' )
+   * @return
+   */
+  public Set<LogicalExpression> getExpressionsOnlyInEquality() {
+
+    Set<LogicalExpression> onlyInEquality = Sets.newHashSet();
+
+    Set<LogicalExpression> notInEqSet = Sets.newHashSet();
+
+    Set<LogicalExpression> inEqMoreThanOnce = Sets.newHashSet();
+
+    notInEqSet.addAll(notInEquality.values());
+
+    for (LogicalExpression expr : equalityExpressions.values()) {
+      //only process expr that is not in any non-equality condition(!notInEqSet.contains)
+      if (!notInEqSet.contains(expr)) {
+
+        //expr appear in two and more equality conditions should be ignored too
+        if (inEqMoreThanOnce.contains(expr)) {
+          continue;
+        }
+
+        //we already have recorded this expr in equality condition, move it to inEqMoreThanOnce
+        if (onlyInEquality.contains(expr)) {
+          inEqMoreThanOnce.add(expr);
+          onlyInEquality.remove(expr);
+          continue;
+        }
+
+        //finally we could take this expr
+        onlyInEquality.add(expr);
+      }
+    }
+    return onlyInEquality;
+  }
+
+  @Override
+  public Boolean visitInputRef(RexInputRef rexInputRef) {
+    return directCompareOp;
+  }
+
+  public boolean containInputRef(RexNode rex) {
+    if (rex instanceof RexInputRef) {
+      return true;
+    }
+    if ((rex instanceof RexCall) && "ITEM".equals(((RexCall)rex).getOperator().getName())) {
+      return true;
+    }
+    //TODO: use a visitor search recursively for inputRef, if found one return true
+    return false;
+  }
+
+  public boolean operandsAreIndexable(RexCall call) {
+    SqlKind kind = call.getKind();
+    boolean kindIsRight = (SqlKind.COMPARISON.contains(kind) || kind==SqlKind.LIKE || kind == SqlKind.SIMILAR);
+
+    if (!kindIsRight) {
+      return false;
+    }
+
+    int inputReference = 0;
+    for (RexNode operand : call.operands) {
+      //if for this operator, there are two operands and more have inputRef, which means it is something like:
+      // a.b = a.c, instead of a.b ='hello', so this cannot apply index
+      if (containInputRef(operand)) {
+        inputReference++;
+        if(inputReference>=2) {
+          return false;
+        }
+      }
+    }
+    return true;
+  }
+
+  @Override
+  public Boolean visitCall(RexCall call) {
+    if (call.getKind() == SqlKind.NOT) {
+      // Conditions under NOT are not indexable
+      return false;
+    }
+    if (operandsAreIndexable(call)) {
+      for (RexNode operand : call.operands) {
+        directCompareOp = true;
+        contextCall = call;
+        boolean markIt = operand.accept(this);
+        directCompareOp = false;
+        contextCall = null;
+        if (markIt) {
+          LogicalExpression expr = DrillOptiq.toDrill(parserContext, inputRel, operand);
+          desiredExpressions.put(operand, expr);
+          if (call.getKind() == SqlKind.EQUALS) {
+            equalityExpressions.put(operand, expr);
+          }
+          else {
+            notInEquality.put(operand, expr);
+          }
+        }
+      }
+      return false;
+    }
+
+    //now we are handling a call directly under comparison e.g. <([call], literal)
+    if (directCompareOp) {
+      // if it is an item, or CAST function
+      if ("ITEM".equals(call.getOperator().getName())) {
+        return directCompareOp;
+      }
+      else if (call.getKind() == SqlKind.CAST) {
+        //For now, we care only direct CAST: CAST's operand is a field(schemaPath),
+        // either ITEM call(nested name) or inputRef
+
+        //cast as char/varchar in equals function
+        if(contextCall != null && contextCall.getKind() == SqlKind.EQUALS
+            && (call.getType().getSqlTypeName()== SqlTypeName.CHAR
+                || call.getType().getSqlTypeName()==SqlTypeName.VARCHAR)) {
+          equalOnCastChar.put(contextCall, DrillOptiq.toDrill(parserContext, inputRel, call));
+        }
+
+        RexNode castOp = call.operands.get(0);
+        if (castOp instanceof RexInputRef) {
+          return true;
+        }
+        if ((castOp instanceof RexCall) && ("ITEM".equals(((RexCall)castOp).getOperator().getName()))) {
+          return true;
+        }
+      }
+    }
+
+    for (RexNode operand : call.operands) {
+      boolean bret = operand.accept(this);
+    }
+    return false;
+  }
+
+  public Boolean visitLocalRef(RexLocalRef localRef) {
+    return false;
+  }
+
+  public Boolean visitLiteral(RexLiteral literal) {
+    return false;
+  }
+
+  public Boolean visitOver(RexOver over) {
+    return false;
+  }
+
+  public Boolean visitCorrelVariable(RexCorrelVariable correlVariable) {
+    return false;
+  }
+
+  public Boolean visitDynamicParam(RexDynamicParam dynamicParam) {
+    return false;
+  }
+
+  public Boolean visitRangeRef(RexRangeRef rangeRef) {
+    return false;
+  }
+
+  public Boolean visitFieldAccess(RexFieldAccess fieldAccess) {
+    final RexNode expr = fieldAccess.getReferenceExpr();
+    return expr.accept(this);
+  }
+}
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/index/InvalidIndexDefinitionException.java b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/index/InvalidIndexDefinitionException.java
new file mode 100644
index 0000000..c17d09f
--- /dev/null
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/index/InvalidIndexDefinitionException.java
@@ -0,0 +1,27 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.drill.exec.planner.index;
+
+import org.apache.drill.common.exceptions.DrillRuntimeException;
+
+public class InvalidIndexDefinitionException extends DrillRuntimeException {
+  public InvalidIndexDefinitionException(String message) {
+    super(message);
+  }
+
+}
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
new file mode 100644
index 0000000..2859102
--- /dev/null
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/index/Statistics.java
@@ -0,0 +1,66 @@
+/*
+ * 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.calcite.rel.RelNode;
+import org.apache.calcite.rex.RexNode;
+import org.apache.drill.exec.planner.common.DrillScanRelBase;
+
+public interface Statistics {
+
+  double ROWCOUNT_UNKNOWN = -1;
+  //HUGE is same as DrillCostBase.HUGE
+  double ROWCOUNT_HUGE = Double.MAX_VALUE;
+  double AVG_ROWSIZE_UNKNOWN = -1;
+  long AVG_COLUMN_SIZE = 10;
+
+  /** Returns whether statistics are available. Should be called prior to using the statistics
+   */
+  boolean isStatsAvailable();
+
+  /** Returns a unique index identifier
+   *  @param idx - Index specified as a {@link IndexDescriptor}
+   *  @return The unique index identifier
+   */
+  String buildUniqueIndexIdentifier(IndexDescriptor idx);
+
+  /** Returns the rowcount for the specified filter condition
+   *  @param condition - Filter specified as a {@link RexNode}
+   *  @param tabIdxName - The index name generated using {@code buildUniqueIndexIdentifier}
+   *  @param scanRel - The current scan rel
+   *  @return the rowcount for the filter
+   */
+  double getRowCount(RexNode condition, String tabIdxName, RelNode scanRel);
+
+  /** Returns the leading rowcount for the specified filter condition
+   *  Leading rowcount means rowcount for filter condition only on leading index columns.
+   *  @param condition - Filter specified as a {@link RexNode}
+   *  @param tabIdxName - The index name generated using {@code buildUniqueIndexIdentifier}
+   *  @param scanRel - The current scan rel
+   *  @return the leading rowcount
+   */
+  double getLeadingRowCount(RexNode condition, String tabIdxName, RelNode 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)
+   */
+  double getAvgRowSize(String tabIdxName, boolean isIndexScan);
+
+  boolean initialize(RexNode condition, DrillScanRelBase scanRel, IndexCallContext context);
+}
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/index/StatisticsPayload.java b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/index/StatisticsPayload.java
new file mode 100644
index 0000000..6894e4f
--- /dev/null
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/index/StatisticsPayload.java
@@ -0,0 +1,24 @@
+/*
+ * 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;
+
+public interface StatisticsPayload {
+  double getRowCount();
+  double getLeadingRowCount();
+  double getAvgRowSize();
+}
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/logical/DrillTable.java b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/logical/DrillTable.java
index 53036f1..ed9b32f 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/logical/DrillTable.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/logical/DrillTable.java
@@ -91,6 +91,10 @@ public abstract class DrillTable implements Table {
     this.options = options;
   }
 
+  public void setGroupScan(GroupScan scan) {
+    this.scan = scan;
+  }
+
   public GroupScan getGroupScan() throws IOException{
     if (scan == null) {
       if (selection instanceof FileSelection && ((FileSelection) selection).isEmptyDirectory()) {
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/physical/PartitionFunction.java b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/physical/PartitionFunction.java
new file mode 100644
index 0000000..754c5d7
--- /dev/null
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/physical/PartitionFunction.java
@@ -0,0 +1,56 @@
+/*
+ * 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.util.List;
+
+import org.apache.drill.common.expression.FieldReference;
+import org.apache.drill.exec.record.VectorWrapper;
+
+import com.fasterxml.jackson.annotation.JsonTypeInfo;
+
+@JsonTypeInfo(use=JsonTypeInfo.Id.CLASS, include=JsonTypeInfo.As.PROPERTY, property="@class")
+public interface PartitionFunction  {
+
+  /**
+   * Return the list of FieldReferences that participate in the partitioning function
+   * @return list of FieldReferences
+   */
+  List<FieldReference> getPartitionRefList();
+
+  /**
+   * Setup method for the partitioning function
+   * @param partitionKeys a list of partition columns on which range partitioning is needed
+   */
+  void setup(List<VectorWrapper<?>> partitionKeys);
+
+  /**
+   * Evaluate a partitioning function for a particular row index and return the partition id
+   * @param index the integer index into the partition keys vector for a specific 'row' of values
+   * @param numPartitions the max number of partitions that are allowed
+   * @return partition id, an integer value
+   */
+  int eval(int index, int numPartitions);
+
+  /**
+   * Returns a FieldReference (LogicalExpression) for the partition function
+   * @return FieldReference for the partition function
+   */
+  FieldReference getPartitionFieldRef();
+
+}
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/record/AbstractRecordBatch.java b/exec/java-exec/src/main/java/org/apache/drill/exec/record/AbstractRecordBatch.java
index eb6112d..cb79091 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/record/AbstractRecordBatch.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/record/AbstractRecordBatch.java
@@ -80,7 +80,7 @@ public abstract class AbstractRecordBatch<T extends PhysicalOperator> implements
     }
   }
 
-  protected static enum BatchState {
+  public static enum BatchState {
     /** Need to build schema and return. */
     BUILD_SCHEMA,
     /** This is still the first data batch. */
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/store/AbstractRecordReader.java b/exec/java-exec/src/main/java/org/apache/drill/exec/store/AbstractRecordReader.java
index 9314da6..1bbbe76 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/store/AbstractRecordReader.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/store/AbstractRecordReader.java
@@ -101,6 +101,11 @@ public abstract class AbstractRecordReader implements RecordReader {
     }
   }
 
+  @Override
+  public boolean hasNext() {
+    return false;
+  }
+
   protected List<SchemaPath> getDefaultColumnsToRead() {
     return GroupScan.ALL_COLUMNS;
   }
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/store/RecordReader.java b/exec/java-exec/src/main/java/org/apache/drill/exec/store/RecordReader.java
index edd91d1..33b361c 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/store/RecordReader.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/store/RecordReader.java
@@ -43,6 +43,14 @@ public interface RecordReader extends AutoCloseable {
   void allocate(Map<String, ValueVector> vectorMap) throws OutOfMemoryException;
 
   /**
+   * Check if the reader may have potentially more data to be read in subsequent iterations. Certain types of readers
+   * such as repeatable readers can be invoked multiple times, so this method will allow ScanBatch to check with
+   * the reader before closing it.
+   * @return return true if there could potentially be more reads, false otherwise
+   */
+  boolean hasNext();
+
+  /**
    * Increments this record reader forward, writing via the provided output
    * mutator into the output batch.
    *
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/util/EncodedSchemaPathSet.java b/exec/java-exec/src/main/java/org/apache/drill/exec/util/EncodedSchemaPathSet.java
new file mode 100644
index 0000000..5f9eef8
--- /dev/null
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/util/EncodedSchemaPathSet.java
@@ -0,0 +1,291 @@
+/*
+ * 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.util;
+
+
+import org.apache.drill.shaded.guava.com.google.common.io.BaseEncoding;
+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.common.exceptions.DrillRuntimeException;
+import org.apache.drill.common.expression.SchemaPath;
+import org.apache.drill.exec.planner.physical.PlannerSettings;
+
+import java.io.ByteArrayOutputStream;
+import java.io.IOException;
+import java.io.UnsupportedEncodingException;
+import java.util.Arrays;
+import java.util.Collection;
+import java.util.Iterator;
+import java.util.List;
+
+/**
+ * This class provided utility methods to encode and decode a set of user specified
+ * SchemaPaths to a set of encoded SchemaPaths with the following properties.
+ * <ol>
+ * <li>Valid Drill identifier as per its grammar with only one, root name segment.
+ * <li>A single identifier can not exceed 1024 characters in length.
+ * </ol>
+ * <p>
+ * Format of the encoded SchemaPath:
+ * <blockquote><pre>$$ENC\d\dlt;base32 encoded input paths&gt;</pre></blockquote>
+ * <p>
+ * We use Base-32 over Base-64 because the later's charset includes '\' and '+'.
+ */
+public class EncodedSchemaPathSet {
+
+  private static final int ESTIMATED_ENCODED_SIZE = 1024;
+
+  private static final String ENC_PREFIX = "$$ENC";
+
+  private static final String ENC_FORMAT_STRING = ENC_PREFIX + "%02d%s";
+  private static final int ENC_PREFIX_SIZE = ENC_PREFIX.length() + "00".length();
+  private static final int MAX_ENC_IDENTIFIER_SIZE = (PlannerSettings.DEFAULT_IDENTIFIER_MAX_LENGTH - ENC_PREFIX_SIZE);
+  private static final int MAX_ENC_IDENTIFIER_COUNT = 100; // "$$ENC00*...$$ENC99*"
+
+  private static final BaseEncoding CODEC = BaseEncoding.base32().omitPadding(); // no-padding version
+
+  public static final String ENCODED_STAR_COLUMN = encode("*")[0];
+
+  /*
+   * Performance of various methods of encoding a Java String to UTF-8 keeps changing
+   * between releases, hence we'll encapsulate the actual methods within these functions
+   * and use them everywhere in Drill
+   */
+  private static final String UTF_8 = "utf-8";
+
+
+  private static byte[] encodeUTF(String input) {
+    try {
+      return input.getBytes(UTF_8);
+    } catch (UnsupportedEncodingException e) {
+      throw new DrillRuntimeException(e); // should never come to this
+    }
+  }
+
+  private static String decodeUTF(byte[] input) {
+    try {
+      return new String(input, UTF_8);
+    } catch (UnsupportedEncodingException e) {
+      throw new DrillRuntimeException(e); // should never come to this
+    }
+  }
+
+  private static String decodeUTF(byte[] input, int offset, int length) {
+    try {
+      return new String(input, offset, length, UTF_8);
+    } catch (UnsupportedEncodingException e) {
+      throw new DrillRuntimeException(e); // should never come to this
+    }
+  }
+
+  /**
+   * Returns the encoded array of SchemaPath identifiers from the input array of SchemaPath.
+   * <p>
+   * The returned identifiers have the following properties:
+   * <ul>
+   *  <li>Each SchemaPath identifier in the array has only one single root NameSegment.</li>
+   *  <li>Maximum length of each such identifier is equal to the maximum length of Drill identifier (currently 1024).</li>
+   * </ul>
+   * <p>
+   * We take advantage of the fact that Java's modified utf-8 encoding can never contain
+   * embedded null byte.
+   * @see <a>http://docs.oracle.com/javase/8/docs/api/java/io/DataInput.html#modified-utf-8</a>
+   */
+  public static String[] encode(final String... schemaPaths) {
+    Preconditions.checkArgument(schemaPaths != null && schemaPaths.length > 0,
+        "At least one schema path should be provided");
+
+    NoCopyByteArrayOutputStream out = new NoCopyByteArrayOutputStream(ESTIMATED_ENCODED_SIZE);
+    int bufOffset = 1; // 1st byte is NULL
+    for (String schemaPath : schemaPaths) {
+      out.write(0);
+      out.write(encodeUTF(schemaPath));
+    }
+    out.close();
+
+    final int bufLen = out.size() - 1; // not counting the first NULL byte
+    String encodedStr = CODEC.encode(out.getBuffer(), bufOffset, bufLen);
+    assert !encodedStr.endsWith("=") : String.format("Encoded string '%s' ends with '='", encodedStr);
+    return splitIdentifiers(encodedStr);
+  }
+
+  public static boolean isEncodedSchemaPath(SchemaPath schemaPath) {
+    return schemaPath != null && isEncodedSchemaPath(schemaPath.getRootSegment().getNameSegment().getPath());
+  }
+
+  public static boolean isEncodedSchemaPath(String schemaPath) {
+    return schemaPath != null && schemaPath.startsWith(ENC_PREFIX);
+  }
+
+  /**
+   * Returns the decoded Collection of SchemaPath from the input which
+   * may contain a mix of encoded and non-encoded SchemaPaths.
+   * <p>
+   * The size of returned Collection is always equal to or greater than the
+   * input array.
+   * <p>
+   * The non-encoded SchemaPaths are collated in the beginning to the returned
+   * array, in the same order as that of the input array.
+   */
+  public static Collection<SchemaPath> decode(final Collection<SchemaPath> encodedPaths) {
+    String[] schemaPathStrings = new String[encodedPaths.size()];
+    Iterator<SchemaPath> encodedPathsItr = encodedPaths.iterator();
+    for (int i = 0; i < schemaPathStrings.length; i++) {
+      SchemaPath schemaPath = encodedPathsItr.next();
+      if (schemaPath.getRootSegmentPath().startsWith(ENC_PREFIX)) {
+        // encoded schema path contains only root segment
+        schemaPathStrings[i] = schemaPath.getRootSegmentPath();
+      } else {
+        schemaPathStrings[i] = schemaPath.toExpr();
+      }
+    }
+    String[] decodedStrings = decode(schemaPathStrings);
+    if (decodedStrings == schemaPathStrings) {
+      return encodedPaths; // return the original collection as no encoded SchemaPath was found
+    } else {
+      ImmutableList.Builder<SchemaPath> builder = new ImmutableList.Builder<>();
+      for (String decodedString : decodedStrings) {
+        if ("*".equals(decodedString) || "`*`".equals(decodedString)) {
+          builder.add(SchemaPath.STAR_COLUMN);
+        } else {
+          builder.add(SchemaPath.parseFromString(decodedString));
+        }
+      }
+      return builder.build();
+    }
+  }
+
+  /**
+   * Returns the decoded array of SchemaPath strings from the input which
+   * may contain a mix of encoded and non-encoded SchemaPaths.
+   * <p>
+   * The size of returned array is always equal to or greater than the
+   * input array.
+   * <p>
+   * The non-encoded SchemaPaths are collated in the beginning to the returned
+   * array, in the same order as that of the input array.
+   */
+  public static String[] decode(final String... encodedPaths) {
+    Preconditions.checkArgument(encodedPaths != null && encodedPaths.length > 0,
+        "At least one encoded path should be provided");
+
+    StringBuilder sb = new StringBuilder(ESTIMATED_ENCODED_SIZE);
+
+    // As the encoded schema path move across components, they could get reordered.
+    // Sorting ensures that the original order is restored before concatenating the
+    // components back to the full encoded String.
+    Arrays.sort(encodedPaths);
+
+    List<String> decodedPathList = Lists.newArrayList();
+    for (String encodedPath : encodedPaths) {
+      if (encodedPath.startsWith(ENC_PREFIX)) {
+        sb.append(encodedPath, ENC_PREFIX_SIZE, encodedPath.length());
+      } else {
+        decodedPathList.add(encodedPath);
+      }
+    }
+
+    if (sb.length() > 0) {
+      byte[] decodedBytes;
+      try {
+        decodedBytes = CODEC.decode(sb);
+      } catch (IllegalArgumentException e) {
+        throw new DrillRuntimeException(String.format(
+            "Unable to decode the input strings as encoded schema paths:\n%s", Arrays.asList(encodedPaths)), e);
+      }
+
+      int start = 0, index = 0;
+      for (; index < decodedBytes.length; index++) {
+        if (decodedBytes[index] == 0 && index - start > 0) {
+          decodedPathList.add(decodeUTF(decodedBytes, start, index-start));
+          start = index + 1;
+        }
+      }
+      if (index - start > 0) {
+        String lastSchemaPath = decodeUTF(decodedBytes, start, index-start).trim();
+        if (!lastSchemaPath.isEmpty()) {
+          decodedPathList.add(lastSchemaPath);
+        }
+      }
+      return decodedPathList.toArray(new String[decodedPathList.size()]);
+    } else {
+      // original list did not have any encoded path, return as is
+      return encodedPaths;
+    }
+  }
+
+  /**
+   * Splits the input string so that the length of each encoded string,
+   * including the signature prefix is less than or equal to MAX_DRILL_IDENTIFIER_SIZE.
+   */
+  private static String[] splitIdentifiers(String input) {
+    if (input.length() < MAX_ENC_IDENTIFIER_SIZE) {
+      return new String[] { String.format(ENC_FORMAT_STRING, 0, input) };
+    }
+    int splitsCount = (int) Math.ceil(input.length() / (double)MAX_ENC_IDENTIFIER_SIZE);
+    if (splitsCount > MAX_ENC_IDENTIFIER_COUNT) {
+      throw new DrillRuntimeException(String.format(
+          "Encoded size of the SchemaPath identifier '%s' exceeded maximum value.", input));
+    }
+    String[] result = new String[splitsCount];
+    for (int i = 0, startIdx = 0; i < result.length; i++, startIdx += MAX_ENC_IDENTIFIER_SIZE) {
+      // TODO: see if we can avoid memcpy due to input.substring() call
+      result[i] = String.format(ENC_FORMAT_STRING, i, input.substring(startIdx, Math.min(input.length(), startIdx + MAX_ENC_IDENTIFIER_SIZE)));
+    }
+    return result;
+  }
+
+  /**
+   * Optimized version of Java's ByteArrayOutputStream which returns the underlying
+   * byte array instead of making a copy
+   */
+  private static class NoCopyByteArrayOutputStream extends ByteArrayOutputStream {
+    public NoCopyByteArrayOutputStream(int size) {
+      super(size);
+    }
+
+    public byte[] getBuffer() {
+      return buf;
+    }
+
+    public int size() {
+      return count;
+    }
+
+    @Override
+    public void write(int b) {
+      super.write(b);
+    }
+
+    @Override
+    public void write(byte[] b) {
+      super.write(b, 0, b.length);
+    }
+
+    @Override
+    public void close() {
+      try {
+        super.close();
+      } catch (IOException e) {
+        throw new DrillRuntimeException(e); // should never come to this
+      }
+    }
+  }
+
+}
diff --git a/protocol/src/main/java/org/apache/drill/exec/proto/UserBitShared.java b/protocol/src/main/java/org/apache/drill/exec/proto/UserBitShared.java
index d346182..2f5c3de 100644
--- a/protocol/src/main/java/org/apache/drill/exec/proto/UserBitShared.java
+++ b/protocol/src/main/java/org/apache/drill/exec/proto/UserBitShared.java
@@ -414,9 +414,9 @@ public final class UserBitShared {
      */
     UNORDERED_RECEIVER(11, 11),
     /**
-     * <code>RANGE_SENDER = 12;</code>
+     * <code>RANGE_PARTITION_SENDER = 12;</code>
      */
-    RANGE_SENDER(12, 12),
+    RANGE_PARTITION_SENDER(12, 12),
     /**
      * <code>SCREEN = 13;</code>
      */
@@ -593,6 +593,10 @@ public final class UserBitShared {
      * <code>RUNTIME_FILTER = 56;</code>
      */
     RUNTIME_FILTER(56, 56),
+    /**
+     * <code>ROWKEY_JOIN = 57;</code>
+     */
+    ROWKEY_JOIN(57, 57),
     ;
 
     /**
@@ -644,9 +648,9 @@ public final class UserBitShared {
      */
     public static final int UNORDERED_RECEIVER_VALUE = 11;
     /**
-     * <code>RANGE_SENDER = 12;</code>
+     * <code>RANGE_PARTITION_SENDER = 12;</code>
      */
-    public static final int RANGE_SENDER_VALUE = 12;
+    public static final int RANGE_PARTITION_SENDER_VALUE = 12;
     /**
      * <code>SCREEN = 13;</code>
      */
@@ -823,6 +827,10 @@ public final class UserBitShared {
      * <code>RUNTIME_FILTER = 56;</code>
      */
     public static final int RUNTIME_FILTER_VALUE = 56;
+    /**
+     * <code>ROWKEY_JOIN = 57;</code>
+     */
+    public static final int ROWKEY_JOIN_VALUE = 57;
 
 
     public final int getNumber() { return value; }
@@ -841,7 +849,7 @@ public final class UserBitShared {
         case 9: return ORDERED_PARTITION_SENDER;
         case 10: return PROJECT;
         case 11: return UNORDERED_RECEIVER;
-        case 12: return RANGE_SENDER;
+        case 12: return RANGE_PARTITION_SENDER;
         case 13: return SCREEN;
         case 14: return SELECTION_VECTOR_REMOVER;
         case 15: return STREAMING_AGGREGATE;
@@ -886,6 +894,7 @@ public final class UserBitShared {
         case 54: return PARTITION_LIMIT;
         case 55: return PCAPNG_SUB_SCAN;
         case 56: return RUNTIME_FILTER;
+        case 57: return ROWKEY_JOIN;
         default: return null;
       }
     }
@@ -24422,40 +24431,40 @@ public final class UserBitShared {
       "TATEMENT\020\005*\207\001\n\rFragmentState\022\013\n\007SENDING\020" +
       "\000\022\027\n\023AWAITING_ALLOCATION\020\001\022\013\n\007RUNNING\020\002\022" +
       "\014\n\010FINISHED\020\003\022\r\n\tCANCELLED\020\004\022\n\n\006FAILED\020\005" +
-      "\022\032\n\026CANCELLATION_REQUESTED\020\006*\367\010\n\020CoreOpe" +
+      "\022\032\n\026CANCELLATION_REQUESTED\020\006*\222\t\n\020CoreOpe" +
       "ratorType\022\021\n\rSINGLE_SENDER\020\000\022\024\n\020BROADCAS" +
       "T_SENDER\020\001\022\n\n\006FILTER\020\002\022\022\n\016HASH_AGGREGATE" +
       "\020\003\022\r\n\tHASH_JOIN\020\004\022\016\n\nMERGE_JOIN\020\005\022\031\n\025HAS" +
       "H_PARTITION_SENDER\020\006\022\t\n\005LIMIT\020\007\022\024\n\020MERGI" +
       "NG_RECEIVER\020\010\022\034\n\030ORDERED_PARTITION_SENDE" +
       "R\020\t\022\013\n\007PROJECT\020\n\022\026\n\022UNORDERED_RECEIVER\020\013",
-      "\022\020\n\014RANGE_SENDER\020\014\022\n\n\006SCREEN\020\r\022\034\n\030SELECT" +
-      "ION_VECTOR_REMOVER\020\016\022\027\n\023STREAMING_AGGREG" +
-      "ATE\020\017\022\016\n\nTOP_N_SORT\020\020\022\021\n\rEXTERNAL_SORT\020\021" +
-      "\022\t\n\005TRACE\020\022\022\t\n\005UNION\020\023\022\014\n\010OLD_SORT\020\024\022\032\n\026" +
-      "PARQUET_ROW_GROUP_SCAN\020\025\022\021\n\rHIVE_SUB_SCA" +
-      "N\020\026\022\025\n\021SYSTEM_TABLE_SCAN\020\027\022\021\n\rMOCK_SUB_S" +
-      "CAN\020\030\022\022\n\016PARQUET_WRITER\020\031\022\023\n\017DIRECT_SUB_" +
-      "SCAN\020\032\022\017\n\013TEXT_WRITER\020\033\022\021\n\rTEXT_SUB_SCAN" +
-      "\020\034\022\021\n\rJSON_SUB_SCAN\020\035\022\030\n\024INFO_SCHEMA_SUB" +
-      "_SCAN\020\036\022\023\n\017COMPLEX_TO_JSON\020\037\022\025\n\021PRODUCER",
-      "_CONSUMER\020 \022\022\n\016HBASE_SUB_SCAN\020!\022\n\n\006WINDO" +
-      "W\020\"\022\024\n\020NESTED_LOOP_JOIN\020#\022\021\n\rAVRO_SUB_SC" +
-      "AN\020$\022\021\n\rPCAP_SUB_SCAN\020%\022\022\n\016KAFKA_SUB_SCA" +
-      "N\020&\022\021\n\rKUDU_SUB_SCAN\020\'\022\013\n\007FLATTEN\020(\022\020\n\014L" +
-      "ATERAL_JOIN\020)\022\n\n\006UNNEST\020*\022,\n(HIVE_DRILL_" +
-      "NATIVE_PARQUET_ROW_GROUP_SCAN\020+\022\r\n\tJDBC_" +
-      "SCAN\020,\022\022\n\016REGEX_SUB_SCAN\020-\022\023\n\017MAPRDB_SUB" +
-      "_SCAN\020.\022\022\n\016MONGO_SUB_SCAN\020/\022\017\n\013KUDU_WRIT" +
-      "ER\0200\022\026\n\022OPEN_TSDB_SUB_SCAN\0201\022\017\n\013JSON_WRI" +
-      "TER\0202\022\026\n\022HTPPD_LOG_SUB_SCAN\0203\022\022\n\016IMAGE_S",
-      "UB_SCAN\0204\022\025\n\021SEQUENCE_SUB_SCAN\0205\022\023\n\017PART" +
-      "ITION_LIMIT\0206\022\023\n\017PCAPNG_SUB_SCAN\0207\022\022\n\016RU" +
-      "NTIME_FILTER\0208*g\n\nSaslStatus\022\020\n\014SASL_UNK" +
-      "NOWN\020\000\022\016\n\nSASL_START\020\001\022\024\n\020SASL_IN_PROGRE" +
-      "SS\020\002\022\020\n\014SASL_SUCCESS\020\003\022\017\n\013SASL_FAILED\020\004B" +
-      ".\n\033org.apache.drill.exec.protoB\rUserBitS" +
-      "haredH\001"
+      "\022\032\n\026RANGE_PARTITION_SENDER\020\014\022\n\n\006SCREEN\020\r" +
+      "\022\034\n\030SELECTION_VECTOR_REMOVER\020\016\022\027\n\023STREAM" +
+      "ING_AGGREGATE\020\017\022\016\n\nTOP_N_SORT\020\020\022\021\n\rEXTER" +
+      "NAL_SORT\020\021\022\t\n\005TRACE\020\022\022\t\n\005UNION\020\023\022\014\n\010OLD_" +
+      "SORT\020\024\022\032\n\026PARQUET_ROW_GROUP_SCAN\020\025\022\021\n\rHI" +
+      "VE_SUB_SCAN\020\026\022\025\n\021SYSTEM_TABLE_SCAN\020\027\022\021\n\r" +
+      "MOCK_SUB_SCAN\020\030\022\022\n\016PARQUET_WRITER\020\031\022\023\n\017D" +
+      "IRECT_SUB_SCAN\020\032\022\017\n\013TEXT_WRITER\020\033\022\021\n\rTEX" +
+      "T_SUB_SCAN\020\034\022\021\n\rJSON_SUB_SCAN\020\035\022\030\n\024INFO_" +
+      "SCHEMA_SUB_SCAN\020\036\022\023\n\017COMPLEX_TO_JSON\020\037\022\025",
+      "\n\021PRODUCER_CONSUMER\020 \022\022\n\016HBASE_SUB_SCAN\020" +
+      "!\022\n\n\006WINDOW\020\"\022\024\n\020NESTED_LOOP_JOIN\020#\022\021\n\rA" +
+      "VRO_SUB_SCAN\020$\022\021\n\rPCAP_SUB_SCAN\020%\022\022\n\016KAF" +
+      "KA_SUB_SCAN\020&\022\021\n\rKUDU_SUB_SCAN\020\'\022\013\n\007FLAT" +
+      "TEN\020(\022\020\n\014LATERAL_JOIN\020)\022\n\n\006UNNEST\020*\022,\n(H" +
+      "IVE_DRILL_NATIVE_PARQUET_ROW_GROUP_SCAN\020" +
+      "+\022\r\n\tJDBC_SCAN\020,\022\022\n\016REGEX_SUB_SCAN\020-\022\023\n\017" +
+      "MAPRDB_SUB_SCAN\020.\022\022\n\016MONGO_SUB_SCAN\020/\022\017\n" +
+      "\013KUDU_WRITER\0200\022\026\n\022OPEN_TSDB_SUB_SCAN\0201\022\017" +
+      "\n\013JSON_WRITER\0202\022\026\n\022HTPPD_LOG_SUB_SCAN\0203\022",
+      "\022\n\016IMAGE_SUB_SCAN\0204\022\025\n\021SEQUENCE_SUB_SCAN" +
+      "\0205\022\023\n\017PARTITION_LIMIT\0206\022\023\n\017PCAPNG_SUB_SC" +
+      "AN\0207\022\022\n\016RUNTIME_FILTER\0208\022\017\n\013ROWKEY_JOIN\020" +
+      "9*g\n\nSaslStatus\022\020\n\014SASL_UNKNOWN\020\000\022\016\n\nSAS" +
+      "L_START\020\001\022\024\n\020SASL_IN_PROGRESS\020\002\022\020\n\014SASL_" +
+      "SUCCESS\020\003\022\017\n\013SASL_FAILED\020\004B.\n\033org.apache" +
+      ".drill.exec.protoB\rUserBitSharedH\001"
     };
     com.google.protobuf.Descriptors.FileDescriptor.InternalDescriptorAssigner assigner =
       new com.google.protobuf.Descriptors.FileDescriptor.InternalDescriptorAssigner() {
diff --git a/protocol/src/main/java/org/apache/drill/exec/proto/beans/CoreOperatorType.java b/protocol/src/main/java/org/apache/drill/exec/proto/beans/CoreOperatorType.java
index 6b36fb4..6138ad6 100644
--- a/protocol/src/main/java/org/apache/drill/exec/proto/beans/CoreOperatorType.java
+++ b/protocol/src/main/java/org/apache/drill/exec/proto/beans/CoreOperatorType.java
@@ -34,7 +34,7 @@ public enum CoreOperatorType implements com.dyuproject.protostuff.EnumLite<CoreO
     ORDERED_PARTITION_SENDER(9),
     PROJECT(10),
     UNORDERED_RECEIVER(11),
-    RANGE_SENDER(12),
+    RANGE_PARTITION_SENDER(12),
     SCREEN(13),
     SELECTION_VECTOR_REMOVER(14),
     STREAMING_AGGREGATE(15),
@@ -78,7 +78,8 @@ public enum CoreOperatorType implements com.dyuproject.protostuff.EnumLite<CoreO
     SEQUENCE_SUB_SCAN(53),
     PARTITION_LIMIT(54),
     PCAPNG_SUB_SCAN(55),
-    RUNTIME_FILTER(56);
+    RUNTIME_FILTER(56),
+    ROWKEY_JOIN(57);
     
     public final int number;
     
@@ -108,7 +109,7 @@ public enum CoreOperatorType implements com.dyuproject.protostuff.EnumLite<CoreO
             case 9: return ORDERED_PARTITION_SENDER;
             case 10: return PROJECT;
             case 11: return UNORDERED_RECEIVER;
-            case 12: return RANGE_SENDER;
+            case 12: return RANGE_PARTITION_SENDER;
             case 13: return SCREEN;
             case 14: return SELECTION_VECTOR_REMOVER;
             case 15: return STREAMING_AGGREGATE;
@@ -153,6 +154,7 @@ public enum CoreOperatorType implements com.dyuproject.protostuff.EnumLite<CoreO
             case 54: return PARTITION_LIMIT;
             case 55: return PCAPNG_SUB_SCAN;
             case 56: return RUNTIME_FILTER;
+            case 57: return ROWKEY_JOIN;
             default: return null;
         }
     }
diff --git a/protocol/src/main/protobuf/UserBitShared.proto b/protocol/src/main/protobuf/UserBitShared.proto
index 16d44c3..843c6d8 100644
--- a/protocol/src/main/protobuf/UserBitShared.proto
+++ b/protocol/src/main/protobuf/UserBitShared.proto
@@ -300,7 +300,7 @@ enum CoreOperatorType {
   ORDERED_PARTITION_SENDER = 9;
   PROJECT = 10;
   UNORDERED_RECEIVER = 11;
-  RANGE_SENDER = 12;
+  RANGE_PARTITION_SENDER = 12;
   SCREEN = 13;
   SELECTION_VECTOR_REMOVER = 14;
   STREAMING_AGGREGATE = 15;
@@ -345,6 +345,7 @@ enum CoreOperatorType {
   PARTITION_LIMIT = 54;
   PCAPNG_SUB_SCAN = 55;
   RUNTIME_FILTER = 56;
+  ROWKEY_JOIN = 57;
 }
 
 /* Registry that contains list of jars, each jar contains its name and list of function signatures.