You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@carbondata.apache.org by qi...@apache.org on 2020/04/01 12:09:11 UTC

[carbondata] branch master updated: [CARBONDATA-3704] Support create materialized view on all type table, and make mv support mutil-tenant.

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

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


The following commit(s) were added to refs/heads/master by this push:
     new 3cbe8bc  [CARBONDATA-3704] Support create materialized view on all type table, and make mv support mutil-tenant.
3cbe8bc is described below

commit 3cbe8bcbd8cb732a4b0b6acf1c063eba57f440a6
Author: Jacky Li <ja...@qq.com>
AuthorDate: Sat Mar 7 09:33:06 2020 +0800

    [CARBONDATA-3704] Support create materialized view on all type table, and make mv support mutil-tenant.
    
    Why is this PR needed?
    Support create materialized view on all table, include parquest table, orc table, hive table and carbon table.
    Materialized view DDL is common in databases, carbondata should change its materialized view related SQL syntax as other database.
    Materialized view support mutil-tenant.
    
    What changes were proposed in this PR?
    Define materialized view related commands: CREATE MATERIALIZED VIEW, DROP MATERIALIZED VIEW, REFRESH MATERIALIZED VIEW and SHOW MATERIALIZED VIEW.
    Move materialized view schema files to each database directory.
    Support create materialized view on all table, remove carbon table related check.
    Include some datamap rename change, rename datamap to index.
    
    This closes #3661
    
    Co-authored-by: niuge01 <37...@qq.com>
---
 ...on.java => MalformedIndexCommandException.java} |    8 +-
 ...ption.java => MalformedMVCommandException.java} |    6 +-
 ...andException.java => NoSuchIndexException.java} |   17 +-
 ...ommandException.java => NoSuchMVException.java} |   13 +-
 .../carbondata/core/cache/CacheProvider.java       |    4 +-
 .../core/constants/CarbonCommonConstants.java      |   15 +-
 ...stractDataMapJob.java => AbstractIndexJob.java} |    8 +-
 .../carbondata/core/datamap/DataMapProvider.java   |   32 +-
 .../core/datamap/DataMapStoreManager.java          |  298 ++--
 .../{DataMapChooser.java => IndexChooser.java}     |  136 +-
 .../{DataMapFilter.java => IndexFilter.java}       |   14 +-
 ...bleDataMapFormat.java => IndexInputFormat.java} |   52 +-
 ...aMapDistributable.java => IndexInputSplit.java} |    4 +-
 .../datamap/{DataMapJob.java => IndexJob.java}     |   12 +-
 .../datamap/{DataMapLevel.java => IndexLevel.java} |    6 +-
 .../datamap/{DataMapMeta.java => IndexMeta.java}   |   12 +-
 .../{DataMapRegistry.java => IndexRegistry.java}   |   36 +-
 .../datamap/{DataMapUtil.java => IndexUtil.java}   |  125 +-
 .../{DataMapCatalog.java => MVCatalog.java}        |    4 +-
 ...entDataMapGroup.java => SegmentIndexGroup.java} |    8 +-
 .../datamap/{TableDataMap.java => TableIndex.java} |  186 +--
 .../core/datamap/dev/BlockletSerializer.java       |    2 +-
 .../{CacheableDataMap.java => CacheableIndex.java} |   20 +-
 .../core/datamap/dev/DataMapSyncStatus.java        |   10 +-
 .../core/datamap/dev/{DataMap.java => Index.java}  |   12 +-
 .../dev/{DataMapBuilder.java => IndexBuilder.java} |   10 +-
 .../dev/{DataMapFactory.java => IndexFactory.java} |   94 +-
 .../dev/{DataMapModel.java => IndexModel.java}     |    6 +-
 .../dev/{DataMapWriter.java => IndexWriter.java}   |    8 +-
 ...arseGrainDataMap.java => CoarseGrainIndex.java} |   10 +-
 ...apFactory.java => CoarseGrainIndexFactory.java} |   16 +-
 ...apExprWrapper.java => AndIndexExprWrapper.java} |   24 +-
 ...taMapExprWrapper.java => IndexExprWrapper.java} |   18 +-
 ...rWrapperImpl.java => IndexExprWrapperImpl.java} |   44 +-
 ...bleWrapper.java => IndexInputSplitWrapper.java} |   10 +-
 ...SimpleInfo.java => IndexWrapperSimpleInfo.java} |   36 +-
 ...MapExprWrapper.java => OrIndexExprWrapper.java} |   24 +-
 .../datamap/dev/fgdatamap/FineGrainBlocklet.java   |    4 +-
 .../{FineGrainDataMap.java => FineGrainIndex.java} |    8 +-
 ...aMapFactory.java => FineGrainIndexFactory.java} |   16 +-
 .../core/datamap/status/DataMapStatus.java         |    2 +-
 .../status/DatabaseDataMapStatusProvider.java      |    2 +-
 .../status/DiskBasedDataMapStatusProvider.java     |    2 +-
 ...entStatusUtil.java => MVSegmentStatusUtil.java} |    2 +-
 .../core/datastore/filesystem/LocalCarbonFile.java |    4 +-
 .../core/datastore/impl/FileFactory.java           |   35 +
 .../core/indexstore/AbstractMemoryDMStore.java     |    8 +-
 .../carbondata/core/indexstore/Blocklet.java       |    2 +-
 .../core/indexstore/BlockletDetailInfo.java        |    6 +-
 ...aMapIndexStore.java => BlockletIndexStore.java} |  114 +-
 ...IndexWrapper.java => BlockletIndexWrapper.java} |   18 +-
 .../core/indexstore/ExtendedBlocklet.java          |    6 +-
 .../core/indexstore/SafeMemoryDMStore.java         |   30 +-
 .../core/indexstore/SegmentPropertiesFetcher.java  |    4 +-
 .../core/indexstore/UnsafeMemoryDMStore.java       |   16 +-
 .../{BlockDataMap.java => BlockIndex.java}         |  138 +-
 .../{BlockletDataMap.java => BlockletIndex.java}   |   48 +-
 ...taMapFactory.java => BlockletIndexFactory.java} |  208 +--
 ...ributable.java => BlockletIndexInputSplit.java} |    8 +-
 ...etDataMapModel.java => BlockletIndexModel.java} |   10 +-
 ...owIndexes.java => BlockletIndexRowIndexes.java} |    6 +-
 .../row/{DataMapRow.java => IndexRow.java}         |   10 +-
 .../row/{DataMapRowImpl.java => IndexRowImpl.java} |   12 +-
 .../{UnsafeDataMapRow.java => UnsafeIndexRow.java} |   10 +-
 .../core/indexstore/schema/SchemaGenerator.java    |    8 +-
 .../carbondata/core/locks/CarbonLockUtil.java      |    2 +-
 .../apache/carbondata/core/locks/LockUsage.java    |    1 +
 .../carbondata/core/metadata/SegmentFileStore.java |    8 +-
 .../schema/datamap/DataMapClassProvider.java       |   24 +-
 .../metadata/schema/datamap/DataMapProperty.java   |    2 +-
 .../schema/datamap/MVProviderName.java}            |    9 +-
 .../core/metadata/schema/table/CarbonTable.java    |   54 +-
 .../core/metadata/schema/table/DataMapSchema.java  |   30 +-
 .../table/DatabaseDMSchemaStorageProvider.java     |    4 +-
 .../table/DiskBasedDMSchemaStorageProvider.java    |   10 +-
 .../carbondata/core/profiler/ExplainCollector.java |   12 +-
 .../carbondata/core/profiler/TablePruningInfo.java |   20 +-
 .../scan/executor/impl/AbstractQueryExecutor.java  |   26 +-
 .../carbondata/core/scan/model/QueryModel.java     |   14 +-
 .../core/scan/model/QueryModelBuilder.java         |   16 +-
 .../core/statusmanager/SegmentStatusManager.java   |   19 +
 ...kletDataMapUtil.java => BlockletIndexUtil.java} |   12 +-
 .../carbondata/core/util/CarbonProperties.java     |   15 +-
 .../apache/carbondata/core/util/CarbonUtil.java    |    2 +-
 .../carbondata/core/util/DeleteLoadFolders.java    |   10 +-
 .../apache/carbondata/core/util/SessionParams.java |    2 +
 .../DataMapCatalog.java => view/MVCatalog.java}    |   35 +-
 .../MVCatalogFactory.java}                         |   13 +-
 .../org/apache/carbondata/core/view/MVManager.java |  330 +++++
 .../DataMapProperty.java => view/MVProperty.java}  |   34 +-
 .../apache/carbondata/core/view/MVProvider.java    |  573 ++++++++
 .../org/apache/carbondata/core/view/MVSchema.java  |  285 ++++
 .../DataMapStatus.java => view/MVStatus.java}      |    6 +-
 .../carbondata/core/view/MVStatusDetail.java       |   44 +-
 .../apache/carbondata/hadoop/CarbonInputSplit.java |   66 +-
 ...BlockletDataMap.java => TestBlockletIndex.java} |    6 +-
 ...pFactory.java => TestBlockletIndexFactory.java} |   42 +-
 dev/findbugs-exclude.xml                           |   10 +-
 docs/faq.md                                        |    7 +-
 docs/index-developer-guide.md                      |    4 +-
 docs/index/lucene-index-guide.md                   |    8 +-
 .../apache/carbondata/examples/FlinkExample.scala  |    5 +-
 ...taMapExample.scala => LuceneIndexExample.scala} |   11 +-
 .../org/apache/carbondata/examples/MVExample.scala |    8 +-
 .../apache/carbondata/examplesCI/RunExamples.scala |    2 +-
 .../carbondata/hadoop/CarbonRecordReader.java      |    2 +-
 .../hadoop/api/CarbonFileInputFormat.java          |    8 +-
 .../carbondata/hadoop/api/CarbonInputFormat.java   |  124 +-
 .../hadoop/api/CarbonOutputCommitter.java          |    6 +-
 .../hadoop/api/CarbonTableInputFormat.java         |   38 +-
 .../hadoop/stream/StreamRecordReader.java          |    4 +-
 .../carbondata/hadoop/testutil/StoreCreator.java   |    2 +-
 .../hadoop/util/CarbonInputFormatUtil.java         |    8 +-
 .../hadoop/ft/CarbonTableInputFormatTest.java      |    6 +-
 ...apWriter.java => AbstractBloomIndexWriter.java} |    9 +-
 ...rainDataMap.java => BloomCoarseGrainIndex.java} |   20 +-
 ...tory.java => BloomCoarseGrainIndexFactory.java} |  140 +-
 ...mDataMapBuilder.java => BloomIndexBuilder.java} |    8 +-
 ...BloomDataMapCache.java => BloomIndexCache.java} |    4 +-
 ...istributable.java => BloomIndexInputSplit.java} |    6 +-
 ...BloomDataMapModel.java => BloomIndexModel.java} |    6 +-
 ...oomDataMapWriter.java => BloomIndexWriter.java} |    4 +-
 .../datamap/examples/MinMaxIndexDataMap.java       |    2 +-
 .../examples/MinMaxIndexDataMapFactory.java        |    6 +-
 .../datamap/examples/MinMaxDataMapSuite.scala      |    2 +-
 ...GrainDataMap.java => LuceneFineGrainIndex.java} |   18 +-
 ...ctory.java => LuceneFineGrainIndexFactory.java} |   48 +-
 ...DataMapBuilder.java => LuceneIndexBuilder.java} |   20 +-
 ...actoryBase.java => LuceneIndexFactoryBase.java} |  104 +-
 ...stributable.java => LuceneIndexInputSplit.java} |    6 +-
 ...neDataMapWriter.java => LuceneIndexWriter.java} |   28 +-
 .../CarbonDataFileMergeTestCaseOnSI.scala          |   22 +-
 .../testsuite/secondaryindex/DropTableTest.scala   |    2 +-
 ...a => TestAlterTableColumnRenameWithIndex.scala} |    2 +-
 .../TestBroadCastSIFilterPushJoinWithUDF.scala     |    2 +-
 ...econdaryIndex.scala => TestCTASWithIndex.scala} |    2 +-
 .../TestCreateIndexWithLoadAndCompaction.scala     |    2 +-
 ...ala => TestIndexModelForORFilterPushDown.scala} |    2 +-
 ...es.scala => TestIndexModelWithAggQueries.scala} |   21 +-
 ...exWithIUD.scala => TestIndexModelWithIUD.scala} |    2 +-
 ...ala => TestIndexModelWithLocalDictionary.scala} |    2 +-
 ...la => TestIndexModelWithUnsafeColumnPage.scala} |    2 +-
 ...ithIndexModelOnFirstColumnAndSortColumns.scala} |    2 +-
 ...aryIndex.scala => TestLikeQueryWithIndex.scala} |    2 +-
 ...ndaryIndex.scala => TestNIQueryWithIndex.scala} |    2 +-
 .../secondaryindex/TestSIWithSecondryIndex.scala   |    6 +-
 .../carbondata/hive/MapredCarbonInputFormat.java   |    4 +-
 .../carbondata/hive/Hive2CarbonExpressionTest.java |   26 +-
 .../carbondata/presto/CarbondataPageSource.java    |   10 +-
 .../carbondata/presto/impl/CarbonTableReader.java  |   10 +-
 .../carbondata/presto/CarbondataPageSource.java    |    6 +-
 .../carbondata/presto/impl/CarbonTableReader.java  |    8 +-
 ...stCase.scala => BloomFilterIndexTestCase.scala} |   10 +-
 .../cluster/sdv/generated/LuceneTestCase.scala     |   67 +-
 integration/spark/pom.xml                          |    5 +
 .../apache/carbondata/datamap/DataMapManager.java  |    6 +-
 ...ndexDataMapProvider.java => IndexProvider.java} |   52 +-
 ...taMapRebuildRDD.scala => IndexRebuildRDD.scala} |   24 +-
 .../carbondata/events/AlterTableEvents.scala       |    6 +-
 .../apache/carbondata/events/DropTableEvents.scala |    3 +-
 .../carbondata/indexserver/DataMapJobs.scala       |   14 +-
 .../indexserver/DistributedCountRDD.scala          |   18 +-
 .../indexserver/DistributedPruneRDD.scala          |   20 +-
 .../indexserver/DistributedRDDUtils.scala          |   18 +-
 .../indexserver/DistributedShowCacheRDD.scala      |   20 +-
 .../carbondata/indexserver/IndexServer.scala       |   16 +-
 .../indexserver/InvalidateSegmentCacheRDD.scala    |    6 +-
 .../carbondata/indexserver/SegmentPruneRDD.scala   |   12 +-
 .../spark/rdd/CarbonDataRDDFactory.scala           |   25 +-
 .../spark/rdd/CarbonDeltaRowScanRDD.scala          |    4 +-
 .../carbondata/spark/rdd/CarbonScanRDD.scala       |    4 +-
 .../spark/rdd/CarbonTableCompactor.scala           |    6 +-
 .../vectorreader/VectorizedCarbonRecordReader.java |    4 +-
 .../apache/carbondata/store/SparkCarbonStore.scala |    4 +-
 .../apache/carbondata/view/MVCatalogInSpark.scala  |  212 +++
 .../org/apache/carbondata/view/MVEvents.scala      |   60 +
 .../scala/org/apache/carbondata/view/MVField.scala |   13 +-
 .../org/apache/carbondata/view/MVFunctions.scala   |   21 +-
 .../org/apache/carbondata/view/MVHelper.scala      |  340 +++++
 .../apache/carbondata/view/MVManagerInSpark.scala  |   70 +
 .../org/apache/carbondata/view/MVPlanWrapper.scala |   17 +-
 .../org/apache/carbondata/view/MVRefresher.scala   |  397 ++++++
 .../apache/carbondata/view/MVSchemaWrapper.scala   |   18 +-
 .../apache/carbondata/view/MVTimeGranularity.scala |   46 +
 .../spark/sql/CarbonDatasourceHadoopRelation.scala |    7 +-
 .../scala/org/apache/spark/sql/CarbonEnv.scala     |   39 +-
 .../org/apache/spark/sql/CarbonExtensions.scala    |   15 +-
 .../scala/org/apache/spark/sql/CarbonSession.scala |    2 +-
 .../execution/datasources/CarbonFileIndex.scala    |    4 +-
 .../datasources/SparkCarbonFileFormat.scala        |    4 +-
 .../spark/sql/catalyst/CarbonDDLSqlParser.scala    |    8 +
 .../sql/events/MergeBloomIndexEventListener.scala  |    4 +-
 .../sql/execution/command/cache/CacheUtil.scala    |    6 +-
 .../command/cache/CarbonDropCacheCommand.scala     |   19 +-
 .../command/cache/CarbonShowCacheCommand.scala     |   22 +-
 .../datamap/CarbonCreateDataMapCommand.scala       |  231 ---
 .../datamap/CarbonDataMapRebuildCommand.scala      |   93 --
 .../command/datamap/CarbonDropDataMapCommand.scala |  206 ---
 .../command/index/CarbonCreateIndexCommand.scala   |  197 +++
 .../command/index/CarbonDropIndexCommand.scala     |  178 +++
 .../command/index/CarbonRefreshIndexCommand.scala  |  101 ++
 .../CarbonShowIndexCommand.scala}                  |   10 +-
 .../command/index}/DropIndexCommand.scala          |   83 +-
 .../command/index/ShowIndexesCommand.scala         |  123 ++
 .../command/management/CarbonAddLoadCommand.scala  |   15 +-
 .../CarbonAlterTableCompactionCommand.scala        |   10 +
 .../management/CarbonCleanFilesCommand.scala       |   19 +-
 .../management/CarbonDeleteStageFilesCommand.scala |    4 +-
 .../management/CarbonInsertFromStageCommand.scala  |    2 +-
 .../management/CarbonInsertIntoWithDf.scala        |    2 +-
 .../command/management/CommonLoadUtils.scala       |   12 +-
 .../mutation/CarbonProjectForDeleteCommand.scala   |   12 +-
 .../mutation/CarbonProjectForUpdateCommand.scala   |   11 +-
 .../CarbonAlterTableAddHivePartitionCommand.scala  |    2 +-
 .../CarbonAlterTableDropHivePartitionCommand.scala |   11 +-
 .../schema/CarbonAlterTableRenameCommand.scala     |    6 +-
 .../table/CarbonCreateTableLikeCommand.scala       |    2 +-
 .../command/table/CarbonDropTableCommand.scala     |   45 +-
 .../command/view/CarbonCreateMVCommand.scala       |  815 +++++++++++
 .../command/view/CarbonDropMVCommand.scala         |  112 ++
 .../command/view/CarbonRefreshMVCommand.scala      |   68 +
 .../command/view/CarbonShowMVCommand.scala         |   88 ++
 .../spark/sql/execution/strategy/DDLHelper.scala   |   11 +-
 .../spark/sql/execution/strategy/DDLStrategy.scala |   33 +-
 .../spark/sql/hive/CarbonAnalysisRules.scala       |   30 +-
 .../spark/sql/hive/CarbonFileMetastore.scala       |    6 +-
 .../spark/sql/hive/CarbonHiveMetaStore.scala       |    2 +-
 .../org/apache/spark/sql/hive/CarbonRelation.scala |    2 +-
 .../execution/command/CarbonHiveCommands.scala     |    7 +-
 .../sql/listeners/DropCacheEventListeners.scala    |   17 +-
 .../apache/spark/sql/listeners/MVListeners.scala   |  504 ++++---
 .../spark/sql/listeners/PrePrimingListener.scala   |    4 +-
 .../sql/listeners/ShowCacheEventListeners.scala    |    2 +-
 .../org/apache/spark/sql/optimizer/MVMatcher.scala | 1468 ++++++++++++++++++++
 .../org/apache/spark/sql/optimizer/MVRewrite.scala | 1020 ++++++++++++++
 .../apache/spark/sql/optimizer/MVRewriteRule.scala |  254 ++++
 .../spark/sql/parser/CarbonSpark2SqlParser.scala   |  277 ++--
 .../apache/spark/sql/parser/MVQueryParser.scala    |  121 ++
 ...ma.java => BlockletIndexDetailsWithSchema.java} |   24 +-
 ...apLoader.java => BlockletIndexInputFormat.java} |   66 +-
 .../Jobs/CarbonBlockLoaderHelper.java              |    6 +-
 ...Job.scala => SparkBlockletIndexLoaderJob.scala} |   62 +-
 .../command/RegisterIndexTableCommand.scala        |   30 +-
 .../secondaryindex/command/SICreationCommand.scala |  375 ++---
 .../sql/secondaryindex/command/SILoadCommand.scala |   19 +-
 ...tCommand.scala => SIRebuildSegmentRunner.scala} |  109 +-
 .../command/ShowIndexesCommand.scala               |  164 ---
 .../AlterTableCompactionPostEventListener.scala    |    6 +-
 .../events/AlterTableDropColumnEventListener.scala |    2 +-
 .../AlterTableMergeIndexSIEventListener.scala      |    6 +-
 .../events/SILoadEventListener.scala               |    4 +-
 .../SILoadEventListenerForFailedSegments.scala     |    4 +-
 .../hive/CarbonInternalMetastore.scala             |    2 +-
 .../joins/BroadCastSIFilterPushJoin.scala          |   18 +-
 .../spark/sql/secondaryindex/load/Compactor.scala  |   11 +-
 .../optimizer/CarbonSecondaryIndexOptimizer.scala  |    2 +-
 .../rdd/CarbonSecondaryIndexRDD.scala              |    8 +-
 .../secondaryindex/rdd/SecondaryIndexCreator.scala |   30 +-
 .../util/CarbonInternalScalaUtil.scala             |   27 +-
 .../org/apache/spark/util/AlterTableUtil.scala     |    2 +-
 .../apache/spark/sql/CarbonToSparkAdapter.scala    |   13 +-
 .../apache/spark/sql/CarbonToSparkAdapter.scala    |   11 +-
 ...la => BloomCoarseGrainIndexFunctionSuite.scala} |  486 ++++---
 ...uite.scala => BloomCoarseGrainIndexSuite.scala} |  522 +++----
 ...l.scala => BloomCoarseGrainIndexTestUtil.scala} |   35 +-
 ...ite.scala => LuceneCoarseGrainIndexSuite.scala} |   25 +-
 ...Suite.scala => LuceneFineGrainIndexSuite.scala} |  360 +++--
 .../TestAdaptiveEncodingForPrimitiveTypes.scala    |    4 +-
 ...ryWithColumnMetCacheAndCacheLevelProperty.scala |   42 +-
 .../TestAlterTableSortColumnsProperty.scala        |   10 +-
 .../createTable/TestCreateTableLike.scala          |    7 +-
 .../TestNonTransactionalCarbonTable.scala          |    3 +-
 ...ataMap.scala => TestRenameTableWithIndex.scala} |   21 +-
 .../CarbonIndexFileMergeTestCase.scala             |   22 +-
 ...DataMapTestCase.scala => CGIndexTestCase.scala} |  137 +-
 ...DataMapTestCase.scala => FGIndexTestCase.scala} |  104 +-
 ...MapWriterSuite.scala => IndexWriterSuite.scala} |   70 +-
 .../testsuite/datamap/TestDataMapCommand.scala     |  156 ---
 .../spark/testsuite/datamap/TestIndexCommand.scala |  162 +++
 ...stDataMapStatus.scala => TestIndexStatus.scala} |  103 +-
 .../filterexpr/TestImplicitFilterExpression.scala  |    6 +-
 .../testsuite/iud/DeleteCarbonTableTestCase.scala  |    7 +-
 .../iud/TestInsertAndOtherCommandConcurrent.scala  |   50 +-
 .../testsuite/iud/UpdateCarbonTableTestCase.scala  |    7 +-
 .../sql/commands/TestCarbonDropCacheCommand.scala  |    5 +-
 .../sql/commands/TestCarbonShowCacheCommand.scala  |    8 +-
 .../scala/org/apache/carbondata/view/MVTest.scala  |  194 +++
 .../indexserver/DistributedRDDUtilsTest.scala      |   16 +-
 .../AlterTableColumnRenameTestCase.scala           |   13 +-
 .../datasource/SparkCarbonDataSourceTest.scala     |   60 +-
 ...TestCreateTableUsingSparkCarbonFileFormat.scala |    6 +-
 mv/core/pom.xml                                    |    5 +
 .../carbondata/mv/extension/MVAnalyzerRule.scala   |    4 +-
 .../mv/extension/MVDataMapProvider.scala           |   22 +-
 .../apache/carbondata/mv/extension/MVHelper.scala  |   33 +-
 .../apache/carbondata/mv/extension/MVParser.scala  |   10 +-
 .../apache/carbondata/mv/extension/MVUtil.scala    |    4 +-
 ...izedViewCommand.scala => CreateMVCommand.scala} |    8 +-
 ...alizedViewCommand.scala => DropMVCommand.scala} |    2 +-
 ...zedViewCommand.scala => RefreshMVCommand.scala} |    7 +-
 ...alizedViewCommand.scala => ShowMVCommand.scala} |    4 +-
 .../apache/carbondata/mv/rewrite/Navigator.scala   |    6 +-
 .../carbondata/mv/rewrite/QueryRewrite.scala       |   23 +-
 .../mv/rewrite/SummaryDatasetCatalog.scala         |    4 +-
 .../carbondata/mv/timeseries/TimeSeriesUtil.scala  |    7 +-
 .../carbondata/mv/rewrite/MVCoalesceTestCase.scala |    2 +-
 .../carbondata/mv/rewrite/MVCreateTestCase.scala   |   53 +-
 .../mv/rewrite/MVExceptionTestCase.scala           |   14 +-
 .../mv/rewrite/MVIncrementalLoadingTestcase.scala  |   84 +-
 .../mv/rewrite/TestAllOperationsOnMV.scala         |  108 +-
 .../mv/rewrite/TestPartitionWithMV.scala           |  150 +-
 ...TestMVTimeSeriesCarbonCreateIndexCommand.scala} |   10 +-
 .../timeseries/TestMVTimeSeriesLoadAndQuery.scala  |   26 +-
 mv/plan/pom.xml                                    |   80 +-
 .../carbondata/mv/plans/modular/Harmonizer.scala   |   48 +-
 .../carbondata/mv/plans/modular/ModularPlan.scala  |    6 +-
 .../mv/plans/modular/basicOperators.scala          |    6 +-
 .../mv/plans/util/BirdcageOptimizer.scala          |   57 -
 .../apache/carbondata/mv/plans/util/Printers.scala |   34 +-
 .../carbondata/mv/plans/util/SQLBuilder.scala      |   44 +-
 .../mv/plans/modular/ExpressionHelper.scala        |   50 +
 .../mv/plans/modular/ExpressionHelper.scala        |   50 +
 ...terException.java => IndexWriterException.java} |    4 +-
 ...riterListener.java => IndexWriterListener.java} |   72 +-
 .../loading/AbstractDataLoadProcessorStep.java     |    4 +-
 .../CarbonRowDataWriterProcessorStepImpl.java      |    4 +-
 .../loading/steps/DataWriterProcessorStepImpl.java |    6 +-
 .../merger/CarbonCompactionExecutor.java           |    4 +-
 .../processing/merger/CarbonDataMergerUtil.java    |   38 +-
 .../store/CarbonFactDataHandlerModel.java          |   20 +-
 .../carbondata/processing/store/TablePage.java     |    2 +-
 .../store/writer/AbstractFactDataWriter.java       |    6 +-
 .../processing/util/CarbonLoaderUtil.java          |    2 +-
 .../carbondata/sdk/file/CarbonReaderBuilder.java   |    8 +-
 .../apache/carbondata/store/LocalCarbonStore.java  |    4 +-
 .../carbondata/sdk/file/CarbonReaderTest.java      |   38 +-
 336 files changed, 12096 insertions(+), 5227 deletions(-)

diff --git a/common/src/main/java/org/apache/carbondata/common/exceptions/sql/MalformedDataMapCommandException.java b/common/src/main/java/org/apache/carbondata/common/exceptions/sql/MalformedIndexCommandException.java
similarity index 80%
copy from common/src/main/java/org/apache/carbondata/common/exceptions/sql/MalformedDataMapCommandException.java
copy to common/src/main/java/org/apache/carbondata/common/exceptions/sql/MalformedIndexCommandException.java
index 83cae7c..5d24cea 100644
--- a/common/src/main/java/org/apache/carbondata/common/exceptions/sql/MalformedDataMapCommandException.java
+++ b/common/src/main/java/org/apache/carbondata/common/exceptions/sql/MalformedIndexCommandException.java
@@ -21,21 +21,21 @@ import org.apache.carbondata.common.annotations.InterfaceAudience;
 import org.apache.carbondata.common.annotations.InterfaceStability;
 
 /**
- * This exception will be thrown when Datamap related SQL statement is invalid
+ * This exception will be thrown when index related SQL statement is invalid
  */
 @InterfaceAudience.User
 @InterfaceStability.Stable
-public class MalformedDataMapCommandException extends MalformedCarbonCommandException {
+public class MalformedIndexCommandException extends MalformedCarbonCommandException {
   /**
    * default serial version ID.
    */
   private static final long serialVersionUID = 1L;
 
-  public MalformedDataMapCommandException(String msg) {
+  public MalformedIndexCommandException(String msg) {
     super(msg);
   }
 
-  public MalformedDataMapCommandException(String msg, Throwable e) {
+  public MalformedIndexCommandException(String msg, Throwable e) {
     super(msg, e);
   }
 }
diff --git a/common/src/main/java/org/apache/carbondata/common/exceptions/sql/MalformedMaterializedViewException.java b/common/src/main/java/org/apache/carbondata/common/exceptions/sql/MalformedMVCommandException.java
similarity index 85%
rename from common/src/main/java/org/apache/carbondata/common/exceptions/sql/MalformedMaterializedViewException.java
rename to common/src/main/java/org/apache/carbondata/common/exceptions/sql/MalformedMVCommandException.java
index 7b84d12..1b61edb 100644
--- a/common/src/main/java/org/apache/carbondata/common/exceptions/sql/MalformedMaterializedViewException.java
+++ b/common/src/main/java/org/apache/carbondata/common/exceptions/sql/MalformedMVCommandException.java
@@ -25,18 +25,18 @@ import org.apache.carbondata.common.annotations.InterfaceStability;
  */
 @InterfaceAudience.User
 @InterfaceStability.Stable
-public class MalformedMaterializedViewException extends MalformedCarbonCommandException {
+public class MalformedMVCommandException extends MalformedCarbonCommandException {
 
   /**
    * default serial version ID.
    */
   private static final long serialVersionUID = 1L;
 
-  public MalformedMaterializedViewException(String msg) {
+  public MalformedMVCommandException(String msg) {
     super(msg);
   }
 
-  public MalformedMaterializedViewException(String msg, Throwable e) {
+  public MalformedMVCommandException(String msg, Throwable e) {
     super(msg, e);
   }
 }
diff --git a/common/src/main/java/org/apache/carbondata/common/exceptions/sql/MalformedDataMapCommandException.java b/common/src/main/java/org/apache/carbondata/common/exceptions/sql/NoSuchIndexException.java
similarity index 70%
copy from common/src/main/java/org/apache/carbondata/common/exceptions/sql/MalformedDataMapCommandException.java
copy to common/src/main/java/org/apache/carbondata/common/exceptions/sql/NoSuchIndexException.java
index 83cae7c..e8c2bff 100644
--- a/common/src/main/java/org/apache/carbondata/common/exceptions/sql/MalformedDataMapCommandException.java
+++ b/common/src/main/java/org/apache/carbondata/common/exceptions/sql/NoSuchIndexException.java
@@ -21,21 +21,18 @@ import org.apache.carbondata.common.annotations.InterfaceAudience;
 import org.apache.carbondata.common.annotations.InterfaceStability;
 
 /**
- * This exception will be thrown when Datamap related SQL statement is invalid
+ * This exception will be thrown if index is not found when executing index
+ * related SQL statement
  */
 @InterfaceAudience.User
 @InterfaceStability.Stable
-public class MalformedDataMapCommandException extends MalformedCarbonCommandException {
-  /**
-   * default serial version ID.
-   */
-  private static final long serialVersionUID = 1L;
+public class NoSuchIndexException extends MalformedCarbonCommandException {
 
-  public MalformedDataMapCommandException(String msg) {
-    super(msg);
+  public NoSuchIndexException(String indexName, String tableName) {
+    super("Index with name " + indexName + " does not exist under table " + tableName);
   }
 
-  public MalformedDataMapCommandException(String msg, Throwable e) {
-    super(msg, e);
+  public NoSuchIndexException(String indexName) {
+    super("Index with name " + indexName + " does not exist");
   }
 }
diff --git a/common/src/main/java/org/apache/carbondata/common/exceptions/sql/MalformedDataMapCommandException.java b/common/src/main/java/org/apache/carbondata/common/exceptions/sql/NoSuchMVException.java
similarity index 77%
rename from common/src/main/java/org/apache/carbondata/common/exceptions/sql/MalformedDataMapCommandException.java
rename to common/src/main/java/org/apache/carbondata/common/exceptions/sql/NoSuchMVException.java
index 83cae7c..6f14468 100644
--- a/common/src/main/java/org/apache/carbondata/common/exceptions/sql/MalformedDataMapCommandException.java
+++ b/common/src/main/java/org/apache/carbondata/common/exceptions/sql/NoSuchMVException.java
@@ -21,21 +21,20 @@ import org.apache.carbondata.common.annotations.InterfaceAudience;
 import org.apache.carbondata.common.annotations.InterfaceStability;
 
 /**
- * This exception will be thrown when Datamap related SQL statement is invalid
+ * This exception will be thrown if mv is not found when executing mv
+ * related SQL statement
  */
 @InterfaceAudience.User
 @InterfaceStability.Stable
-public class MalformedDataMapCommandException extends MalformedCarbonCommandException {
+public class NoSuchMVException extends MalformedCarbonCommandException {
+
   /**
    * default serial version ID.
    */
   private static final long serialVersionUID = 1L;
 
-  public MalformedDataMapCommandException(String msg) {
-    super(msg);
+  public NoSuchMVException(String databaseName, String mvName) {
+    super("Materialized view with name " + databaseName + "." + mvName + " does not exist");
   }
 
-  public MalformedDataMapCommandException(String msg, Throwable e) {
-    super(msg, e);
-  }
 }
diff --git a/core/src/main/java/org/apache/carbondata/core/cache/CacheProvider.java b/core/src/main/java/org/apache/carbondata/core/cache/CacheProvider.java
index b207905..257795c 100644
--- a/core/src/main/java/org/apache/carbondata/core/cache/CacheProvider.java
+++ b/core/src/main/java/org/apache/carbondata/core/cache/CacheProvider.java
@@ -23,7 +23,7 @@ import java.util.Map;
 
 import org.apache.carbondata.common.logging.LogServiceFactory;
 import org.apache.carbondata.core.constants.CarbonCommonConstants;
-import org.apache.carbondata.core.indexstore.BlockletDataMapIndexStore;
+import org.apache.carbondata.core.indexstore.BlockletIndexStore;
 import org.apache.carbondata.core.util.CarbonProperties;
 
 import org.apache.log4j.Logger;
@@ -131,7 +131,7 @@ public class CacheProvider {
   private void createBlockletDataMapCache(CacheType cacheType) {
     Cache cacheObject = null;
     if (cacheType.equals(cacheType.DRIVER_BLOCKLET_DATAMAP)) {
-      cacheObject = new BlockletDataMapIndexStore(carbonLRUCache);
+      cacheObject = new BlockletIndexStore(carbonLRUCache);
     }
     cacheTypeToCacheMap.put(cacheType, cacheObject);
   }
diff --git a/core/src/main/java/org/apache/carbondata/core/constants/CarbonCommonConstants.java b/core/src/main/java/org/apache/carbondata/core/constants/CarbonCommonConstants.java
index fbfacec..41ac51a 100644
--- a/core/src/main/java/org/apache/carbondata/core/constants/CarbonCommonConstants.java
+++ b/core/src/main/java/org/apache/carbondata/core/constants/CarbonCommonConstants.java
@@ -460,8 +460,8 @@ public final class CarbonCommonConstants {
   public static final String LOCAL_DICTIONARY_EXCLUDE = "local_dictionary_exclude";
 
   /**
-   * DMPROPERTY for Index DataMap, like lucene, bloomfilter DataMap,
-   * to indicate a list of column name to be indexed
+   * Internal property to store for index column names
+   * TODO: remove it after index metadata refactory
    */
   public static final String INDEX_COLUMNS = "INDEX_COLUMNS";
 
@@ -2265,6 +2265,11 @@ public final class CarbonCommonConstants {
    */
   public static final String PARENT_TABLES = "parent_tables";
 
+  /**
+   * This property will be used to store table name's related with mv
+   */
+  public static final String MV_RELATED_TABLES = "mv_related_tables";
+
   public static final String LOAD_SYNC_TIME = "load_sync_time";
 
   public static final String CARBON_INDEX_SERVER_JOBNAME_LENGTH =
@@ -2432,4 +2437,10 @@ public final class CarbonCommonConstants {
    * Make this false, to load index for the matched segments from filter expression
    */
   public static final String CARBON_LOAD_ALL_SEGMENT_INDEXES_TO_CACHE_DEFAULT = "true";
+
+  /**
+   * Materialized view thread context properties
+   */
+  @CarbonProperty
+  public static final String DISABLE_SQL_REWRITE = "disable_sql_rewrite";
 }
diff --git a/core/src/main/java/org/apache/carbondata/core/datamap/AbstractDataMapJob.java b/core/src/main/java/org/apache/carbondata/core/datamap/AbstractIndexJob.java
similarity index 80%
rename from core/src/main/java/org/apache/carbondata/core/datamap/AbstractDataMapJob.java
rename to core/src/main/java/org/apache/carbondata/core/datamap/AbstractIndexJob.java
index 6d30a52..ff55976 100644
--- a/core/src/main/java/org/apache/carbondata/core/datamap/AbstractDataMapJob.java
+++ b/core/src/main/java/org/apache/carbondata/core/datamap/AbstractIndexJob.java
@@ -19,7 +19,7 @@ package org.apache.carbondata.core.datamap;
 
 import java.util.List;
 
-import org.apache.carbondata.core.indexstore.BlockletDataMapIndexWrapper;
+import org.apache.carbondata.core.indexstore.BlockletIndexWrapper;
 import org.apache.carbondata.core.indexstore.ExtendedBlocklet;
 import org.apache.carbondata.core.metadata.schema.table.CarbonTable;
 
@@ -28,15 +28,15 @@ import org.apache.hadoop.mapreduce.lib.input.FileInputFormat;
 /**
  * abstract class for data map job
  */
-public abstract class AbstractDataMapJob implements DataMapJob {
+public abstract class AbstractIndexJob implements IndexJob {
 
   @Override
   public void execute(CarbonTable carbonTable,
-      FileInputFormat<Void, BlockletDataMapIndexWrapper> format) {
+      FileInputFormat<Void, BlockletIndexWrapper> format) {
   }
 
   @Override
-  public List<ExtendedBlocklet> execute(DistributableDataMapFormat dataMapFormat) {
+  public List<ExtendedBlocklet> execute(IndexInputFormat dataMapFormat) {
     return null;
   }
 }
diff --git a/core/src/main/java/org/apache/carbondata/core/datamap/DataMapProvider.java b/core/src/main/java/org/apache/carbondata/core/datamap/DataMapProvider.java
index 6df7e56..491fdbd 100644
--- a/core/src/main/java/org/apache/carbondata/core/datamap/DataMapProvider.java
+++ b/core/src/main/java/org/apache/carbondata/core/datamap/DataMapProvider.java
@@ -27,13 +27,13 @@ import java.util.Map;
 import java.util.stream.Collectors;
 
 import org.apache.carbondata.common.annotations.InterfaceAudience;
-import org.apache.carbondata.common.exceptions.sql.MalformedDataMapCommandException;
+import org.apache.carbondata.common.exceptions.sql.MalformedIndexCommandException;
 import org.apache.carbondata.common.exceptions.sql.NoSuchDataMapException;
 import org.apache.carbondata.common.logging.LogServiceFactory;
 import org.apache.carbondata.core.constants.CarbonCommonConstants;
-import org.apache.carbondata.core.datamap.dev.DataMapFactory;
-import org.apache.carbondata.core.datamap.status.DataMapSegmentStatusUtil;
+import org.apache.carbondata.core.datamap.dev.IndexFactory;
 import org.apache.carbondata.core.datamap.status.DataMapStatusManager;
+import org.apache.carbondata.core.datamap.status.MVSegmentStatusUtil;
 import org.apache.carbondata.core.locks.ICarbonLock;
 import org.apache.carbondata.core.metadata.AbsoluteTableIdentifier;
 import org.apache.carbondata.core.metadata.schema.table.CarbonTable;
@@ -48,20 +48,20 @@ import com.google.gson.Gson;
 import org.apache.log4j.Logger;
 
 /**
- * DataMap is a accelerator for certain type of query. Developer can add new DataMap
+ * Index is a accelerator for certain type of query. Developer can add new Index
  * implementation to improve query performance.
  *
- * Currently two types of DataMap are supported
+ * Currently two types of Index are supported
  * <ol>
- *   <li> MVDataMap: materialized view type of DataMap to accelerate olap style query,
+ *   <li> MVDataMap: materialized view type of Index to accelerate olap style query,
  * like SPJG query (select, predicate, join, groupby) </li>
- *   <li> DataMap: index type of DataMap to accelerate filter query </li>
+ *   <li> Index: index type of Index to accelerate filter query </li>
  * </ol>
  *
  * <p>
  * In following command <br>
  * {@code CREATE DATAMAP dm ON TABLE main USING 'provider'}, <br>
- * the <b>provider</b> string can be a short name or class name of the DataMap implementation.
+ * the <b>provider</b> string can be a short name or class name of the Index implementation.
  *
  * <br>Currently CarbonData supports following provider:
  * <ol>
@@ -97,7 +97,7 @@ public abstract class DataMapProvider {
    * This is called when user creates datamap, for example "CREATE DATAMAP dm ON TABLE mainTable"
    * Implementation should initialize metadata for datamap, like creating table
    */
-  public abstract void initMeta(String ctasSqlStatement) throws MalformedDataMapCommandException,
+  public abstract void initMeta(String ctasSqlStatement) throws MalformedIndexCommandException,
       IOException;
 
   /**
@@ -197,7 +197,7 @@ public abstract class DataMapProvider {
               .collect(Collectors.toList());
           for (RelationIdentifier relationIdentifier : relationIdentifiers) {
             List<String> mainTableSegmentList =
-                DataMapUtil.getMainTableValidSegmentList(relationIdentifier);
+                IndexUtil.getMainTableValidSegmentList(relationIdentifier);
             if (mainTableSegmentList.isEmpty()) {
               return false;
             }
@@ -272,7 +272,7 @@ public abstract class DataMapProvider {
       // If segment Map is empty, load all valid segments from main tables to dataMap
       for (RelationIdentifier relationIdentifier : relationIdentifiers) {
         List<String> mainTableSegmentList =
-            DataMapUtil.getMainTableValidSegmentList(relationIdentifier);
+            IndexUtil.getMainTableValidSegmentList(relationIdentifier);
         // If mainTableSegmentList is empty, no need to trigger load command
         // TODO: handle in case of multiple tables load to datamap table
         if (mainTableSegmentList.isEmpty()) {
@@ -287,13 +287,13 @@ public abstract class DataMapProvider {
         List<String> dataMapTableSegmentList = new ArrayList<>();
         // Get all segments for parent relationIdentifier
         List<String> mainTableSegmentList =
-            DataMapUtil.getMainTableValidSegmentList(relationIdentifier);
+            IndexUtil.getMainTableValidSegmentList(relationIdentifier);
         boolean ifTableStatusUpdateRequired = false;
         for (LoadMetadataDetails loadMetaDetail : listOfLoadFolderDetails) {
           if (loadMetaDetail.getSegmentStatus() == SegmentStatus.SUCCESS
               || loadMetaDetail.getSegmentStatus() == SegmentStatus.INSERT_IN_PROGRESS) {
             Map<String, List<String>> segmentMaps =
-                DataMapSegmentStatusUtil.getSegmentMap(loadMetaDetail.getExtraInfo());
+                MVSegmentStatusUtil.getSegmentMap(loadMetaDetail.getExtraInfo());
             String mainTableMetaDataPath =
                 CarbonTablePath.getMetadataPath(relationIdentifier.getTablePath());
             LoadMetadataDetails[] parentTableLoadMetaDataDetails =
@@ -346,7 +346,7 @@ public abstract class DataMapProvider {
               if (loadMetaDetail.getSegmentStatus() == SegmentStatus.SUCCESS
                   || loadMetaDetail.getSegmentStatus() == SegmentStatus.INSERT_IN_PROGRESS) {
                 Map<String, List<String>> segmentMaps =
-                    DataMapSegmentStatusUtil.getSegmentMap(loadMetaDetail.getExtraInfo());
+                    MVSegmentStatusUtil.getSegmentMap(loadMetaDetail.getExtraInfo());
                 List<String> segmentIds = segmentMaps.get(
                     relationIdentifier.getDatabaseName() + CarbonCommonConstants.POINT
                         + relationIdentifier.getTableName());
@@ -405,11 +405,11 @@ public abstract class DataMapProvider {
    * Provide the datamap catalog instance or null if this datamap not required to rewrite
    * the query.
    */
-  public DataMapCatalog createDataMapCatalog() {
+  public MVCatalog createDataMapCatalog() {
     return null;
   }
 
-  public abstract DataMapFactory getDataMapFactory();
+  public abstract IndexFactory getIndexFactory();
 
   public abstract boolean supportRebuild();
 
diff --git a/core/src/main/java/org/apache/carbondata/core/datamap/DataMapStoreManager.java b/core/src/main/java/org/apache/carbondata/core/datamap/DataMapStoreManager.java
index 2ddcf2b..65ddc5c 100644
--- a/core/src/main/java/org/apache/carbondata/core/datamap/DataMapStoreManager.java
+++ b/core/src/main/java/org/apache/carbondata/core/datamap/DataMapStoreManager.java
@@ -23,22 +23,22 @@ import java.util.HashMap;
 import java.util.Iterator;
 import java.util.List;
 import java.util.Map;
+import java.util.Optional;
 import java.util.concurrent.ConcurrentHashMap;
 
 import org.apache.carbondata.common.annotations.InterfaceAudience;
 import org.apache.carbondata.common.exceptions.MetadataProcessException;
-import org.apache.carbondata.common.exceptions.sql.MalformedDataMapCommandException;
+import org.apache.carbondata.common.exceptions.sql.MalformedIndexCommandException;
 import org.apache.carbondata.common.exceptions.sql.NoSuchDataMapException;
 import org.apache.carbondata.common.logging.LogServiceFactory;
 import org.apache.carbondata.core.constants.CarbonCommonConstants;
-import org.apache.carbondata.core.datamap.dev.DataMapFactory;
+import org.apache.carbondata.core.datamap.dev.IndexFactory;
 import org.apache.carbondata.core.datastore.block.SegmentPropertiesAndSchemaHolder;
 import org.apache.carbondata.core.indexstore.BlockletDetailsFetcher;
 import org.apache.carbondata.core.indexstore.SegmentPropertiesFetcher;
-import org.apache.carbondata.core.indexstore.blockletindex.BlockletDataMapFactory;
+import org.apache.carbondata.core.indexstore.blockletindex.BlockletIndexFactory;
 import org.apache.carbondata.core.metadata.AbsoluteTableIdentifier;
 import org.apache.carbondata.core.metadata.CarbonMetadata;
-import org.apache.carbondata.core.metadata.schema.datamap.DataMapClassProvider;
 import org.apache.carbondata.core.metadata.schema.table.CarbonTable;
 import org.apache.carbondata.core.metadata.schema.table.DataMapSchema;
 import org.apache.carbondata.core.metadata.schema.table.DataMapSchemaFactory;
@@ -52,6 +52,8 @@ import org.apache.carbondata.core.util.CarbonProperties;
 import org.apache.carbondata.core.util.CarbonSessionInfo;
 import org.apache.carbondata.core.util.ThreadLocalSessionInfo;
 
+import static org.apache.carbondata.core.metadata.schema.datamap.DataMapClassProvider.MV;
+
 import org.apache.commons.collections.CollectionUtils;
 import org.apache.commons.lang.StringUtils;
 import org.apache.hadoop.fs.Path;
@@ -65,14 +67,14 @@ public final class DataMapStoreManager {
 
   private static DataMapStoreManager instance = new DataMapStoreManager();
 
-  public Map<String, List<TableDataMap>> getAllDataMaps() {
-    return allDataMaps;
+  public Map<String, List<TableIndex>> getTableIndexForAllTables() {
+    return allIndexes;
   }
 
   /**
-   * Contains the list of datamaps for each table.
+   * Contains the list of indexes for each table.
    */
-  private Map<String, List<TableDataMap>> allDataMaps = new ConcurrentHashMap<>();
+  private Map<String, List<TableIndex>> allIndexes = new ConcurrentHashMap<>();
 
   /**
    * Contains the table name to the tablepath mapping.
@@ -80,9 +82,9 @@ public final class DataMapStoreManager {
   private Map<String, String> tablePathMap = new ConcurrentHashMap<>();
 
   /**
-   * Contains the datamap catalog for each datamap provider.
+   * Contains the mv catalog for each mv provider.
    */
-  private Map<String, DataMapCatalog> dataMapCatalogs = null;
+  private Map<String, MVCatalog> mvCatalogMap = null;
 
   private Map<String, TableSegmentRefresher> segmentRefreshMap = new ConcurrentHashMap<>();
 
@@ -101,12 +103,12 @@ public final class DataMapStoreManager {
   /**
    * It only gives the visible datamaps
    */
-  List<TableDataMap> getAllVisibleDataMap(CarbonTable carbonTable) throws IOException {
+  List<TableIndex> getAllVisibleIndexes(CarbonTable carbonTable) throws IOException {
     CarbonSessionInfo sessionInfo = ThreadLocalSessionInfo.getCarbonSessionInfo();
-    List<TableDataMap> allDataMaps = getAllDataMap(carbonTable);
-    Iterator<TableDataMap> dataMapIterator = allDataMaps.iterator();
+    List<TableIndex> allDataMaps = getAllIndexes(carbonTable);
+    Iterator<TableIndex> dataMapIterator = allDataMaps.iterator();
     while (dataMapIterator.hasNext()) {
-      TableDataMap dataMap = dataMapIterator.next();
+      TableIndex dataMap = dataMapIterator.next();
       String dbName = carbonTable.getDatabaseName();
       String tableName = carbonTable.getTableName();
       String dmName = dataMap.getDataMapSchema().getDataMapName();
@@ -129,23 +131,23 @@ public final class DataMapStoreManager {
   }
 
   /**
-   * It gives all datamaps except the default datamap.
+   * It gives all indexes except the default index.
    *
    * @return
    */
-  public List<TableDataMap> getAllDataMap(CarbonTable carbonTable) throws IOException {
+  public List<TableIndex> getAllIndexes(CarbonTable carbonTable) throws IOException {
     List<DataMapSchema> dataMapSchemas = getDataMapSchemasOfTable(carbonTable);
-    List<TableDataMap> dataMaps = new ArrayList<>();
+    List<TableIndex> indexes = new ArrayList<>();
     if (dataMapSchemas != null) {
       for (DataMapSchema dataMapSchema : dataMapSchemas) {
         RelationIdentifier identifier = dataMapSchema.getParentTables().get(0);
-        if (dataMapSchema.isIndexDataMap() && identifier.getTableId()
+        if (dataMapSchema.isIndex() && identifier.getTableId()
             .equals(carbonTable.getTableId())) {
-          dataMaps.add(getDataMap(carbonTable, dataMapSchema));
+          indexes.add(getIndex(carbonTable, dataMapSchema));
         }
       }
     }
-    return dataMaps;
+    return indexes;
   }
 
   /**
@@ -163,6 +165,17 @@ public final class DataMapStoreManager {
     return provider.retrieveAllSchemas();
   }
 
+  /**
+   * Return first match of the specified index name in table
+   *
+   */
+  public Optional<DataMapSchema> getIndexInTable(CarbonTable carbonTable, String indexName)
+      throws IOException {
+    return provider.retrieveSchemas(carbonTable).stream()
+        .filter(schema -> schema.getDataMapName().equalsIgnoreCase(indexName))
+        .findFirst();
+  }
+
   public DataMapSchema getDataMapSchema(String dataMapName)
       throws NoSuchDataMapException, IOException {
     return provider.retrieveSchema(dataMapName);
@@ -199,7 +212,7 @@ public final class DataMapStoreManager {
       String tableId = relationIdentifier.getTableId();
       String providerName = dataMapSchema.getProviderName();
       // if the mv datamap,not be modified the relationIdentifier
-      if (!providerName.equalsIgnoreCase(DataMapClassProvider.MV.toString())) {
+      if (!providerName.equalsIgnoreCase(MV.toString())) {
         RelationIdentifier newRelationIdentifier = new RelationIdentifier(dataBaseName,
             newTableName, tableId);
         dataMapSchema.setRelationIdentifier(newRelationIdentifier);
@@ -228,29 +241,29 @@ public final class DataMapStoreManager {
    * @param dataMapProvider
    * @param dataMapSchema
    */
-  public synchronized void registerDataMapCatalog(DataMapProvider dataMapProvider,
+  public synchronized void registerMVCatalog(DataMapProvider dataMapProvider,
       DataMapSchema dataMapSchema, boolean clearCatalogs) throws IOException {
     // this check is added to check if when registering the datamapCatalog, if the catalog map has
     // datasets with old session, then need to clear and reload the map, else error can be thrown
     // if the databases are different in both the sessions
     if (clearCatalogs) {
-      dataMapCatalogs = null;
+      mvCatalogMap = null;
     }
-    initializeDataMapCatalogs(dataMapProvider);
+    initializeMVCatalogs(dataMapProvider);
     String name = dataMapSchema.getProviderName().toLowerCase();
-    DataMapCatalog dataMapCatalog = dataMapCatalogs.get(name);
-    if (dataMapCatalog == null) {
-      dataMapCatalog = dataMapProvider.createDataMapCatalog();
+    MVCatalog mvCatalog = mvCatalogMap.get(name);
+    if (mvCatalog == null) {
+      mvCatalog = dataMapProvider.createDataMapCatalog();
       // If MVDataMapProvider, then createDataMapCatalog will return summaryDatasetCatalog
       // instance, which needs to be added to dataMapCatalogs.
       // For other datamaps, createDataMapCatalog will return null, so no need to register
-      if (dataMapCatalog != null) {
-        dataMapCatalogs.put(name, dataMapCatalog);
-        dataMapCatalog.registerSchema(dataMapSchema);
+      if (mvCatalog != null) {
+        mvCatalogMap.put(name, mvCatalog);
+        mvCatalog.registerSchema(dataMapSchema);
       }
     } else {
-      if (!dataMapCatalog.isMVExists(dataMapSchema.getDataMapName())) {
-        dataMapCatalog.registerSchema(dataMapSchema);
+      if (!mvCatalog.isMVExists(dataMapSchema.getDataMapName())) {
+        mvCatalog.registerSchema(dataMapSchema);
       }
     }
   }
@@ -260,13 +273,13 @@ public final class DataMapStoreManager {
    * @param dataMapSchema
    */
   public synchronized void unRegisterDataMapCatalog(DataMapSchema dataMapSchema) {
-    if (dataMapCatalogs == null) {
+    if (mvCatalogMap == null) {
       return;
     }
     String name = dataMapSchema.getProviderName().toLowerCase();
-    DataMapCatalog dataMapCatalog = dataMapCatalogs.get(name);
-    if (dataMapCatalog != null) {
-      dataMapCatalog.unregisterSchema(dataMapSchema.getDataMapName());
+    MVCatalog MVCatalog = mvCatalogMap.get(name);
+    if (MVCatalog != null) {
+      MVCatalog.unregisterSchema(dataMapSchema.getDataMapName());
     }
   }
 
@@ -275,42 +288,42 @@ public final class DataMapStoreManager {
    * @param providerName
    * @return
    */
-  public synchronized DataMapCatalog getDataMapCatalog(
+  public synchronized MVCatalog getMVCatalog(
       DataMapProvider dataMapProvider,
       String providerName,
       boolean clearCatalogs) throws IOException {
     // This method removes the datamapCatalog for the corresponding provider if the session gets
     // refreshed or updated
     if (clearCatalogs) {
-      dataMapCatalogs = null;
+      mvCatalogMap = null;
     }
-    initializeDataMapCatalogs(dataMapProvider);
-    return dataMapCatalogs.get(providerName.toLowerCase());
+    initializeMVCatalogs(dataMapProvider);
+    return mvCatalogMap.get(providerName.toLowerCase());
   }
 
   /**
    * Initialize by reading all datamaps from store and re register it
    * @param dataMapProvider
    */
-  private synchronized void initializeDataMapCatalogs(DataMapProvider dataMapProvider)
+  private synchronized void initializeMVCatalogs(DataMapProvider dataMapProvider)
       throws IOException {
-    if (dataMapCatalogs == null) {
-      dataMapCatalogs = new ConcurrentHashMap<>();
+    if (mvCatalogMap == null) {
+      mvCatalogMap = new ConcurrentHashMap<>();
       List<DataMapSchema> dataMapSchemas = getAllDataMapSchemas();
       for (DataMapSchema schema : dataMapSchemas) {
         if (schema.getProviderName()
             .equalsIgnoreCase(dataMapProvider.getDataMapSchema().getProviderName())) {
-          DataMapCatalog dataMapCatalog =
-              dataMapCatalogs.get(schema.getProviderName().toLowerCase());
-          if (dataMapCatalog == null) {
-            dataMapCatalog = dataMapProvider.createDataMapCatalog();
-            if (null == dataMapCatalog) {
+          MVCatalog MVCatalog =
+              mvCatalogMap.get(schema.getProviderName().toLowerCase());
+          if (MVCatalog == null) {
+            MVCatalog = dataMapProvider.createDataMapCatalog();
+            if (null == MVCatalog) {
               throw new RuntimeException("Internal Error.");
             }
-            dataMapCatalogs.put(schema.getProviderName().toLowerCase(), dataMapCatalog);
+            mvCatalogMap.put(schema.getProviderName().toLowerCase(), MVCatalog);
           }
           try {
-            dataMapCatalog.registerSchema(schema);
+            MVCatalog.registerSchema(schema);
           } catch (Exception e) {
             // Ignore the schema
             LOGGER.error("Error while registering schema", e);
@@ -321,51 +334,51 @@ public final class DataMapStoreManager {
   }
 
   /**
-   * It gives the default datamap of the table. Default datamap of any table is BlockletDataMap
+   * It gives the default datamap of the table. Default datamap of any table is BlockletIndex
    *
    * @param table
    * @return
    */
-  public TableDataMap getDefaultDataMap(CarbonTable table) {
-    return getDataMap(table, BlockletDataMapFactory.DATA_MAP_SCHEMA);
+  public TableIndex getDefaultIndex(CarbonTable table) {
+    return getIndex(table, BlockletIndexFactory.DATA_MAP_SCHEMA);
   }
 
   /**
    * Get the datamap for reading data.
    */
-  public TableDataMap getDataMap(CarbonTable table, DataMapSchema dataMapSchema) {
+  public TableIndex getIndex(CarbonTable table, DataMapSchema dataMapSchema) {
     String tableId =
         table.getAbsoluteTableIdentifier().getCarbonTableIdentifier().getTableId();
-    List<TableDataMap> tableIndices = allDataMaps.get(table.getTableId());
+    List<TableIndex> tableIndices = allIndexes.get(table.getTableId());
     if (tableIndices == null && !table.isTransactionalTable()) {
       String keyUsingTablePath = getKeyUsingTablePath(table.getTablePath());
       if (keyUsingTablePath != null) {
         tableId = keyUsingTablePath;
-        tableIndices = allDataMaps.get(tableId);
+        tableIndices = allIndexes.get(tableId);
       }
     }
     // in case of fileformat or sdk, when table is dropped or schema is changed the datamaps are
     // not cleared, they need to be cleared by using API, so compare the columns, if not same, clear
     // the datamaps on that table
-    if (allDataMaps.size() > 0 && !CollectionUtils.isEmpty(allDataMaps.get(tableId))
-        && !allDataMaps.get(tableId).get(0).getTable().getTableInfo().getFactTable()
+    if (allIndexes.size() > 0 && !CollectionUtils.isEmpty(allIndexes.get(tableId))
+        && !allIndexes.get(tableId).get(0).getTable().getTableInfo().getFactTable()
         .getListOfColumns().equals(table.getTableInfo().getFactTable().getListOfColumns())) {
-      clearDataMaps(tableId);
+      clearIndex(tableId);
       tableIndices = null;
     }
-    TableDataMap dataMap = null;
+    TableIndex dataMap = null;
     if (tableIndices != null) {
-      dataMap = getTableDataMap(dataMapSchema.getDataMapName(), tableIndices);
+      dataMap = getTableIndex(dataMapSchema.getDataMapName(), tableIndices);
     }
     if (dataMap == null) {
       synchronized (tableId.intern()) {
-        tableIndices = allDataMaps.get(tableId);
+        tableIndices = allIndexes.get(tableId);
         if (tableIndices != null) {
-          dataMap = getTableDataMap(dataMapSchema.getDataMapName(), tableIndices);
+          dataMap = getTableIndex(dataMapSchema.getDataMapName(), tableIndices);
         }
         if (dataMap == null) {
           try {
-            dataMap = createAndRegisterDataMap(table, dataMapSchema);
+            dataMap = createAndRegisterIndex(table, dataMapSchema);
           } catch (Exception e) {
             throw new RuntimeException(e);
           }
@@ -381,7 +394,7 @@ public final class DataMapStoreManager {
     // once a table is dropped and recreated with the same name again then because the dataMap
     // contains the stale carbon table schema mismatch exception is thrown. To avoid such scenarios
     // it is always better to update the carbon table object retrieved
-    dataMap.getDataMapFactory().setCarbonTable(table);
+    dataMap.getIndexFactory().setCarbonTable(table);
     return dataMap;
   }
 
@@ -401,19 +414,19 @@ public final class DataMapStoreManager {
    * Return a new datamap instance and registered in the store manager.
    * The datamap is created using datamap name, datamap factory class and table identifier.
    */
-  public DataMapFactory getDataMapFactoryClass(CarbonTable table, DataMapSchema dataMapSchema)
-      throws MalformedDataMapCommandException {
+  public IndexFactory getDataMapFactoryClass(CarbonTable table, DataMapSchema dataMapSchema)
+      throws MalformedIndexCommandException {
     try {
-      // try to create datamap by reflection to test whether it is a valid DataMapFactory class
-      return (DataMapFactory)
+      // try to create datamap by reflection to test whether it is a valid IndexFactory class
+      return (IndexFactory)
           Class.forName(dataMapSchema.getProviderName()).getConstructors()[0]
               .newInstance(table, dataMapSchema);
     } catch (ClassNotFoundException e) {
       // try to create DataMapClassProvider instance by taking providerName as short name
-      return DataMapRegistry.getDataMapFactoryByShortName(table, dataMapSchema);
+      return IndexRegistry.getDataMapFactoryByShortName(table, dataMapSchema);
     } catch (Throwable e) {
       throw new MetadataProcessException(
-          "failed to get DataMap factory for'" + dataMapSchema.getProviderName() + "'", e);
+          "failed to get Index factory for'" + dataMapSchema.getProviderName() + "'", e);
     }
   }
 
@@ -421,24 +434,23 @@ public final class DataMapStoreManager {
    * registered in the store manager.
    * The datamap is created using datamap name, datamap factory class and table identifier.
    */
-  // TODO: make it private
-  public TableDataMap createAndRegisterDataMap(CarbonTable table,
-      DataMapSchema dataMapSchema) throws MalformedDataMapCommandException {
-    DataMapFactory dataMapFactory  = getDataMapFactoryClass(table, dataMapSchema);
-    return registerDataMap(table, dataMapSchema, dataMapFactory);
+  private TableIndex createAndRegisterIndex(CarbonTable table,
+      DataMapSchema dataMapSchema) throws MalformedIndexCommandException {
+    IndexFactory indexFactory = getDataMapFactoryClass(table, dataMapSchema);
+    return registerIndex(table, dataMapSchema, indexFactory);
   }
 
-  public TableDataMap registerDataMap(CarbonTable table,
-      DataMapSchema dataMapSchema,  DataMapFactory dataMapFactory) {
+  public TableIndex registerIndex(CarbonTable table,
+      DataMapSchema dataMapSchema,  IndexFactory indexFactory) {
     String tableUniqueName = table.getCarbonTableIdentifier().getTableUniqueName();
     // Just update the segmentRefreshMap with the table if not added.
     getTableSegmentRefresher(table);
-    List<TableDataMap> tableIndices = allDataMaps.get(table.getTableId());
+    List<TableIndex> tableIndices = allIndexes.get(table.getTableId());
     if (tableIndices == null) {
       String keyUsingTablePath = getKeyUsingTablePath(table.getTablePath());
       if (keyUsingTablePath != null) {
         tableUniqueName = keyUsingTablePath;
-        tableIndices = allDataMaps.get(table.getTableId());
+        tableIndices = allIndexes.get(table.getTableId());
       }
     }
     if (tableIndices == null) {
@@ -447,26 +459,26 @@ public final class DataMapStoreManager {
 
     BlockletDetailsFetcher blockletDetailsFetcher;
     SegmentPropertiesFetcher segmentPropertiesFetcher = null;
-    if (dataMapFactory instanceof BlockletDetailsFetcher) {
-      blockletDetailsFetcher = (BlockletDetailsFetcher) dataMapFactory;
+    if (indexFactory instanceof BlockletDetailsFetcher) {
+      blockletDetailsFetcher = (BlockletDetailsFetcher) indexFactory;
     } else {
       blockletDetailsFetcher = getBlockletDetailsFetcher(table);
     }
     segmentPropertiesFetcher = (SegmentPropertiesFetcher) blockletDetailsFetcher;
-    TableDataMap dataMap = new TableDataMap(table,
-        dataMapSchema, dataMapFactory, blockletDetailsFetcher, segmentPropertiesFetcher);
+    TableIndex dataMap = new TableIndex(table,
+        dataMapSchema, indexFactory, blockletDetailsFetcher, segmentPropertiesFetcher);
 
     tableIndices.add(dataMap);
-    allDataMaps.put(table.getTableId(), tableIndices);
+    allIndexes.put(table.getTableId(), tableIndices);
     tablePathMap.put(table.getTableId(), table.getTablePath());
     return dataMap;
   }
 
-  private TableDataMap getTableDataMap(String dataMapName, List<TableDataMap> tableIndices) {
-    TableDataMap dataMap = null;
-    for (TableDataMap tableDataMap : tableIndices) {
-      if (tableDataMap.getDataMapSchema().getDataMapName().equals(dataMapName)) {
-        dataMap = tableDataMap;
+  private TableIndex getTableIndex(String dataMapName, List<TableIndex> tableIndices) {
+    TableIndex dataMap = null;
+    for (TableIndex tableIndex : tableIndices) {
+      if (tableIndex.getDataMapSchema().getDataMapName().equals(dataMapName)) {
+        dataMap = tableIndex;
         break;
       }
     }
@@ -481,10 +493,10 @@ public final class DataMapStoreManager {
    */
   public void clearInvalidSegments(CarbonTable carbonTable, List<String> segments)
       throws IOException {
-    getDefaultDataMap(carbonTable).clear(segments);
-    List<TableDataMap> allDataMap = getAllDataMap(carbonTable);
-    for (TableDataMap dataMap: allDataMap) {
-      dataMap.clear(segments);
+    getDefaultIndex(carbonTable).clear(segments);
+    List<TableIndex> indexes = getAllIndexes(carbonTable);
+    for (TableIndex index: indexes) {
+      index.clear(segments);
     }
 
   }
@@ -517,19 +529,19 @@ public final class DataMapStoreManager {
    *
    * @param identifier Table identifier
    */
-  public void clearDataMaps(AbsoluteTableIdentifier identifier) {
+  public void clearIndex(AbsoluteTableIdentifier identifier) {
     CarbonTable carbonTable = getCarbonTable(identifier);
     boolean launchJob = false;
     try {
       // launchJob will be true if either the table has a CGDatamap or index server is enabled for
       // the specified table.
-      launchJob = hasCGDataMap(carbonTable) ||
+      launchJob = hasCGIndex(carbonTable) ||
           CarbonProperties.getInstance().isDistributedPruningEnabled(identifier.getDatabaseName(),
               identifier.getTableName());
     } catch (IOException e) {
       LOGGER.warn("Unable to launch job to clear datamaps.", e);
     }
-    clearDataMapCache(identifier, launchJob);
+    clearIndexCache(identifier, launchJob);
   }
 
   /**
@@ -537,7 +549,7 @@ public final class DataMapStoreManager {
    *
    * @param identifier Table identifier
    */
-  public void clearDataMapCache(AbsoluteTableIdentifier identifier, boolean clearInAllWorkers) {
+  public void clearIndexCache(AbsoluteTableIdentifier identifier, boolean clearInAllWorkers) {
     String tableId = identifier.getCarbonTableIdentifier().getTableId();
     if (clearInAllWorkers) {
       // carbon table need to lookup only if launch job is set.
@@ -546,12 +558,12 @@ public final class DataMapStoreManager {
         String jobClassName;
         if (CarbonProperties.getInstance()
             .isDistributedPruningEnabled(identifier.getDatabaseName(), identifier.getTableName())) {
-          jobClassName = DataMapUtil.DISTRIBUTED_JOB_NAME;
+          jobClassName = IndexUtil.DISTRIBUTED_JOB_NAME;
         } else {
-          jobClassName = DataMapUtil.EMBEDDED_JOB_NAME;
+          jobClassName = IndexUtil.EMBEDDED_JOB_NAME;
         }
         try {
-          DataMapUtil.executeClearDataMapJob(carbonTable, jobClassName);
+          IndexUtil.executeClearIndexJob(carbonTable, jobClassName);
         } catch (IOException e) {
           LOGGER.error("clear dataMap job failed", e);
           // ignoring the exception
@@ -563,8 +575,8 @@ public final class DataMapStoreManager {
       CarbonMetadata.getInstance()
           .removeTable(identifier.getDatabaseName(), identifier.getTableName());
     }
-    List<TableDataMap> tableIndices =
-        allDataMaps.get(identifier.getCarbonTableIdentifier().getTableId());
+    List<TableIndex> tableIndices =
+        allIndexes.get(identifier.getCarbonTableIdentifier().getTableId());
     if (tableIndices == null) {
       String keyUsingTablePath = getKeyUsingTablePath(identifier.getTablePath());
       if (keyUsingTablePath != null) {
@@ -572,8 +584,8 @@ public final class DataMapStoreManager {
       }
     }
     segmentRefreshMap.remove(tableId);
-    clearDataMaps(tableId);
-    allDataMaps.remove(tableId);
+    clearIndex(tableId);
+    allIndexes.remove(tableId);
     tablePathMap.remove(tableId);
   }
 
@@ -602,19 +614,19 @@ public final class DataMapStoreManager {
   /**
    * this methods clears the datamap of table from memory
    */
-  public void clearDataMaps(String tableId) {
-    List<TableDataMap> tableIndices = allDataMaps.get(tableId);
+  public void clearIndex(String tableId) {
+    List<TableIndex> tableIndices = allIndexes.get(tableId);
     if (tableIndices != null) {
-      for (TableDataMap tableDataMap : tableIndices) {
-        if (tableDataMap != null) {
+      for (TableIndex tableIndex : tableIndices) {
+        if (tableIndex != null) {
           // clear the segmentMap in BlockletDetailsFetcher,else the Segment will remain in executor
           // and the query fails as we will check whether the blocklet contains in the index or not
-          tableDataMap.getBlockletDetailsFetcher().clear();
-          tableDataMap.clear();
+          tableIndex.getBlockletDetailsFetcher().clear();
+          tableIndex.clear();
         }
       }
     }
-    allDataMaps.remove(tableId);
+    allIndexes.remove(tableId);
     tablePathMap.remove(tableId);
   }
 
@@ -624,7 +636,7 @@ public final class DataMapStoreManager {
   /**
    * Clear the datamap/datamaps of a table from memory and disk
    */
-  public void deleteDataMap(CarbonTable carbonTable, String dataMapName) {
+  public void deleteIndex(CarbonTable carbonTable, String dataMapName) {
     if (carbonTable == null) {
       // If carbon table is null then it means table is already deleted, therefore return without
       // doing any further changes.
@@ -634,34 +646,34 @@ public final class DataMapStoreManager {
     if (CarbonProperties.getInstance()
         .isDistributedPruningEnabled(carbonTable.getDatabaseName(), carbonTable.getTableName())) {
       try {
-        DataMapUtil
-            .executeClearDataMapJob(carbonTable, DataMapUtil.DISTRIBUTED_JOB_NAME, dataMapName);
+        IndexUtil
+            .executeClearIndexJob(carbonTable, IndexUtil.DISTRIBUTED_JOB_NAME, dataMapName);
       } catch (IOException e) {
         LOGGER.error("clear dataMap job failed", e);
         // ignoring the exception
       }
     } else {
-      List<TableDataMap> tableIndices = allDataMaps.get(tableId);
+      List<TableIndex> tableIndices = allIndexes.get(tableId);
       if (tableIndices != null) {
         int i = 0;
-        for (TableDataMap tableDataMap : tableIndices) {
-          if (tableDataMap != null && dataMapName
-              .equalsIgnoreCase(tableDataMap.getDataMapSchema().getDataMapName())) {
+        for (TableIndex tableIndex : tableIndices) {
+          if (tableIndex != null && dataMapName
+              .equalsIgnoreCase(tableIndex.getDataMapSchema().getDataMapName())) {
             try {
-              DataMapUtil
-                  .executeClearDataMapJob(carbonTable, DataMapUtil.EMBEDDED_JOB_NAME, dataMapName);
-              tableDataMap.clear();
+              IndexUtil
+                  .executeClearIndexJob(carbonTable, IndexUtil.EMBEDDED_JOB_NAME, dataMapName);
+              tableIndex.clear();
             } catch (IOException e) {
               LOGGER.error("clear dataMap job failed", e);
               // ignoring the exception
             }
-            tableDataMap.deleteDatamapData();
+            tableIndex.deleteDatamapData();
             tableIndices.remove(i);
             break;
           }
           i++;
         }
-        allDataMaps.put(tableId, tableIndices);
+        allIndexes.put(tableId, tableIndices);
       }
     }
   }
@@ -671,9 +683,9 @@ public final class DataMapStoreManager {
    * @return true if exist, else return false
    */
   public boolean isDataMapExist(String tableId, String dmName) {
-    List<TableDataMap> tableDataMaps = allDataMaps.get(tableId);
-    if (tableDataMaps != null) {
-      for (TableDataMap dm : tableDataMaps) {
+    List<TableIndex> tableIndices = allIndexes.get(tableId);
+    if (tableIndices != null) {
+      for (TableIndex dm : tableIndices) {
         if (dm != null && dmName.equalsIgnoreCase(dm.getDataMapSchema().getDataMapName())) {
           return true;
         }
@@ -689,8 +701,8 @@ public final class DataMapStoreManager {
    * @return
    */
   private BlockletDetailsFetcher getBlockletDetailsFetcher(CarbonTable table) {
-    TableDataMap blockletMap = getDataMap(table, BlockletDataMapFactory.DATA_MAP_SCHEMA);
-    return (BlockletDetailsFetcher) blockletMap.getDataMapFactory();
+    TableIndex index = getIndex(table, BlockletIndexFactory.DATA_MAP_SCHEMA);
+    return (BlockletDetailsFetcher) index.getIndexFactory();
   }
 
   /**
@@ -781,35 +793,35 @@ public final class DataMapStoreManager {
     }
   }
 
-  public synchronized void clearInvalidDataMaps(CarbonTable carbonTable, List<String> segmentNos,
+  public synchronized void clearInvalidIndex(CarbonTable carbonTable, List<String> segmentNos,
       String dataMapToClear) throws IOException {
-    List<TableDataMap> dataMaps = getAllDataMap(carbonTable);
-    List<TableDataMap> remainingDataMaps = new ArrayList<>();
+    List<TableIndex> indexes = getAllIndexes(carbonTable);
+    List<TableIndex> remainingIndexes = new ArrayList<>();
     if (StringUtils.isNotEmpty(dataMapToClear)) {
-      Iterator<TableDataMap> dataMapIterator = dataMaps.iterator();
+      Iterator<TableIndex> dataMapIterator = indexes.iterator();
       while (dataMapIterator.hasNext()) {
-        TableDataMap tableDataMap = dataMapIterator.next();
-        if (dataMapToClear.equalsIgnoreCase(tableDataMap.getDataMapSchema().getDataMapName())) {
+        TableIndex tableIndex = dataMapIterator.next();
+        if (dataMapToClear.equalsIgnoreCase(tableIndex.getDataMapSchema().getDataMapName())) {
           for (String segment: segmentNos) {
-            tableDataMap.deleteSegmentDatamapData(segment);
+            tableIndex.deleteSegmentDatamapData(segment);
           }
-          tableDataMap.clear();
+          tableIndex.clear();
         } else {
-          remainingDataMaps.add(tableDataMap);
+          remainingIndexes.add(tableIndex);
         }
       }
-      allDataMaps.put(carbonTable.getTableId(), remainingDataMaps);
+      allIndexes.put(carbonTable.getTableId(), remainingIndexes);
     } else {
-      clearDataMaps(carbonTable.getTableId());
+      clearIndex(carbonTable.getTableId());
       // clear the segment properties cache from executor
       SegmentPropertiesAndSchemaHolder.getInstance()
           .invalidate(carbonTable.getAbsoluteTableIdentifier());
     }
   }
 
-  private boolean hasCGDataMap(CarbonTable carbonTable) throws IOException {
-    for (TableDataMap tableDataMap : getAllVisibleDataMap(carbonTable)) {
-      if (tableDataMap.getDataMapFactory().getDataMapLevel().equals(DataMapLevel.CG)) {
+  private boolean hasCGIndex(CarbonTable carbonTable) throws IOException {
+    for (TableIndex tableIndex : getAllVisibleIndexes(carbonTable)) {
+      if (tableIndex.getIndexFactory().getDataMapLevel().equals(IndexLevel.CG)) {
         return true;
       }
     }
diff --git a/core/src/main/java/org/apache/carbondata/core/datamap/DataMapChooser.java b/core/src/main/java/org/apache/carbondata/core/datamap/IndexChooser.java
similarity index 72%
rename from core/src/main/java/org/apache/carbondata/core/datamap/DataMapChooser.java
rename to core/src/main/java/org/apache/carbondata/core/datamap/IndexChooser.java
index 2831094..3a368c3 100644
--- a/core/src/main/java/org/apache/carbondata/core/datamap/DataMapChooser.java
+++ b/core/src/main/java/org/apache/carbondata/core/datamap/IndexChooser.java
@@ -26,10 +26,10 @@ import java.util.Map;
 import java.util.Set;
 
 import org.apache.carbondata.common.annotations.InterfaceAudience;
-import org.apache.carbondata.core.datamap.dev.expr.AndDataMapExprWrapper;
-import org.apache.carbondata.core.datamap.dev.expr.DataMapExprWrapper;
-import org.apache.carbondata.core.datamap.dev.expr.DataMapExprWrapperImpl;
-import org.apache.carbondata.core.datamap.dev.expr.OrDataMapExprWrapper;
+import org.apache.carbondata.core.datamap.dev.expr.AndIndexExprWrapper;
+import org.apache.carbondata.core.datamap.dev.expr.IndexExprWrapper;
+import org.apache.carbondata.core.datamap.dev.expr.IndexExprWrapperImpl;
+import org.apache.carbondata.core.datamap.dev.expr.OrIndexExprWrapper;
 import org.apache.carbondata.core.datamap.status.DataMapStatusDetail;
 import org.apache.carbondata.core.datamap.status.DataMapStatusManager;
 import org.apache.carbondata.core.metadata.schema.table.CarbonTable;
@@ -59,76 +59,76 @@ import org.apache.carbondata.core.scan.filter.resolver.resolverinfo.TrueConditio
  *   the datamap which has fewer columns that is the first datamap.
  */
 @InterfaceAudience.Internal
-public class DataMapChooser {
+public class IndexChooser {
 
   private CarbonTable carbonTable;
-  private List<TableDataMap> cgDataMaps;
-  private List<TableDataMap> fgDataMaps;
+  private List<TableIndex> cgIndexes;
+  private List<TableIndex> fgIndexes;
 
-  public DataMapChooser(CarbonTable carbonTable) throws IOException {
+  public IndexChooser(CarbonTable carbonTable) throws IOException {
     this.carbonTable = carbonTable;
-    // read all datamaps for this table and populate CG and FG datamap list
-    List<TableDataMap> visibleDataMaps =
-        DataMapStoreManager.getInstance().getAllVisibleDataMap(carbonTable);
+    // read all indexes for this table and populate CG and FG index list
+    List<TableIndex> visibleIndexes =
+        DataMapStoreManager.getInstance().getAllVisibleIndexes(carbonTable);
     Map<String, DataMapStatusDetail> map = DataMapStatusManager.readDataMapStatusMap();
-    cgDataMaps = new ArrayList<>(visibleDataMaps.size());
-    fgDataMaps = new ArrayList<>(visibleDataMaps.size());
-    for (TableDataMap visibleDataMap : visibleDataMaps) {
-      DataMapStatusDetail status = map.get(visibleDataMap.getDataMapSchema().getDataMapName());
+    cgIndexes = new ArrayList<>(visibleIndexes.size());
+    fgIndexes = new ArrayList<>(visibleIndexes.size());
+    for (TableIndex visibleIndex : visibleIndexes) {
+      DataMapStatusDetail status = map.get(visibleIndex.getDataMapSchema().getDataMapName());
       if (status != null && status.isEnabled()) {
-        DataMapLevel level = visibleDataMap.getDataMapFactory().getDataMapLevel();
-        if (level == DataMapLevel.CG) {
-          cgDataMaps.add(visibleDataMap);
+        IndexLevel level = visibleIndex.getIndexFactory().getDataMapLevel();
+        if (level == IndexLevel.CG) {
+          cgIndexes.add(visibleIndex);
         } else {
-          fgDataMaps.add(visibleDataMap);
+          fgIndexes.add(visibleIndex);
         }
       }
     }
   }
 
   /**
-   * Return a chosen datamap based on input filter. See {@link DataMapChooser}
+   * Return a chosen datamap based on input filter. See {@link IndexChooser}
    */
-  public DataMapExprWrapper choose(FilterResolverIntf filter) {
+  public IndexExprWrapper choose(FilterResolverIntf filter) {
     if (filter != null) {
       Expression expression = filter.getFilterExpression();
       // First check for FG datamaps if any exist
-      ExpressionTuple tuple = selectDataMap(expression, fgDataMaps, filter);
-      if (tuple.dataMapExprWrapper == null) {
+      ExpressionTuple tuple = selectDataMap(expression, fgIndexes, filter);
+      if (tuple.indexExprWrapper == null) {
         // Check for CG datamap
-        tuple = selectDataMap(expression, cgDataMaps, filter);
+        tuple = selectDataMap(expression, cgIndexes, filter);
       }
-      if (tuple.dataMapExprWrapper != null) {
-        return tuple.dataMapExprWrapper;
+      if (tuple.indexExprWrapper != null) {
+        return tuple.indexExprWrapper;
       }
     }
     // Return the default datamap if no other datamap exists.
-    return new DataMapExprWrapperImpl(
-        DataMapStoreManager.getInstance().getDefaultDataMap(carbonTable), filter);
+    return new IndexExprWrapperImpl(
+        DataMapStoreManager.getInstance().getDefaultIndex(carbonTable), filter);
   }
 
   /**
-   * Return a chosen FG datamap based on input filter. See {@link DataMapChooser}
+   * Return a chosen FG datamap based on input filter. See {@link IndexChooser}
    */
-  public DataMapExprWrapper chooseFGDataMap(FilterResolverIntf resolverIntf) {
-    return chooseDataMap(DataMapLevel.FG, resolverIntf);
+  public IndexExprWrapper chooseFGDataMap(FilterResolverIntf resolverIntf) {
+    return chooseDataMap(IndexLevel.FG, resolverIntf);
   }
 
   /**
-   * Return a chosen CG datamap based on input filter. See {@link DataMapChooser}
+   * Return a chosen CG datamap based on input filter. See {@link IndexChooser}
    */
-  public DataMapExprWrapper chooseCGDataMap(FilterResolverIntf resolverIntf) {
-    return chooseDataMap(DataMapLevel.CG, resolverIntf);
+  public IndexExprWrapper chooseCGDataMap(FilterResolverIntf resolverIntf) {
+    return chooseDataMap(IndexLevel.CG, resolverIntf);
   }
 
-  DataMapExprWrapper chooseDataMap(DataMapLevel level, FilterResolverIntf resolverIntf) {
+  IndexExprWrapper chooseDataMap(IndexLevel level, FilterResolverIntf resolverIntf) {
     if (resolverIntf != null) {
       Expression expression = resolverIntf.getFilterExpression();
-      List<TableDataMap> datamaps = level == DataMapLevel.CG ? cgDataMaps : fgDataMaps;
+      List<TableIndex> datamaps = level == IndexLevel.CG ? cgIndexes : fgIndexes;
       if (datamaps.size() > 0) {
         ExpressionTuple tuple = selectDataMap(expression, datamaps, resolverIntf);
-        if (tuple.dataMapExprWrapper != null) {
-          return tuple.dataMapExprWrapper;
+        if (tuple.indexExprWrapper != null) {
+          return tuple.indexExprWrapper;
         }
       }
     }
@@ -141,14 +141,14 @@ public class DataMapChooser {
    * @param resolverIntf
    * @return
    */
-  public static DataMapExprWrapper getDefaultDataMap(CarbonTable carbonTable,
+  public static IndexExprWrapper getDefaultDataMap(CarbonTable carbonTable,
       FilterResolverIntf resolverIntf) {
     // Return the default datamap if no other datamap exists.
-    return new DataMapExprWrapperImpl(
-        DataMapStoreManager.getInstance().getDefaultDataMap(carbonTable), resolverIntf);
+    return new IndexExprWrapperImpl(
+        DataMapStoreManager.getInstance().getDefaultIndex(carbonTable), resolverIntf);
   }
 
-  private ExpressionTuple selectDataMap(Expression expression, List<TableDataMap> allDataMap,
+  private ExpressionTuple selectDataMap(Expression expression, List<TableIndex> allDataMap,
       FilterResolverIntf filterResolverIntf) {
     switch (expression.getFilterExpressionType()) {
       case AND:
@@ -161,14 +161,14 @@ public class DataMapChooser {
           Set<ExpressionType> filterExpressionTypes = new HashSet<>();
           // If both left and right has datamap then we can either merge both datamaps to single
           // datamap if possible. Otherwise apply AND expression.
-          if (left.dataMapExprWrapper != null && right.dataMapExprWrapper != null) {
+          if (left.indexExprWrapper != null && right.indexExprWrapper != null) {
             filterExpressionTypes.addAll(left.filterExpressionTypes);
             filterExpressionTypes.addAll(right.filterExpressionTypes);
             List<ColumnExpression> columnExpressions = new ArrayList<>();
             columnExpressions.addAll(left.columnExpressions);
             columnExpressions.addAll(right.columnExpressions);
             // Check if we can merge them to single datamap.
-            TableDataMap dataMap =
+            TableIndex dataMap =
                 chooseDataMap(allDataMap, columnExpressions, filterExpressionTypes);
             TrueConditionalResolverImpl resolver = new TrueConditionalResolverImpl(
                 new AndExpression(left.expression, right.expression), false,
@@ -176,21 +176,21 @@ public class DataMapChooser {
             if (dataMap != null) {
               ExpressionTuple tuple = new ExpressionTuple();
               tuple.columnExpressions = columnExpressions;
-              tuple.dataMapExprWrapper = new DataMapExprWrapperImpl(dataMap, resolver);
+              tuple.indexExprWrapper = new IndexExprWrapperImpl(dataMap, resolver);
               tuple.expression = resolver.getFilterExpression();
               return tuple;
             } else {
               // Apply AND expression.
               ExpressionTuple tuple = new ExpressionTuple();
               tuple.columnExpressions = columnExpressions;
-              tuple.dataMapExprWrapper = new AndDataMapExprWrapper(left.dataMapExprWrapper,
-                  right.dataMapExprWrapper, resolver);
+              tuple.indexExprWrapper = new AndIndexExprWrapper(left.indexExprWrapper,
+                  right.indexExprWrapper, resolver);
               tuple.expression = resolver.getFilterExpression();
               return tuple;
             }
-          } else if (left.dataMapExprWrapper != null) {
+          } else if (left.indexExprWrapper != null) {
             return left;
-          } else if (right.dataMapExprWrapper != null) {
+          } else if (right.indexExprWrapper != null) {
             return right;
           } else {
             return left;
@@ -206,7 +206,7 @@ public class DataMapChooser {
               filterResolverIntf.getRight());
           // If both left and right has datamap then we can either merge both datamaps to single
           // datamap if possible. Otherwise apply OR expression.
-          if (left.dataMapExprWrapper != null && right.dataMapExprWrapper != null) {
+          if (left.indexExprWrapper != null && right.indexExprWrapper != null) {
             TrueConditionalResolverImpl resolver = new TrueConditionalResolverImpl(
                 new OrExpression(left.expression, right.expression), false,
                 true);
@@ -215,12 +215,12 @@ public class DataMapChooser {
             columnExpressions.addAll(right.columnExpressions);
             ExpressionTuple tuple = new ExpressionTuple();
             tuple.columnExpressions = columnExpressions;
-            tuple.dataMapExprWrapper = new OrDataMapExprWrapper(left.dataMapExprWrapper,
-                right.dataMapExprWrapper, resolver);
+            tuple.indexExprWrapper = new OrIndexExprWrapper(left.indexExprWrapper,
+                right.indexExprWrapper, resolver);
             tuple.expression = resolver.getFilterExpression();
             return tuple;
           } else {
-            left.dataMapExprWrapper = null;
+            left.indexExprWrapper = null;
             return left;
           }
         }
@@ -233,10 +233,10 @@ public class DataMapChooser {
         TrueConditionalResolverImpl resolver = new TrueConditionalResolverImpl(
             filterResolverIntf.getFilterExpression(), false,
             true);
-        TableDataMap dataMap =
+        TableIndex dataMap =
             chooseDataMap(allDataMap, tuple.columnExpressions, filterExpressionTypes);
         if (dataMap != null) {
-          tuple.dataMapExprWrapper = new DataMapExprWrapperImpl(dataMap, resolver);
+          tuple.indexExprWrapper = new IndexExprWrapperImpl(dataMap, resolver);
           tuple.filterExpressionTypes.addAll(filterExpressionTypes);
           tuple.expression = filterResolverIntf.getFilterExpression();
         }
@@ -269,14 +269,14 @@ public class DataMapChooser {
     }
   }
 
-  private TableDataMap chooseDataMap(List<TableDataMap> allDataMap,
+  private TableIndex chooseDataMap(List<TableIndex> allDataMap,
       List<ColumnExpression> columnExpressions, Set<ExpressionType> expressionTypes) {
-    List<DataMapTuple> tuples = new ArrayList<>();
-    for (TableDataMap dataMap : allDataMap) {
-      if (null != dataMap.getDataMapFactory().getMeta() && contains(
-          dataMap.getDataMapFactory().getMeta(), columnExpressions, expressionTypes)) {
+    List<IndexTuple> tuples = new ArrayList<>();
+    for (TableIndex dataMap : allDataMap) {
+      if (null != dataMap.getIndexFactory().getMeta() && contains(
+          dataMap.getIndexFactory().getMeta(), columnExpressions, expressionTypes)) {
         tuples.add(
-            new DataMapTuple(dataMap.getDataMapFactory().getMeta().getIndexedColumns().size(),
+            new IndexTuple(dataMap.getIndexFactory().getMeta().getIndexedColumns().size(),
                 dataMap));
       }
     }
@@ -291,7 +291,7 @@ public class DataMapChooser {
    * Return true if the input datamap contains the column that needed in
    * specified expression
    */
-  private boolean contains(DataMapMeta mapMeta, List<ColumnExpression> columnExpressions,
+  private boolean contains(IndexMeta mapMeta, List<ColumnExpression> columnExpressions,
       Set<ExpressionType> expressionTypes) {
     if (mapMeta.getIndexedColumns().size() == 0 || columnExpressions.size() == 0) {
       return false;
@@ -309,7 +309,7 @@ public class DataMapChooser {
 
   private static class ExpressionTuple {
 
-    DataMapExprWrapper dataMapExprWrapper;
+    IndexExprWrapper indexExprWrapper;
 
     List<ColumnExpression> columnExpressions = new ArrayList<>();
 
@@ -319,19 +319,19 @@ public class DataMapChooser {
 
   }
 
-  private static class DataMapTuple implements Comparable<DataMapTuple> {
+  private static class IndexTuple implements Comparable<IndexTuple> {
 
     int order;
 
-    TableDataMap dataMap;
+    TableIndex dataMap;
 
-    public DataMapTuple(int order, TableDataMap dataMap) {
+    public IndexTuple(int order, TableIndex dataMap) {
       this.order = order;
       this.dataMap = dataMap;
     }
 
     @Override
-    public int compareTo(DataMapTuple o) {
+    public int compareTo(IndexTuple o) {
       return order - o.order;
     }
 
@@ -340,7 +340,7 @@ public class DataMapChooser {
       if (this == o) return true;
       if (o == null || getClass() != o.getClass()) return false;
 
-      DataMapTuple that = (DataMapTuple) o;
+      IndexTuple that = (IndexTuple) o;
 
       if (order != that.order) return false;
       return dataMap != null ? dataMap.equals(that.dataMap) : that.dataMap == null;
diff --git a/core/src/main/java/org/apache/carbondata/core/datamap/DataMapFilter.java b/core/src/main/java/org/apache/carbondata/core/datamap/IndexFilter.java
similarity index 94%
rename from core/src/main/java/org/apache/carbondata/core/datamap/DataMapFilter.java
rename to core/src/main/java/org/apache/carbondata/core/datamap/IndexFilter.java
index f85fd53..a015a45 100644
--- a/core/src/main/java/org/apache/carbondata/core/datamap/DataMapFilter.java
+++ b/core/src/main/java/org/apache/carbondata/core/datamap/IndexFilter.java
@@ -42,9 +42,9 @@ import org.apache.carbondata.core.scan.model.QueryModel;
 import org.apache.carbondata.core.util.ObjectSerializationUtil;
 
 /**
- * the filter of DataMap
+ * the filter of Index
  */
-public class DataMapFilter implements Serializable {
+public class IndexFilter implements Serializable {
 
   private static final long serialVersionUID = 6276855832288220240L;
 
@@ -62,11 +62,11 @@ public class DataMapFilter implements Serializable {
 
   private SegmentProperties properties;
 
-  public DataMapFilter(CarbonTable table, Expression expression) {
+  public IndexFilter(CarbonTable table, Expression expression) {
     this(table, expression, false);
   }
 
-  public DataMapFilter(CarbonTable table, Expression expression, boolean lazyResolve) {
+  public IndexFilter(CarbonTable table, Expression expression, boolean lazyResolve) {
     this.expression = expression;
     this.table = table;
     resolve(lazyResolve);
@@ -81,11 +81,11 @@ public class DataMapFilter implements Serializable {
     }
   }
 
-  public DataMapFilter(FilterResolverIntf resolver) {
+  public IndexFilter(FilterResolverIntf resolver) {
     this.resolver = resolver;
   }
 
-  public DataMapFilter(SegmentProperties properties, CarbonTable table, Expression expression) {
+  public IndexFilter(SegmentProperties properties, CarbonTable table, Expression expression) {
     this(table, expression);
     this.properties = properties;
     resolve(false);
@@ -165,7 +165,7 @@ public class DataMapFilter implements Serializable {
    * Process the FilterExpression and create FilterResolverIntf.
    *
    * @param lazyResolve whether to create FilterResolverIntf immediately or not.
-   *                   Pass true if DataMapFilter object is created before checking the valid
+   *                   Pass true if IndexFilter object is created before checking the valid
    *                   segments for pruning.
    */
   public void resolve(boolean lazyResolve) {
diff --git a/core/src/main/java/org/apache/carbondata/core/datamap/DistributableDataMapFormat.java b/core/src/main/java/org/apache/carbondata/core/datamap/IndexInputFormat.java
similarity index 88%
rename from core/src/main/java/org/apache/carbondata/core/datamap/DistributableDataMapFormat.java
rename to core/src/main/java/org/apache/carbondata/core/datamap/IndexInputFormat.java
index 7eff356..3a2b3e6 100644
--- a/core/src/main/java/org/apache/carbondata/core/datamap/DistributableDataMapFormat.java
+++ b/core/src/main/java/org/apache/carbondata/core/datamap/IndexInputFormat.java
@@ -29,7 +29,7 @@ import java.util.UUID;
 
 import org.apache.carbondata.common.logging.LogServiceFactory;
 import org.apache.carbondata.core.constants.CarbonCommonConstants;
-import org.apache.carbondata.core.datamap.dev.expr.DataMapDistributableWrapper;
+import org.apache.carbondata.core.datamap.dev.expr.IndexInputSplitWrapper;
 import org.apache.carbondata.core.datastore.impl.FileFactory;
 import org.apache.carbondata.core.indexstore.ExtendedBlocklet;
 import org.apache.carbondata.core.indexstore.PartitionSpec;
@@ -53,11 +53,11 @@ import org.apache.log4j.Logger;
 /**
  * Input format for datamaps, it makes the datamap pruning distributable.
  */
-public class DistributableDataMapFormat extends FileInputFormat<Void, ExtendedBlocklet>
+public class IndexInputFormat extends FileInputFormat<Void, ExtendedBlocklet>
     implements Serializable, Writable {
 
   private static final transient Logger LOGGER =
-      LogServiceFactory.getLogService(DistributableDataMapFormat.class.getName());
+      LogServiceFactory.getLogService(IndexInputFormat.class.getName());
 
   private static final long serialVersionUID = 9189779090091151248L;
 
@@ -73,7 +73,7 @@ public class DistributableDataMapFormat extends FileInputFormat<Void, ExtendedBl
 
   private boolean isJobToClearDataMaps = false;
 
-  private DataMapLevel dataMapLevel;
+  private IndexLevel indexLevel;
 
   private boolean isFallbackJob = false;
 
@@ -87,7 +87,7 @@ public class DistributableDataMapFormat extends FileInputFormat<Void, ExtendedBl
 
   private String queryId = UUID.randomUUID().toString();
 
-  private transient DataMapChooser dataMapChooser;
+  private transient IndexChooser indexChooser;
 
   private boolean isWriteToFile = true;
 
@@ -96,11 +96,11 @@ public class DistributableDataMapFormat extends FileInputFormat<Void, ExtendedBl
   // Whether AsyncCall to the Index Server(true in the case of prepriming)
   private boolean isAsyncCall;
 
-  DistributableDataMapFormat() {
+  IndexInputFormat() {
 
   }
 
-  DistributableDataMapFormat(CarbonTable table,
+  IndexInputFormat(CarbonTable table,
       List<Segment> validSegments, List<String> invalidSegments, boolean isJobToClearDataMaps,
       String dataMapToClear) {
     this(table, null, validSegments, invalidSegments, null,
@@ -108,9 +108,9 @@ public class DistributableDataMapFormat extends FileInputFormat<Void, ExtendedBl
     this.dataMapToClear = dataMapToClear;
   }
 
-  public DistributableDataMapFormat(CarbonTable table, FilterResolverIntf filterResolverIntf,
+  public IndexInputFormat(CarbonTable table, FilterResolverIntf filterResolverIntf,
       List<Segment> validSegments, List<String> invalidSegments, List<PartitionSpec> partitions,
-      boolean isJobToClearDataMaps, DataMapLevel dataMapLevel, boolean isFallbackJob,
+      boolean isJobToClearDataMaps, IndexLevel indexLevel, boolean isFallbackJob,
       boolean isAsyncCall) {
     this.table = table;
     this.filterResolverIntf = filterResolverIntf;
@@ -121,16 +121,16 @@ public class DistributableDataMapFormat extends FileInputFormat<Void, ExtendedBl
     this.invalidSegments = invalidSegments;
     this.partitions = partitions;
     this.isJobToClearDataMaps = isJobToClearDataMaps;
-    this.dataMapLevel = dataMapLevel;
+    this.indexLevel = indexLevel;
     this.isFallbackJob = isFallbackJob;
     this.isAsyncCall = isAsyncCall;
   }
 
   @Override
   public List<InputSplit> getSplits(JobContext job) throws IOException {
-    List<DataMapDistributableWrapper> distributables;
+    List<IndexInputSplitWrapper> distributables;
     distributables =
-        DataMapChooser.getDefaultDataMap(table, filterResolverIntf).toDistributable(validSegments);
+        IndexChooser.getDefaultDataMap(table, filterResolverIntf).toDistributable(validSegments);
     List<InputSplit> inputSplits = new ArrayList<>(distributables.size());
     inputSplits.addAll(distributables);
     return inputSplits;
@@ -146,23 +146,23 @@ public class DistributableDataMapFormat extends FileInputFormat<Void, ExtendedBl
       @Override
       public void initialize(InputSplit inputSplit, TaskAttemptContext taskAttemptContext)
           throws IOException {
-        DataMapDistributableWrapper distributable = (DataMapDistributableWrapper) inputSplit;
+        IndexInputSplitWrapper distributable = (IndexInputSplitWrapper) inputSplit;
         distributable.getDistributable().getSegment().setReadCommittedScope(readCommittedScope);
         List<Segment> segmentsToLoad = new ArrayList<>();
         segmentsToLoad.add(distributable.getDistributable().getSegment());
         List<ExtendedBlocklet> blocklets = new ArrayList<>();
-        if (dataMapLevel == null) {
-          TableDataMap defaultDataMap = DataMapStoreManager.getInstance()
-              .getDataMap(table, distributable.getDistributable().getDataMapSchema());
+        if (indexLevel == null) {
+          TableIndex defaultDataMap = DataMapStoreManager.getInstance()
+              .getIndex(table, distributable.getDistributable().getDataMapSchema());
           blocklets = defaultDataMap
-              .prune(segmentsToLoad, new DataMapFilter(filterResolverIntf), partitions);
-          blocklets = DataMapUtil
+              .prune(segmentsToLoad, new IndexFilter(filterResolverIntf), partitions);
+          blocklets = IndexUtil
               .pruneDataMaps(table, filterResolverIntf, segmentsToLoad, partitions, blocklets,
-                  dataMapChooser);
+                  indexChooser);
         } else {
-          blocklets = DataMapUtil
+          blocklets = IndexUtil
               .pruneDataMaps(table, filterResolverIntf, segmentsToLoad, partitions, blocklets,
-                  dataMapLevel, dataMapChooser);
+                  indexLevel, indexChooser);
         }
         blockletIterator = blocklets.iterator();
       }
@@ -199,7 +199,7 @@ public class DistributableDataMapFormat extends FileInputFormat<Void, ExtendedBl
         // Clear the datamaps from executor
         if (isFallbackJob) {
           DataMapStoreManager.getInstance()
-              .clearDataMapCache(table.getAbsoluteTableIdentifier(), false);
+              .clearIndexCache(table.getAbsoluteTableIdentifier(), false);
         }
       }
     };
@@ -218,11 +218,11 @@ public class DistributableDataMapFormat extends FileInputFormat<Void, ExtendedBl
     }
     out.writeBoolean(isJobToClearDataMaps);
     out.writeBoolean(isFallbackJob);
-    if (dataMapLevel == null) {
+    if (indexLevel == null) {
       out.writeBoolean(false);
     } else {
       out.writeBoolean(true);
-      out.writeUTF(dataMapLevel.name());
+      out.writeUTF(indexLevel.name());
     }
     out.writeInt(validSegments.size());
     for (Segment segment : validSegments) {
@@ -268,7 +268,7 @@ public class DistributableDataMapFormat extends FileInputFormat<Void, ExtendedBl
     this.isJobToClearDataMaps = in.readBoolean();
     this.isFallbackJob = in.readBoolean();
     if (in.readBoolean()) {
-      this.dataMapLevel = DataMapLevel.valueOf(in.readUTF());
+      this.indexLevel = IndexLevel.valueOf(in.readUTF());
     }
     int validSegmentSize = in.readInt();
     validSegments = new ArrayList<>(validSegmentSize);
@@ -420,7 +420,7 @@ public class DistributableDataMapFormat extends FileInputFormat<Void, ExtendedBl
 
   public void createDataMapChooser() throws IOException {
     if (null != filterResolverIntf) {
-      this.dataMapChooser = new DataMapChooser(table);
+      this.indexChooser = new IndexChooser(table);
     }
   }
 
diff --git a/core/src/main/java/org/apache/carbondata/core/datamap/DataMapDistributable.java b/core/src/main/java/org/apache/carbondata/core/datamap/IndexInputSplit.java
similarity index 95%
rename from core/src/main/java/org/apache/carbondata/core/datamap/DataMapDistributable.java
rename to core/src/main/java/org/apache/carbondata/core/datamap/IndexInputSplit.java
index 76f5c76..fa9353e 100644
--- a/core/src/main/java/org/apache/carbondata/core/datamap/DataMapDistributable.java
+++ b/core/src/main/java/org/apache/carbondata/core/datamap/IndexInputSplit.java
@@ -26,10 +26,10 @@ import org.apache.carbondata.core.metadata.schema.table.DataMapSchema;
 import org.apache.hadoop.mapreduce.InputSplit;
 
 /**
- * Distributable class for datamap.
+ * input split for index.
  */
 @InterfaceAudience.Internal
-public abstract class DataMapDistributable extends InputSplit
+public abstract class IndexInputSplit extends InputSplit
     implements Distributable, Serializable {
 
   private String tablePath;
diff --git a/core/src/main/java/org/apache/carbondata/core/datamap/DataMapJob.java b/core/src/main/java/org/apache/carbondata/core/datamap/IndexJob.java
similarity index 75%
rename from core/src/main/java/org/apache/carbondata/core/datamap/DataMapJob.java
rename to core/src/main/java/org/apache/carbondata/core/datamap/IndexJob.java
index 5bcc8d0..23ded3c 100644
--- a/core/src/main/java/org/apache/carbondata/core/datamap/DataMapJob.java
+++ b/core/src/main/java/org/apache/carbondata/core/datamap/IndexJob.java
@@ -20,22 +20,22 @@ package org.apache.carbondata.core.datamap;
 import java.io.Serializable;
 import java.util.List;
 
-import org.apache.carbondata.core.indexstore.BlockletDataMapIndexWrapper;
+import org.apache.carbondata.core.indexstore.BlockletIndexWrapper;
 import org.apache.carbondata.core.indexstore.ExtendedBlocklet;
 import org.apache.carbondata.core.metadata.schema.table.CarbonTable;
 
 import org.apache.hadoop.mapreduce.lib.input.FileInputFormat;
 
 /**
- * Distributable datamap job to execute the #DistributableDataMapFormat in cluster. it prunes the
+ * Distributable datamap job to execute the #IndexInputFormat in cluster. it prunes the
  * datamaps distributably and returns the final blocklet list
  */
-public interface DataMapJob extends Serializable {
+public interface IndexJob extends Serializable {
 
-  void execute(CarbonTable carbonTable, FileInputFormat<Void, BlockletDataMapIndexWrapper> format);
+  void execute(CarbonTable carbonTable, FileInputFormat<Void, BlockletIndexWrapper> format);
 
-  List<ExtendedBlocklet> execute(DistributableDataMapFormat dataMapFormat);
+  List<ExtendedBlocklet> execute(IndexInputFormat dataMapFormat);
 
-  Long executeCountJob(DistributableDataMapFormat dataMapFormat);
+  Long executeCountJob(IndexInputFormat dataMapFormat);
 
 }
diff --git a/core/src/main/java/org/apache/carbondata/core/datamap/DataMapLevel.java b/core/src/main/java/org/apache/carbondata/core/datamap/IndexLevel.java
similarity index 92%
rename from core/src/main/java/org/apache/carbondata/core/datamap/DataMapLevel.java
rename to core/src/main/java/org/apache/carbondata/core/datamap/IndexLevel.java
index 2e064f5..8711d6f 100644
--- a/core/src/main/java/org/apache/carbondata/core/datamap/DataMapLevel.java
+++ b/core/src/main/java/org/apache/carbondata/core/datamap/IndexLevel.java
@@ -21,11 +21,11 @@ import org.apache.carbondata.common.annotations.InterfaceAudience;
 import org.apache.carbondata.common.annotations.InterfaceStability;
 
 /**
- * Index level of the datamap
+ * Index level
  */
-@InterfaceAudience.Developer("DataMap")
+@InterfaceAudience.Developer("Index")
 @InterfaceStability.Evolving
-public enum DataMapLevel {
+public enum IndexLevel {
   /**
    * Coarse Grain Index, index is of blocklet level
    */
diff --git a/core/src/main/java/org/apache/carbondata/core/datamap/DataMapMeta.java b/core/src/main/java/org/apache/carbondata/core/datamap/IndexMeta.java
similarity index 89%
rename from core/src/main/java/org/apache/carbondata/core/datamap/DataMapMeta.java
rename to core/src/main/java/org/apache/carbondata/core/datamap/IndexMeta.java
index fb6ba67..de170e1 100644
--- a/core/src/main/java/org/apache/carbondata/core/datamap/DataMapMeta.java
+++ b/core/src/main/java/org/apache/carbondata/core/datamap/IndexMeta.java
@@ -29,24 +29,24 @@ import org.apache.commons.collections.Transformer;
 import org.apache.commons.lang3.StringUtils;
 
 /**
- * Metadata of the datamap, set by DataMap developer
+ * Metadata of the datamap, set by Index developer
  */
-@InterfaceAudience.Developer("DataMap")
+@InterfaceAudience.Developer("Index")
 @InterfaceStability.Evolving
-public class DataMapMeta {
+public class IndexMeta {
   private String dataMapName;
 
   private List<CarbonColumn> indexedColumns;
 
   private List<ExpressionType> optimizedOperation;
 
-  public DataMapMeta(List<CarbonColumn> indexedColumns,
+  public IndexMeta(List<CarbonColumn> indexedColumns,
       List<ExpressionType> optimizedOperation) {
     this.indexedColumns = indexedColumns;
     this.optimizedOperation = optimizedOperation;
   }
 
-  public DataMapMeta(String dataMapName, List<CarbonColumn> indexedColumns,
+  public IndexMeta(String dataMapName, List<CarbonColumn> indexedColumns,
       List<ExpressionType> optimizedOperation) {
     this(indexedColumns, optimizedOperation);
     this.dataMapName = dataMapName;
@@ -75,7 +75,7 @@ public class DataMapMeta {
 
   @Override
   public String toString() {
-    return new StringBuilder("DataMapMeta{")
+    return new StringBuilder("IndexMeta{")
         .append("dataMapName='").append(dataMapName).append('\'')
         .append(", indexedColumns=[")
         .append(StringUtils.join(getIndexedColumnNames(), ", ")).append("]\'")
diff --git a/core/src/main/java/org/apache/carbondata/core/datamap/DataMapRegistry.java b/core/src/main/java/org/apache/carbondata/core/datamap/IndexRegistry.java
similarity index 71%
rename from core/src/main/java/org/apache/carbondata/core/datamap/DataMapRegistry.java
rename to core/src/main/java/org/apache/carbondata/core/datamap/IndexRegistry.java
index 8c3640a..ed012ec 100644
--- a/core/src/main/java/org/apache/carbondata/core/datamap/DataMapRegistry.java
+++ b/core/src/main/java/org/apache/carbondata/core/datamap/IndexRegistry.java
@@ -25,9 +25,9 @@ import java.util.concurrent.ConcurrentHashMap;
 import org.apache.carbondata.common.annotations.InterfaceAudience;
 import org.apache.carbondata.common.annotations.InterfaceStability;
 import org.apache.carbondata.common.exceptions.MetadataProcessException;
-import org.apache.carbondata.common.exceptions.sql.MalformedDataMapCommandException;
-import org.apache.carbondata.core.datamap.dev.DataMap;
-import org.apache.carbondata.core.datamap.dev.DataMapFactory;
+import org.apache.carbondata.common.exceptions.sql.MalformedIndexCommandException;
+import org.apache.carbondata.core.datamap.dev.Index;
+import org.apache.carbondata.core.datamap.dev.IndexFactory;
 import org.apache.carbondata.core.metadata.schema.datamap.DataMapClassProvider;
 import org.apache.carbondata.core.metadata.schema.table.CarbonTable;
 import org.apache.carbondata.core.metadata.schema.table.DataMapSchema;
@@ -41,16 +41,16 @@ import org.apache.carbondata.core.metadata.schema.table.DataMapSchema;
  *  USING 'short-name-of-the-datamap'
  * }
  * otherwise, user should use the class name of the datamap implementation to create the datamap
- * (subclass of {@link DataMapFactory})
+ * (subclass of {@link IndexFactory})
  * <p>
  * {@code
  *  CREATE DATAMAP dm ON TABLE table
  *  USING 'class-name-of-the-datamap'
  * }
  */
-@InterfaceAudience.Developer("DataMap")
+@InterfaceAudience.Developer("Index")
 @InterfaceStability.Evolving
-public class DataMapRegistry {
+public class IndexRegistry {
   private static Map<String, String> shortNameToClassName = new ConcurrentHashMap<>();
 
   private static void registerDataMap(String datamapClassName, String shortName) {
@@ -64,33 +64,33 @@ public class DataMapRegistry {
     return shortNameToClassName.get(shortName);
   }
 
-  public static DataMapFactory<? extends DataMap> getDataMapFactoryByShortName(
-      CarbonTable table, DataMapSchema dataMapSchema) throws MalformedDataMapCommandException {
+  public static IndexFactory<? extends Index> getDataMapFactoryByShortName(
+      CarbonTable table, DataMapSchema dataMapSchema) throws MalformedIndexCommandException {
     String providerName = dataMapSchema.getProviderName();
     try {
       registerDataMap(
-          DataMapClassProvider.getDataMapProviderOnName(providerName).getClassName(),
-          DataMapClassProvider.getDataMapProviderOnName(providerName).getShortName());
+          DataMapClassProvider.get(providerName).getClassName(),
+          DataMapClassProvider.get(providerName).getShortName());
     } catch (UnsupportedOperationException ex) {
-      throw new MalformedDataMapCommandException("DataMap '" + providerName + "' not found", ex);
+      throw new MalformedIndexCommandException("Index '" + providerName + "' not found", ex);
     }
-    DataMapFactory<? extends DataMap> dataMapFactory;
+    IndexFactory<? extends Index> indexFactory;
     String className = getDataMapClassName(providerName.toLowerCase());
     if (className != null) {
       try {
-        dataMapFactory = (DataMapFactory<? extends DataMap>)
+        indexFactory = (IndexFactory<? extends Index>)
             Class.forName(className).getConstructors()[0].newInstance(table, dataMapSchema);
       } catch (ClassNotFoundException ex) {
-        throw new MalformedDataMapCommandException("DataMap '" + providerName + "' not found", ex);
+        throw new MalformedIndexCommandException("Index '" + providerName + "' not found", ex);
       } catch (InvocationTargetException ex) {
-        throw new MalformedDataMapCommandException(ex.getTargetException().getMessage());
+        throw new MalformedIndexCommandException(ex.getTargetException().getMessage());
       } catch (InstantiationException | IllegalAccessException | IllegalArgumentException ex) {
         throw new MetadataProcessException(
-            "failed to create DataMap '" + providerName + "': " + ex.getMessage(), ex);
+            "failed to create Index '" + providerName + "': " + ex.getMessage(), ex);
       }
     } else {
-      throw new MalformedDataMapCommandException("DataMap '" + providerName + "' not found");
+      throw new MalformedIndexCommandException("Index '" + providerName + "' not found");
     }
-    return dataMapFactory;
+    return indexFactory;
   }
 }
diff --git a/core/src/main/java/org/apache/carbondata/core/datamap/DataMapUtil.java b/core/src/main/java/org/apache/carbondata/core/datamap/IndexUtil.java
similarity index 73%
rename from core/src/main/java/org/apache/carbondata/core/datamap/DataMapUtil.java
rename to core/src/main/java/org/apache/carbondata/core/datamap/IndexUtil.java
index 41da1c2..21f4bf7 100644
--- a/core/src/main/java/org/apache/carbondata/core/datamap/DataMapUtil.java
+++ b/core/src/main/java/org/apache/carbondata/core/datamap/IndexUtil.java
@@ -28,9 +28,9 @@ import java.util.Map;
 import java.util.Set;
 
 import org.apache.carbondata.common.logging.LogServiceFactory;
-import org.apache.carbondata.core.datamap.dev.DataMap;
-import org.apache.carbondata.core.datamap.dev.expr.DataMapDistributableWrapper;
-import org.apache.carbondata.core.datamap.dev.expr.DataMapExprWrapper;
+import org.apache.carbondata.core.datamap.dev.Index;
+import org.apache.carbondata.core.datamap.dev.expr.IndexExprWrapper;
+import org.apache.carbondata.core.datamap.dev.expr.IndexInputSplitWrapper;
 import org.apache.carbondata.core.datastore.impl.FileFactory;
 import org.apache.carbondata.core.indexstore.ExtendedBlocklet;
 import org.apache.carbondata.core.indexstore.PartitionSpec;
@@ -39,7 +39,7 @@ import org.apache.carbondata.core.metadata.schema.table.CarbonTable;
 import org.apache.carbondata.core.metadata.schema.table.RelationIdentifier;
 import org.apache.carbondata.core.scan.filter.resolver.FilterResolverIntf;
 import org.apache.carbondata.core.statusmanager.SegmentStatusManager;
-import org.apache.carbondata.core.util.BlockletDataMapUtil;
+import org.apache.carbondata.core.util.BlockletIndexUtil;
 import org.apache.carbondata.core.util.CarbonProperties;
 import org.apache.carbondata.core.util.ObjectSerializationUtil;
 import org.apache.carbondata.core.util.path.CarbonTablePath;
@@ -48,21 +48,21 @@ import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.mapreduce.lib.input.FileInputFormat;
 import org.apache.log4j.Logger;
 
-public class DataMapUtil {
+public class IndexUtil {
 
   private static final String DATA_MAP_DSTR = "mapreduce.input.carboninputformat.datamapdstr";
 
   public static final String EMBEDDED_JOB_NAME =
-      "org.apache.carbondata.indexserver.EmbeddedDataMapJob";
+      "org.apache.carbondata.indexserver.EmbeddedIndexJob";
 
   public static final String DISTRIBUTED_JOB_NAME =
-      "org.apache.carbondata.indexserver.DistributedDataMapJob";
+      "org.apache.carbondata.indexserver.DistributedIndexJob";
 
   private static final Logger LOGGER =
-      LogServiceFactory.getLogService(DataMapUtil.class.getName());
+      LogServiceFactory.getLogService(IndexUtil.class.getName());
 
   /**
-   * Creates instance for the DataMap Job class
+   * Creates instance for the Index Job class
    *
    * @param className
    * @return
@@ -93,13 +93,13 @@ public class DataMapUtil {
   /**
    * get datamap job from the configuration
    * @param configuration job configuration
-   * @return DataMap Job
+   * @return Index Job
    * @throws IOException
    */
-  public static DataMapJob getDataMapJob(Configuration configuration) throws IOException {
+  public static IndexJob getDataMapJob(Configuration configuration) throws IOException {
     String jobString = configuration.get(DATA_MAP_DSTR);
     if (jobString != null) {
-      return (DataMapJob) ObjectSerializationUtil.convertStringToObject(jobString);
+      return (IndexJob) ObjectSerializationUtil.convertStringToObject(jobString);
     }
     return null;
   }
@@ -111,7 +111,7 @@ public class DataMapUtil {
    * @param carbonTable
    * @throws IOException
    */
-  private static void executeClearDataMapJob(DataMapJob dataMapJob,
+  private static void executeClearIndexJob(IndexJob indexJob,
       CarbonTable carbonTable, String dataMapToClear) throws IOException {
     SegmentStatusManager.ValidAndInvalidSegmentsInfo validAndInvalidSegmentsInfo =
             getValidAndInvalidSegments(carbonTable, FileFactory.getConfiguration());
@@ -119,11 +119,11 @@ public class DataMapUtil {
     for (Segment segment : validAndInvalidSegmentsInfo.getInvalidSegments()) {
       invalidSegment.add(segment.getSegmentNo());
     }
-    DistributableDataMapFormat dataMapFormat =
-        new DistributableDataMapFormat(carbonTable, validAndInvalidSegmentsInfo.getValidSegments(),
+    IndexInputFormat indexInputFormat =
+        new IndexInputFormat(carbonTable, validAndInvalidSegmentsInfo.getValidSegments(),
             invalidSegment, true, dataMapToClear);
     try {
-      dataMapJob.execute(dataMapFormat);
+      indexJob.execute(indexInputFormat);
     } catch (Exception e) {
       // Consider a scenario where clear datamap job is called from drop table
       // and index server crashes, in this no exception should be thrown and
@@ -132,26 +132,26 @@ public class DataMapUtil {
     }
   }
 
-  public static void executeClearDataMapJob(CarbonTable carbonTable, String jobClassName)
+  public static void executeClearIndexJob(CarbonTable carbonTable, String jobClassName)
       throws IOException {
-    executeClearDataMapJob(carbonTable, jobClassName, "");
+    executeClearIndexJob(carbonTable, jobClassName, "");
   }
 
-  static void executeClearDataMapJob(CarbonTable carbonTable, String jobClassName,
+  static void executeClearIndexJob(CarbonTable carbonTable, String jobClassName,
       String dataMapToClear) throws IOException {
-    DataMapJob dataMapJob = (DataMapJob) createDataMapJob(jobClassName);
-    if (dataMapJob == null) {
+    IndexJob indexJob = (IndexJob) createDataMapJob(jobClassName);
+    if (indexJob == null) {
       return;
     }
-    executeClearDataMapJob(dataMapJob, carbonTable, dataMapToClear);
+    executeClearIndexJob(indexJob, carbonTable, dataMapToClear);
   }
 
-  public static DataMapJob getEmbeddedJob() {
-    DataMapJob dataMapJob = (DataMapJob) DataMapUtil.createDataMapJob(EMBEDDED_JOB_NAME);
-    if (dataMapJob == null) {
+  public static IndexJob getEmbeddedJob() {
+    IndexJob indexJob = (IndexJob) IndexUtil.createDataMapJob(EMBEDDED_JOB_NAME);
+    if (indexJob == null) {
       throw new ExceptionInInitializerError("Unable to create EmbeddedDataMapJob");
     }
-    return dataMapJob;
+    return indexJob;
   }
 
   /**
@@ -185,38 +185,37 @@ public class DataMapUtil {
    Loads the datamaps in parallel by utilizing executor
    *
    @param carbonTable
-   @param dataMapExprWrapper
+   @param indexExprWrapper
    @param validSegments
    @param partitionsToPrune
    @throws IOException
    */
-  public static void loadDataMaps(CarbonTable carbonTable, DataMapExprWrapper dataMapExprWrapper,
+  public static void loadDataMaps(CarbonTable carbonTable, IndexExprWrapper indexExprWrapper,
       List<Segment> validSegments, List<PartitionSpec> partitionsToPrune) throws IOException {
     if (!CarbonProperties.getInstance()
         .isDistributedPruningEnabled(carbonTable.getDatabaseName(), carbonTable.getTableName())
-        && BlockletDataMapUtil.loadDataMapsParallel(carbonTable)) {
-      String clsName = "org.apache.spark.sql.secondaryindex.Jobs.SparkBlockletDataMapLoaderJob";
-      DataMapJob dataMapJob = (DataMapJob) createDataMapJob(clsName);
-      String className =
-          "org.apache.spark.sql.secondaryindex.Jobs.DistributableBlockletDataMapLoader";
+        && BlockletIndexUtil.loadDataMapsParallel(carbonTable)) {
+      String clsName = "org.apache.spark.sql.secondaryindex.Jobs.SparkBlockletIndexLoaderJob";
+      IndexJob indexJob = (IndexJob) createDataMapJob(clsName);
+      String className = "org.apache.spark.sql.secondaryindex.Jobs.BlockletIndexInputFormat";
       SegmentStatusManager.ValidAndInvalidSegmentsInfo validAndInvalidSegmentsInfo =
           getValidAndInvalidSegments(carbonTable, FileFactory.getConfiguration());
       List<Segment> invalidSegments = validAndInvalidSegmentsInfo.getInvalidSegments();
       FileInputFormat dataMapFormat =
-          createDataMapJob(carbonTable, dataMapExprWrapper, validSegments, invalidSegments,
+          createDataMapJob(carbonTable, indexExprWrapper, validSegments, invalidSegments,
               partitionsToPrune, className, false);
-      dataMapJob.execute(carbonTable, dataMapFormat);
+      indexJob.execute(carbonTable, dataMapFormat);
     }
   }
 
   private static FileInputFormat createDataMapJob(CarbonTable carbonTable,
-      DataMapExprWrapper dataMapExprWrapper, List<Segment> validsegments,
+      IndexExprWrapper indexExprWrapper, List<Segment> validsegments,
       List<Segment> invalidSegments, List<PartitionSpec> partitionsToPrune, String clsName,
       boolean isJobToClearDataMaps) {
     try {
       Constructor<?> cons = Class.forName(clsName).getDeclaredConstructors()[0];
       return (FileInputFormat) cons
-          .newInstance(carbonTable, dataMapExprWrapper, validsegments, invalidSegments,
+          .newInstance(carbonTable, indexExprWrapper, validsegments, invalidSegments,
               partitionsToPrune, isJobToClearDataMaps);
     } catch (Exception e) {
       throw new RuntimeException(e);
@@ -226,42 +225,42 @@ public class DataMapUtil {
   static List<ExtendedBlocklet> pruneDataMaps(CarbonTable table,
       FilterResolverIntf filterResolverIntf, List<Segment> segmentsToLoad,
       List<PartitionSpec> partitions, List<ExtendedBlocklet> blocklets,
-      DataMapChooser dataMapChooser) throws IOException {
-    if (null == dataMapChooser) {
+      IndexChooser indexChooser) throws IOException {
+    if (null == indexChooser) {
       return blocklets;
     }
     pruneSegments(segmentsToLoad, blocklets);
     List<ExtendedBlocklet> cgDataMaps = pruneDataMaps(table, filterResolverIntf, segmentsToLoad,
         partitions, blocklets,
-        DataMapLevel.CG, dataMapChooser);
+        IndexLevel.CG, indexChooser);
     pruneSegments(segmentsToLoad, cgDataMaps);
     return pruneDataMaps(table, filterResolverIntf, segmentsToLoad,
         partitions, cgDataMaps,
-        DataMapLevel.FG, dataMapChooser);
+        IndexLevel.FG, indexChooser);
   }
 
   static List<ExtendedBlocklet> pruneDataMaps(CarbonTable table,
       FilterResolverIntf filterResolverIntf, List<Segment> segmentsToLoad,
-      List<PartitionSpec> partitions, List<ExtendedBlocklet> blocklets, DataMapLevel dataMapLevel,
-      DataMapChooser dataMapChooser)
+      List<PartitionSpec> partitions, List<ExtendedBlocklet> blocklets, IndexLevel indexLevel,
+      IndexChooser indexChooser)
       throws IOException {
-    DataMapExprWrapper dataMapExprWrapper =
-        dataMapChooser.chooseDataMap(dataMapLevel, filterResolverIntf);
-    if (dataMapExprWrapper != null) {
+    IndexExprWrapper indexExprWrapper =
+        indexChooser.chooseDataMap(indexLevel, filterResolverIntf);
+    if (indexExprWrapper != null) {
       List<ExtendedBlocklet> extendedBlocklets = new ArrayList<>();
       // Prune segments from already pruned blocklets
-      for (DataMapDistributableWrapper wrapper : dataMapExprWrapper
+      for (IndexInputSplitWrapper wrapper : indexExprWrapper
           .toDistributable(segmentsToLoad)) {
-        TableDataMap dataMap = DataMapStoreManager.getInstance()
-            .getDataMap(table, wrapper.getDistributable().getDataMapSchema());
-        List<DataMap> dataMaps = dataMap.getTableDataMaps(wrapper.getDistributable());
+        TableIndex dataMap = DataMapStoreManager.getInstance()
+            .getIndex(table, wrapper.getDistributable().getDataMapSchema());
+        List<Index> indices = dataMap.getTableDataMaps(wrapper.getDistributable());
         List<ExtendedBlocklet> prunnedBlocklet = new ArrayList<>();
         if (table.isTransactionalTable()) {
-          prunnedBlocklet.addAll(dataMap.prune(dataMaps, wrapper.getDistributable(),
-              dataMapExprWrapper.getFilterResolverIntf(wrapper.getUniqueId()), partitions));
+          prunnedBlocklet.addAll(dataMap.prune(indices, wrapper.getDistributable(),
+              indexExprWrapper.getFilterResolverIntf(wrapper.getUniqueId()), partitions));
         } else {
           prunnedBlocklet
-              .addAll(dataMap.prune(segmentsToLoad, new DataMapFilter(filterResolverIntf),
+              .addAll(dataMap.prune(segmentsToLoad, new IndexFilter(filterResolverIntf),
                   partitions));
         }
         // For all blocklets initialize the detail info so that it can be serialized to the driver.
@@ -271,7 +270,7 @@ public class DataMapUtil {
         }
         extendedBlocklets.addAll(prunnedBlocklet);
       }
-      return dataMapExprWrapper.pruneBlocklets(extendedBlocklets);
+      return indexExprWrapper.pruneBlocklets(extendedBlocklets);
     }
     return blocklets;
   }
@@ -282,10 +281,10 @@ public class DataMapUtil {
    * @return list of Extended blocklets after pruning
    */
   public static List<ExtendedBlocklet> executeDataMapJob(CarbonTable carbonTable,
-      FilterResolverIntf resolver, DataMapJob dataMapJob, List<PartitionSpec> partitionsToPrune,
-      List<Segment> validSegments, List<Segment> invalidSegments, DataMapLevel level,
+      FilterResolverIntf resolver, IndexJob indexJob, List<PartitionSpec> partitionsToPrune,
+      List<Segment> validSegments, List<Segment> invalidSegments, IndexLevel level,
       List<String> segmentsToBeRefreshed) {
-    return executeDataMapJob(carbonTable, resolver, dataMapJob, partitionsToPrune, validSegments,
+    return executeDataMapJob(carbonTable, resolver, indexJob, partitionsToPrune, validSegments,
         invalidSegments, level, false, segmentsToBeRefreshed, false);
   }
 
@@ -295,29 +294,29 @@ public class DataMapUtil {
    * @return list of Extended blocklets after pruning
    */
   public static List<ExtendedBlocklet> executeDataMapJob(CarbonTable carbonTable,
-      FilterResolverIntf resolver, DataMapJob dataMapJob, List<PartitionSpec> partitionsToPrune,
-      List<Segment> validSegments, List<Segment> invalidSegments, DataMapLevel level,
+      FilterResolverIntf resolver, IndexJob indexJob, List<PartitionSpec> partitionsToPrune,
+      List<Segment> validSegments, List<Segment> invalidSegments, IndexLevel level,
       Boolean isFallbackJob, List<String> segmentsToBeRefreshed, boolean isCountJob) {
     List<String> invalidSegmentNo = new ArrayList<>();
     for (Segment segment : invalidSegments) {
       invalidSegmentNo.add(segment.getSegmentNo());
     }
     invalidSegmentNo.addAll(segmentsToBeRefreshed);
-    DistributableDataMapFormat dataMapFormat =
-        new DistributableDataMapFormat(carbonTable, resolver, validSegments, invalidSegmentNo,
+    IndexInputFormat dataMapFormat =
+        new IndexInputFormat(carbonTable, resolver, validSegments, invalidSegmentNo,
             partitionsToPrune, false, level, isFallbackJob, false);
     if (isCountJob) {
       dataMapFormat.setCountStarJob();
       dataMapFormat.setIsWriteToFile(false);
     }
-    return dataMapJob.execute(dataMapFormat);
+    return indexJob.execute(dataMapFormat);
   }
 
   public static SegmentStatusManager.ValidAndInvalidSegmentsInfo getValidAndInvalidSegments(
       CarbonTable carbonTable, Configuration configuration) throws IOException {
     SegmentStatusManager ssm =
         new SegmentStatusManager(carbonTable.getAbsoluteTableIdentifier(), configuration);
-    return ssm.getValidAndInvalidSegments(carbonTable.isChildTableForMV());
+    return ssm.getValidAndInvalidSegments(carbonTable.isMV());
   }
 
   /**
diff --git a/core/src/main/java/org/apache/carbondata/core/datamap/DataMapCatalog.java b/core/src/main/java/org/apache/carbondata/core/datamap/MVCatalog.java
similarity index 93%
copy from core/src/main/java/org/apache/carbondata/core/datamap/DataMapCatalog.java
copy to core/src/main/java/org/apache/carbondata/core/datamap/MVCatalog.java
index d4eb0c3..461fb6e 100644
--- a/core/src/main/java/org/apache/carbondata/core/datamap/DataMapCatalog.java
+++ b/core/src/main/java/org/apache/carbondata/core/datamap/MVCatalog.java
@@ -20,10 +20,10 @@ package org.apache.carbondata.core.datamap;
 import org.apache.carbondata.core.metadata.schema.table.DataMapSchema;
 
 /**
- * This is the interface for inmemory catalog registry for datamap.
+ * This is the interface for inmemory catalog registry for MV.
  * @since 1.4.0
  */
-public interface DataMapCatalog<T> {
+public interface MVCatalog<T> {
 
   /**
    * Register schema to the catalog.
diff --git a/core/src/main/java/org/apache/carbondata/core/datamap/SegmentDataMapGroup.java b/core/src/main/java/org/apache/carbondata/core/datamap/SegmentIndexGroup.java
similarity index 80%
rename from core/src/main/java/org/apache/carbondata/core/datamap/SegmentDataMapGroup.java
rename to core/src/main/java/org/apache/carbondata/core/datamap/SegmentIndexGroup.java
index 0e0e774..fa25f17 100644
--- a/core/src/main/java/org/apache/carbondata/core/datamap/SegmentDataMapGroup.java
+++ b/core/src/main/java/org/apache/carbondata/core/datamap/SegmentIndexGroup.java
@@ -20,10 +20,10 @@ package org.apache.carbondata.core.datamap;
 import java.io.Serializable;
 
 /**
- * Wrapper that have a group of selected datamap for segment.
- * this have information about from which index to which index need to process the datamap.
+ * Wrapper that have a group of selected index for segment.
+ * this have information about from which index need to process.
  */
-public class SegmentDataMapGroup implements Serializable {
+public class SegmentIndexGroup implements Serializable {
 
   private Segment segment;
 
@@ -31,7 +31,7 @@ public class SegmentDataMapGroup implements Serializable {
 
   private int toIndex;
 
-  public SegmentDataMapGroup(Segment segment, int fromIndex, int toIndex) {
+  public SegmentIndexGroup(Segment segment, int fromIndex, int toIndex) {
     this.segment = segment;
     this.fromIndex = fromIndex;
     this.toIndex = toIndex;
diff --git a/core/src/main/java/org/apache/carbondata/core/datamap/TableDataMap.java b/core/src/main/java/org/apache/carbondata/core/datamap/TableIndex.java
similarity index 78%
rename from core/src/main/java/org/apache/carbondata/core/datamap/TableDataMap.java
rename to core/src/main/java/org/apache/carbondata/core/datamap/TableIndex.java
index 8d680b9..edbdee0 100644
--- a/core/src/main/java/org/apache/carbondata/core/datamap/TableDataMap.java
+++ b/core/src/main/java/org/apache/carbondata/core/datamap/TableIndex.java
@@ -34,10 +34,10 @@ import org.apache.carbondata.common.annotations.InterfaceAudience;
 import org.apache.carbondata.common.logging.LogServiceFactory;
 import org.apache.carbondata.core.constants.CarbonCommonConstants;
 import org.apache.carbondata.core.datamap.dev.BlockletSerializer;
-import org.apache.carbondata.core.datamap.dev.DataMap;
-import org.apache.carbondata.core.datamap.dev.DataMapFactory;
-import org.apache.carbondata.core.datamap.dev.cgdatamap.CoarseGrainDataMap;
-import org.apache.carbondata.core.datamap.dev.expr.DataMapDistributableWrapper;
+import org.apache.carbondata.core.datamap.dev.Index;
+import org.apache.carbondata.core.datamap.dev.IndexFactory;
+import org.apache.carbondata.core.datamap.dev.cgdatamap.CoarseGrainIndex;
+import org.apache.carbondata.core.datamap.dev.expr.IndexInputSplitWrapper;
 import org.apache.carbondata.core.datamap.dev.fgdatamap.FineGrainBlocklet;
 import org.apache.carbondata.core.datastore.block.SegmentProperties;
 import org.apache.carbondata.core.datastore.impl.FileFactory;
@@ -61,15 +61,15 @@ import org.apache.carbondata.events.OperationEventListener;
 import org.apache.log4j.Logger;
 
 /**
- * Index at the table level, user can add any number of DataMap for one table, by
+ * Index at the table level, user can add any number of Index for one table, by
  * {@code
- *   CREATE DATAMAP dm ON TABLE table
- *   USING 'class name of DataMapFactory implementation'
+ *   CREATE INDEX index ON table
+ *   USING 'IndexProvider'
  * }
  * Depends on the filter condition it can prune the data (blocklet or row level).
  */
 @InterfaceAudience.Internal
-public final class TableDataMap extends OperationEventListener {
+public final class TableIndex extends OperationEventListener {
 
   private CarbonTable table;
 
@@ -77,25 +77,25 @@ public final class TableDataMap extends OperationEventListener {
 
   private DataMapSchema dataMapSchema;
 
-  private DataMapFactory dataMapFactory;
+  private IndexFactory indexFactory;
 
   private BlockletDetailsFetcher blockletDetailsFetcher;
 
   private SegmentPropertiesFetcher segmentPropertiesFetcher;
 
   private static final Logger LOG =
-      LogServiceFactory.getLogService(TableDataMap.class.getName());
+      LogServiceFactory.getLogService(TableIndex.class.getName());
 
   /**
    * It is called to initialize and load the required table datamap metadata.
    */
-  TableDataMap(CarbonTable table, DataMapSchema dataMapSchema,
-      DataMapFactory dataMapFactory, BlockletDetailsFetcher blockletDetailsFetcher,
+  TableIndex(CarbonTable table, DataMapSchema dataMapSchema,
+      IndexFactory indexFactory, BlockletDetailsFetcher blockletDetailsFetcher,
       SegmentPropertiesFetcher segmentPropertiesFetcher) {
     this.identifier = table.getAbsoluteTableIdentifier();
     this.table = table;
     this.dataMapSchema = dataMapSchema;
-    this.dataMapFactory = dataMapFactory;
+    this.indexFactory = indexFactory;
     this.blockletDetailsFetcher = blockletDetailsFetcher;
     this.segmentPropertiesFetcher = segmentPropertiesFetcher;
   }
@@ -115,16 +115,16 @@ public final class TableDataMap extends OperationEventListener {
    * @param filter
    * @return
    */
-  public List<ExtendedBlocklet> prune(List<Segment> allsegments, final DataMapFilter filter,
+  public List<ExtendedBlocklet> prune(List<Segment> allsegments, final IndexFilter filter,
       final List<PartitionSpec> partitions) throws IOException {
     final List<ExtendedBlocklet> blocklets = new ArrayList<>();
     List<Segment> segments = getCarbonSegments(allsegments);
-    final Map<Segment, List<DataMap>> dataMaps;
+    final Map<Segment, List<Index>> dataMaps;
     boolean isFilterPresent = filter != null && !filter.isEmpty();
     if (table.isHivePartitionTable() && isFilterPresent && partitions != null) {
-      dataMaps = dataMapFactory.getDataMaps(segments, partitions, filter);
+      dataMaps = indexFactory.getIndexes(segments, partitions, filter);
     } else {
-      dataMaps = dataMapFactory.getDataMaps(segments, filter);
+      dataMaps = indexFactory.getIndexes(segments, filter);
     }
 
     if (dataMaps.isEmpty()) {
@@ -140,8 +140,8 @@ public final class TableDataMap extends OperationEventListener {
       segments = new ArrayList<>(dataMaps.keySet());
     }
     for (Segment segment : segments) {
-      for (DataMap dataMap: dataMaps.get(segment)) {
-        totalFiles += dataMap.getNumberOfEntries();
+      for (Index index : dataMaps.get(segment)) {
+        totalFiles += index.getNumberOfEntries();
         datamapsCount++;
       }
     }
@@ -187,9 +187,9 @@ public final class TableDataMap extends OperationEventListener {
     return blocklets;
   }
 
-  private List<ExtendedBlocklet> pruneWithFilter(List<Segment> segments, DataMapFilter filter,
+  private List<ExtendedBlocklet> pruneWithFilter(List<Segment> segments, IndexFilter filter,
       List<PartitionSpec> partitions, List<ExtendedBlocklet> blocklets,
-      Map<Segment, List<DataMap>> dataMaps) throws IOException {
+      Map<Segment, List<Index>> dataMaps) throws IOException {
     for (Segment segment : segments) {
       if (dataMaps.get(segment).isEmpty() || dataMaps.get(segment) == null) {
         continue;
@@ -209,13 +209,13 @@ public final class TableDataMap extends OperationEventListener {
               .getFilterExecuterTree(filter.getExternalSegmentResolver(), segmentProperties, null,
                   table.getMinMaxCacheColumns(segmentProperties), false);
         }
-        for (DataMap dataMap : dataMaps.get(segment)) {
+        for (Index index : dataMaps.get(segment)) {
           if (!isExternalSegment) {
-            pruneBlocklets.addAll(dataMap
+            pruneBlocklets.addAll(index
                 .prune(filter.getResolver(), segmentProperties, partitions, filterExecuter,
                     this.table));
           } else {
-            pruneBlocklets.addAll(dataMap
+            pruneBlocklets.addAll(index
                 .prune(filter.getExternalSegmentResolver(), segmentProperties, partitions,
                     filterExecuter, this.table));
           }
@@ -225,20 +225,20 @@ public final class TableDataMap extends OperationEventListener {
         Expression expression = filter.getExpression();
         if (!isExternalSegment) {
           filterExecuter = FilterUtil.getFilterExecuterTree(
-              new DataMapFilter(segmentProperties, table, expression).getResolver(),
+              new IndexFilter(segmentProperties, table, expression).getResolver(),
               segmentProperties, null, table.getMinMaxCacheColumns(segmentProperties), false);
         } else {
           filterExecuter = FilterUtil.getFilterExecuterTree(
-              new DataMapFilter(segmentProperties, table, expression).getExternalSegmentResolver(),
+              new IndexFilter(segmentProperties, table, expression).getExternalSegmentResolver(),
               segmentProperties, null, table.getMinMaxCacheColumns(segmentProperties), false);
         }
-        for (DataMap dataMap : dataMaps.get(segment)) {
+        for (Index index : dataMaps.get(segment)) {
           if (!isExternalSegment) {
-            pruneBlocklets.addAll(dataMap
+            pruneBlocklets.addAll(index
                 .prune(filter.getExpression(), segmentProperties, partitions, table,
                     filterExecuter));
           } else {
-            pruneBlocklets.addAll(dataMap
+            pruneBlocklets.addAll(index
                 .prune(filter.getExternalSegmentFilter(), segmentProperties, partitions, table,
                     filterExecuter));
           }
@@ -252,8 +252,8 @@ public final class TableDataMap extends OperationEventListener {
   }
 
   private List<ExtendedBlocklet> pruneMultiThread(List<Segment> segments,
-      final DataMapFilter filter, final List<PartitionSpec> partitions,
-      List<ExtendedBlocklet> blocklets, final Map<Segment, List<DataMap>> dataMaps,
+      final IndexFilter filter, final List<PartitionSpec> partitions,
+      List<ExtendedBlocklet> blocklets, final Map<Segment, List<Index>> dataMaps,
       int totalFiles) {
     /*
      *********************************************************************************
@@ -266,7 +266,7 @@ public final class TableDataMap extends OperationEventListener {
      *
      * If number of threads are 4. so filesPerEachThread = 50/4 = 12 files per each thread.
      *
-     * SegmentDataMapGroup look like below: [SegmentId, fromIndex, toIndex]
+     * SegmentIndexGroup look like below: [SegmentId, fromIndex, toIndex]
      * In each segment only those datamaps are processed between fromIndex and toIndex.
      *
      * Final result will be: (4 list created as numOfThreadsForPruning is 4)
@@ -284,23 +284,23 @@ public final class TableDataMap extends OperationEventListener {
     int prev;
     int filesCount = 0;
     int processedFileCount = 0;
-    List<List<SegmentDataMapGroup>> datamapListForEachThread =
+    List<List<SegmentIndexGroup>> datamapListForEachThread =
         new ArrayList<>(numOfThreadsForPruning);
-    List<SegmentDataMapGroup> segmentDataMapGroupList = new ArrayList<>();
+    List<SegmentIndexGroup> segmentIndexGroupList = new ArrayList<>();
     for (Segment segment : segments) {
-      List<DataMap> eachSegmentDataMapList = dataMaps.get(segment);
+      List<Index> eachSegmentIndexList = dataMaps.get(segment);
       prev = 0;
-      for (int i = 0; i < eachSegmentDataMapList.size(); i++) {
-        DataMap dataMap = eachSegmentDataMapList.get(i);
-        filesCount += dataMap.getNumberOfEntries();
+      for (int i = 0; i < eachSegmentIndexList.size(); i++) {
+        Index index = eachSegmentIndexList.get(i);
+        filesCount += index.getNumberOfEntries();
         if (filesCount >= filesPerEachThread) {
           if (datamapListForEachThread.size() != numOfThreadsForPruning - 1) {
             // not the last segmentList
-            segmentDataMapGroupList.add(new SegmentDataMapGroup(segment, prev, i));
+            segmentIndexGroupList.add(new SegmentIndexGroup(segment, prev, i));
             // save the last value to process in next thread
             prev = i + 1;
-            datamapListForEachThread.add(segmentDataMapGroupList);
-            segmentDataMapGroupList = new ArrayList<>();
+            datamapListForEachThread.add(segmentIndexGroupList);
+            segmentIndexGroupList = new ArrayList<>();
             processedFileCount += filesCount;
             filesCount = 0;
           } else {
@@ -310,15 +310,15 @@ public final class TableDataMap extends OperationEventListener {
           }
         }
       }
-      if (prev == 0 || prev != eachSegmentDataMapList.size()) {
+      if (prev == 0 || prev != eachSegmentIndexList.size()) {
         // if prev == 0. Add a segment's all datamaps
-        // eachSegmentDataMapList.size() != prev, adding the last remaining datamaps of this segment
-        segmentDataMapGroupList
-            .add(new SegmentDataMapGroup(segment, prev, eachSegmentDataMapList.size() - 1));
+        // eachSegmentIndexList.size() != prev, adding the last remaining datamaps of this segment
+        segmentIndexGroupList
+            .add(new SegmentIndexGroup(segment, prev, eachSegmentIndexList.size() - 1));
       }
     }
     // adding the last segmentList data
-    datamapListForEachThread.add(segmentDataMapGroupList);
+    datamapListForEachThread.add(segmentIndexGroupList);
     processedFileCount += filesCount;
     if (processedFileCount != totalFiles) {
       // this should not happen
@@ -339,17 +339,17 @@ public final class TableDataMap extends OperationEventListener {
     final ExecutorService executorService = Executors.newFixedThreadPool(numOfThreadsForPruning);
     final String threadName = Thread.currentThread().getName();
     for (int i = 0; i < numOfThreadsForPruning; i++) {
-      final List<SegmentDataMapGroup> segmentDataMapGroups = datamapListForEachThread.get(i);
+      final List<SegmentIndexGroup> segmentIndexGroups = datamapListForEachThread.get(i);
       results.add(executorService.submit(new Callable<Void>() {
         @Override
         public Void call() throws IOException {
           Thread.currentThread().setName(threadName);
-          for (SegmentDataMapGroup segmentDataMapGroup : segmentDataMapGroups) {
+          for (SegmentIndexGroup segmentIndexGroup : segmentIndexGroups) {
             List<ExtendedBlocklet> pruneBlocklets = new ArrayList<>();
-            List<DataMap> dataMapList = dataMaps.get(segmentDataMapGroup.getSegment());
+            List<Index> indexList = dataMaps.get(segmentIndexGroup.getSegment());
             SegmentProperties segmentProperties =
-                segmentPropertiesFetcher.getSegmentPropertiesFromDataMap(dataMapList.get(0));
-            Segment segment = segmentDataMapGroup.getSegment();
+                segmentPropertiesFetcher.getSegmentPropertiesFromDataMap(indexList.get(0));
+            Segment segment = segmentIndexGroup.getSegment();
             boolean isExternalSegment = segment.getSegmentPath() != null;
             if (filter.isResolvedOnSegment(segmentProperties)) {
               FilterExecuter filterExecuter;
@@ -362,15 +362,15 @@ public final class TableDataMap extends OperationEventListener {
                     .getFilterExecuterTree(filter.getExternalSegmentResolver(), segmentProperties,
                         null, table.getMinMaxCacheColumns(segmentProperties), false);
               }
-              for (int i = segmentDataMapGroup.getFromIndex();
-                   i <= segmentDataMapGroup.getToIndex(); i++) {
+              for (int i = segmentIndexGroup.getFromIndex();
+                   i <= segmentIndexGroup.getToIndex(); i++) {
                 List<Blocklet> dmPruneBlocklets;
                 if (!isExternalSegment) {
-                  dmPruneBlocklets = dataMapList.get(i)
+                  dmPruneBlocklets = indexList.get(i)
                       .prune(filter.getResolver(), segmentProperties, partitions, filterExecuter,
                           table);
                 } else {
-                  dmPruneBlocklets = dataMapList.get(i)
+                  dmPruneBlocklets = indexList.get(i)
                       .prune(filter.getExternalSegmentResolver(), segmentProperties, partitions,
                           filterExecuter, table);
                 }
@@ -383,23 +383,23 @@ public final class TableDataMap extends OperationEventListener {
               FilterExecuter filterExecuter;
               if (!isExternalSegment) {
                 filterExecuter = FilterUtil.getFilterExecuterTree(
-                    new DataMapFilter(segmentProperties, table, filterExpression).getResolver(),
+                    new IndexFilter(segmentProperties, table, filterExpression).getResolver(),
                     segmentProperties, null, table.getMinMaxCacheColumns(segmentProperties), false);
               } else {
                 filterExecuter = FilterUtil.getFilterExecuterTree(
-                    new DataMapFilter(segmentProperties, table, filterExpression)
+                    new IndexFilter(segmentProperties, table, filterExpression)
                         .getExternalSegmentResolver(), segmentProperties, null,
                     table.getMinMaxCacheColumns(segmentProperties), false);
               }
-              for (int i = segmentDataMapGroup.getFromIndex();
-                   i <= segmentDataMapGroup.getToIndex(); i++) {
+              for (int i = segmentIndexGroup.getFromIndex();
+                   i <= segmentIndexGroup.getToIndex(); i++) {
                 List<Blocklet> dmPruneBlocklets;
                 if (!isExternalSegment) {
-                  dmPruneBlocklets = dataMapList.get(i)
+                  dmPruneBlocklets = indexList.get(i)
                       .prune(filterExpression, segmentProperties, partitions, table,
                           filterExecuter);
                 } else {
-                  dmPruneBlocklets = dataMapList.get(i)
+                  dmPruneBlocklets = indexList.get(i)
                       .prune(filter.getExternalSegmentFilter(), segmentProperties, partitions,
                           table, filterExecuter);
                 }
@@ -410,11 +410,11 @@ public final class TableDataMap extends OperationEventListener {
             }
             synchronized (prunedBlockletMap) {
               List<ExtendedBlocklet> pruneBlockletsExisting =
-                  prunedBlockletMap.get(segmentDataMapGroup.getSegment());
+                  prunedBlockletMap.get(segmentIndexGroup.getSegment());
               if (pruneBlockletsExisting != null) {
                 pruneBlockletsExisting.addAll(pruneBlocklets);
               } else {
-                prunedBlockletMap.put(segmentDataMapGroup.getSegment(), pruneBlocklets);
+                prunedBlockletMap.put(segmentIndexGroup.getSegment(), pruneBlocklets);
               }
             }
           }
@@ -457,18 +457,18 @@ public final class TableDataMap extends OperationEventListener {
    *
    * @return
    */
-  public List<DataMapDistributable> toDistributable(List<Segment> allsegments) {
-    List<DataMapDistributable> distributables = new ArrayList<>();
+  public List<IndexInputSplit> toDistributable(List<Segment> allsegments) {
+    List<IndexInputSplit> distributables = new ArrayList<>();
     List<Segment> segments = getCarbonSegments(allsegments);
     for (Segment segment : segments) {
-      distributables.addAll(dataMapFactory.toDistributable(segment));
+      distributables.addAll(indexFactory.toDistributable(segment));
     }
     return distributables;
   }
 
-  public DataMapDistributableWrapper toDistributableSegment(Segment segment, String uniqueId)
+  public IndexInputSplitWrapper toDistributableSegment(Segment segment, String uniqueId)
       throws IOException {
-    return dataMapFactory.toDistributableSegment(segment, dataMapSchema, identifier, uniqueId);
+    return indexFactory.toDistributableSegment(segment, dataMapSchema, identifier, uniqueId);
   }
 
   /**
@@ -478,8 +478,8 @@ public final class TableDataMap extends OperationEventListener {
    * @return
    * @throws IOException
    */
-  public List<DataMap> getTableDataMaps(DataMapDistributable distributable) throws IOException {
-    return dataMapFactory.getDataMaps(distributable);
+  public List<Index> getTableDataMaps(IndexInputSplit distributable) throws IOException {
+    return indexFactory.getIndexes(distributable);
   }
 
   /**
@@ -490,7 +490,7 @@ public final class TableDataMap extends OperationEventListener {
    * @param filterExp
    * @return
    */
-  public List<ExtendedBlocklet> prune(List<DataMap> dataMaps, DataMapDistributable distributable,
+  public List<ExtendedBlocklet> prune(List<Index> indices, IndexInputSplit distributable,
       FilterResolverIntf filterExp, List<PartitionSpec> partitions) throws IOException {
     List<ExtendedBlocklet> detailedBlocklets = new ArrayList<>();
     List<Blocklet> blocklets = new ArrayList<>();
@@ -500,21 +500,21 @@ public final class TableDataMap extends OperationEventListener {
         .getFilterExecuterTree(filterExp, segmentProperties,
             null, table.getMinMaxCacheColumns(segmentProperties),
             false);
-    for (DataMap dataMap : dataMaps) {
+    for (Index index : indices) {
       blocklets
-          .addAll(dataMap.prune(filterExp, segmentProperties, partitions, filterExecuter, table));
+          .addAll(index.prune(filterExp, segmentProperties, partitions, filterExecuter, table));
     }
     BlockletSerializer serializer = new BlockletSerializer();
     String writePath =
         identifier.getTablePath() + CarbonCommonConstants.FILE_SEPARATOR + dataMapSchema
             .getDataMapName();
-    if (dataMapFactory.getDataMapLevel() == DataMapLevel.FG) {
+    if (indexFactory.getDataMapLevel() == IndexLevel.FG) {
       FileFactory.mkdirs(writePath);
     }
     for (Blocklet blocklet : blocklets) {
       ExtendedBlocklet detailedBlocklet = blockletDetailsFetcher
           .getExtendedBlocklet(blocklet, distributable.getSegment());
-      if (dataMapFactory.getDataMapLevel() == DataMapLevel.FG) {
+      if (indexFactory.getDataMapLevel() == IndexLevel.FG) {
         String blockletwritePath =
             writePath + CarbonCommonConstants.FILE_SEPARATOR + System.nanoTime();
         detailedBlocklet.setDataMapWriterPath(blockletwritePath);
@@ -532,7 +532,7 @@ public final class TableDataMap extends OperationEventListener {
    */
   public void clear(List<String> segmentIds) {
     for (String segment: segmentIds) {
-      dataMapFactory.clear(segment);
+      indexFactory.clear(segment);
     }
   }
 
@@ -540,8 +540,8 @@ public final class TableDataMap extends OperationEventListener {
    * Clears all datamap
    */
   public void clear() {
-    if (null != dataMapFactory) {
-      dataMapFactory.clear();
+    if (null != indexFactory) {
+      indexFactory.clear();
     }
   }
 
@@ -551,7 +551,7 @@ public final class TableDataMap extends OperationEventListener {
   public void deleteDatamapData(List<Segment> allsegments) throws IOException {
     List<Segment> segments = getCarbonSegments(allsegments);
     for (Segment segment: segments) {
-      dataMapFactory.deleteDatamapData(segment);
+      indexFactory.deleteIndexData(segment);
     }
   }
 
@@ -559,27 +559,27 @@ public final class TableDataMap extends OperationEventListener {
    * delete datamap data if any
    */
   public void deleteDatamapData() {
-    dataMapFactory.deleteDatamapData();
+    indexFactory.deleteIndexData();
   }
 
   /**
    * delete datamap data for a segment if any
    */
   public void deleteSegmentDatamapData(String segmentNo) throws IOException {
-    dataMapFactory.deleteSegmentDatamapData(segmentNo);
+    indexFactory.deleteSegmentIndexData(segmentNo);
   }
 
   public DataMapSchema getDataMapSchema() {
     return dataMapSchema;
   }
 
-  public DataMapFactory getDataMapFactory() {
-    return dataMapFactory;
+  public IndexFactory getIndexFactory() {
+    return indexFactory;
   }
 
   @Override
   public void onEvent(Event event, OperationContext opContext) {
-    dataMapFactory.fireEvent(event);
+    indexFactory.fireEvent(event);
   }
 
   /**
@@ -591,13 +591,13 @@ public final class TableDataMap extends OperationEventListener {
    * @throws IOException
    */
   public Map<String, Long> getBlockRowCount(List<Segment> allsegments,
-      final List<PartitionSpec> partitions, TableDataMap defaultDataMap)
+      final List<PartitionSpec> partitions, TableIndex defaultDataMap)
       throws IOException {
     List<Segment> segments = getCarbonSegments(allsegments);
     Map<String, Long> blockletToRowCountMap = new HashMap<>();
     for (Segment segment : segments) {
-      List<CoarseGrainDataMap> dataMaps = defaultDataMap.getDataMapFactory().getDataMaps(segment);
-      for (CoarseGrainDataMap dataMap : dataMaps) {
+      List<CoarseGrainIndex> dataMaps = defaultDataMap.getIndexFactory().getIndexes(segment);
+      for (CoarseGrainIndex dataMap : dataMaps) {
         dataMap.getRowCountForEachBlock(segment, partitions, blockletToRowCountMap);
       }
     }
@@ -613,12 +613,12 @@ public final class TableDataMap extends OperationEventListener {
    * @throws IOException
    */
   public long getRowCount(List<Segment> allsegments, final List<PartitionSpec> partitions,
-      TableDataMap defaultDataMap) throws IOException {
+      TableIndex defaultDataMap) throws IOException {
     List<Segment> segments = getCarbonSegments(allsegments);
     long totalRowCount = 0L;
     for (Segment segment : segments) {
-      List<CoarseGrainDataMap> dataMaps = defaultDataMap.getDataMapFactory().getDataMaps(segment);
-      for (CoarseGrainDataMap dataMap : dataMaps) {
+      List<CoarseGrainIndex> dataMaps = defaultDataMap.getIndexFactory().getIndexes(segment);
+      for (CoarseGrainIndex dataMap : dataMaps) {
         totalRowCount += dataMap.getRowCount(segment, partitions);
       }
     }
@@ -633,9 +633,9 @@ public final class TableDataMap extends OperationEventListener {
       throws IOException {
     List<Segment> prunedSegments = new ArrayList<>(CarbonCommonConstants.DEFAULT_COLLECTION_SIZE);
     for (Segment segment : segments) {
-      List<DataMap> dataMaps = dataMapFactory.getDataMaps(segment);
-      for (DataMap dataMap : dataMaps) {
-        if (dataMap.isScanRequired(filterExp)) {
+      List<Index> indices = indexFactory.getIndexes(segment);
+      for (Index index : indices) {
+        if (index.isScanRequired(filterExp)) {
           // If any one task in a given segment contains the data that means the segment need to
           // be scanned and we need to validate further data maps in the same segment
           prunedSegments.add(segment);
diff --git a/core/src/main/java/org/apache/carbondata/core/datamap/dev/BlockletSerializer.java b/core/src/main/java/org/apache/carbondata/core/datamap/dev/BlockletSerializer.java
index 0d82275..65ff652 100644
--- a/core/src/main/java/org/apache/carbondata/core/datamap/dev/BlockletSerializer.java
+++ b/core/src/main/java/org/apache/carbondata/core/datamap/dev/BlockletSerializer.java
@@ -27,7 +27,7 @@ import org.apache.carbondata.core.datastore.impl.FileFactory;
 
 /**
  * A serializer/deserializer for {@link FineGrainBlocklet}, it is used after prune the data
- * by {@link org.apache.carbondata.core.datamap.dev.fgdatamap.FineGrainDataMap}
+ * by {@link FineGrainIndex}
  */
 @InterfaceAudience.Internal
 public class BlockletSerializer {
diff --git a/core/src/main/java/org/apache/carbondata/core/datamap/dev/CacheableDataMap.java b/core/src/main/java/org/apache/carbondata/core/datamap/dev/CacheableIndex.java
similarity index 69%
rename from core/src/main/java/org/apache/carbondata/core/datamap/dev/CacheableDataMap.java
rename to core/src/main/java/org/apache/carbondata/core/datamap/dev/CacheableIndex.java
index 924a26e..a67c966 100644
--- a/core/src/main/java/org/apache/carbondata/core/datamap/dev/CacheableDataMap.java
+++ b/core/src/main/java/org/apache/carbondata/core/datamap/dev/CacheableIndex.java
@@ -22,26 +22,26 @@ import java.util.List;
 import java.util.Map;
 import java.util.Set;
 
-import org.apache.carbondata.core.datamap.DataMapDistributable;
+import org.apache.carbondata.core.datamap.IndexInputSplit;
 import org.apache.carbondata.core.datamap.Segment;
-import org.apache.carbondata.core.datamap.dev.expr.DataMapExprWrapper;
-import org.apache.carbondata.core.indexstore.BlockletDataMapIndexWrapper;
+import org.apache.carbondata.core.datamap.dev.expr.IndexExprWrapper;
+import org.apache.carbondata.core.indexstore.BlockletIndexWrapper;
 import org.apache.carbondata.core.indexstore.TableBlockIndexUniqueIdentifier;
 import org.apache.carbondata.core.indexstore.TableBlockIndexUniqueIdentifierWrapper;
 
 /**
  * Interface for data map caching
  */
-public interface CacheableDataMap {
+public interface CacheableIndex {
 
   /**
-   * Add the blockletDataMapIndexWrapper to cache for key tableBlockIndexUniqueIdentifier
+   * Add the blockletIndexWrapper to cache for key tableBlockIndexUniqueIdentifier
    *
    * @param tableBlockIndexUniqueIdentifierWrapper
-   * @param blockletDataMapIndexWrapper
+   * @param blockletIndexWrapper
    */
   void cache(TableBlockIndexUniqueIdentifierWrapper tableBlockIndexUniqueIdentifierWrapper,
-      BlockletDataMapIndexWrapper blockletDataMapIndexWrapper) throws IOException;
+      BlockletIndexWrapper blockletIndexWrapper) throws IOException;
 
   /**
    * Get all the uncached distributables from the list.
@@ -49,11 +49,11 @@ public interface CacheableDataMap {
    * @param distributables
    * @return
    */
-  List<DataMapDistributable> getAllUncachedDistributables(List<DataMapDistributable> distributables)
+  List<IndexInputSplit> getAllUncachedDistributables(List<IndexInputSplit> distributables)
       throws IOException;
 
-  List<DataMapDistributable> getAllUncachedDistributables(
-      List<Segment> segments, DataMapExprWrapper dataMapExprWrapper) throws IOException;
+  List<IndexInputSplit> getAllUncachedDistributables(
+      List<Segment> segments, IndexExprWrapper indexExprWrapper) throws IOException;
 
   void updateSegmentDataMap(
       Map<String, Set<TableBlockIndexUniqueIdentifier>> indexUniqueIdentifier);
diff --git a/core/src/main/java/org/apache/carbondata/core/datamap/dev/DataMapSyncStatus.java b/core/src/main/java/org/apache/carbondata/core/datamap/dev/DataMapSyncStatus.java
index c4f790a..dc54f3f 100644
--- a/core/src/main/java/org/apache/carbondata/core/datamap/dev/DataMapSyncStatus.java
+++ b/core/src/main/java/org/apache/carbondata/core/datamap/dev/DataMapSyncStatus.java
@@ -24,8 +24,8 @@ import java.util.Map;
 
 import org.apache.carbondata.common.annotations.InterfaceAudience;
 import org.apache.carbondata.core.constants.CarbonCommonConstants;
-import org.apache.carbondata.core.datamap.DataMapUtil;
-import org.apache.carbondata.core.datamap.status.DataMapSegmentStatusUtil;
+import org.apache.carbondata.core.datamap.IndexUtil;
+import org.apache.carbondata.core.datamap.status.MVSegmentStatusUtil;
 import org.apache.carbondata.core.metadata.schema.table.DataMapSchema;
 import org.apache.carbondata.core.metadata.schema.table.RelationIdentifier;
 import org.apache.carbondata.core.statusmanager.LoadMetadataDetails;
@@ -36,7 +36,7 @@ import org.apache.carbondata.core.util.path.CarbonTablePath;
 /**
  * Interface to check whether datamap can be enabled
  */
-@InterfaceAudience.Developer("DataMap")
+@InterfaceAudience.Developer("Index")
 public abstract class DataMapSyncStatus {
 
   /**
@@ -57,7 +57,7 @@ public abstract class DataMapSyncStatus {
     for (LoadMetadataDetails loadMetadataDetail : dataMapLoadMetadataDetails) {
       if (loadMetadataDetail.getSegmentStatus() == SegmentStatus.SUCCESS) {
         Map<String, List<String>> segmentMap =
-            DataMapSegmentStatusUtil.getSegmentMap(loadMetadataDetail.getExtraInfo());
+            MVSegmentStatusUtil.getSegmentMap(loadMetadataDetail.getExtraInfo());
         if (dataMapSegmentMap.isEmpty()) {
           dataMapSegmentMap.putAll(segmentMap);
         } else {
@@ -72,7 +72,7 @@ public abstract class DataMapSyncStatus {
     List<RelationIdentifier> parentTables = dataMapSchema.getParentTables();
     for (RelationIdentifier parentTable : parentTables) {
       List<String> mainTableValidSegmentList =
-          DataMapUtil.getMainTableValidSegmentList(parentTable);
+          IndexUtil.getMainTableValidSegmentList(parentTable);
       if (!mainTableValidSegmentList.isEmpty() && !dataMapSegmentMap.isEmpty()) {
         isDataMapInSync = dataMapSegmentMap.get(
             parentTable.getDatabaseName() + CarbonCommonConstants.POINT + parentTable
diff --git a/core/src/main/java/org/apache/carbondata/core/datamap/dev/DataMap.java b/core/src/main/java/org/apache/carbondata/core/datamap/dev/Index.java
similarity index 89%
rename from core/src/main/java/org/apache/carbondata/core/datamap/dev/DataMap.java
rename to core/src/main/java/org/apache/carbondata/core/datamap/dev/Index.java
index b41a355..ce28759 100644
--- a/core/src/main/java/org/apache/carbondata/core/datamap/dev/DataMap.java
+++ b/core/src/main/java/org/apache/carbondata/core/datamap/dev/Index.java
@@ -35,15 +35,15 @@ import org.apache.carbondata.core.scan.filter.resolver.FilterResolverIntf;
  * An entity which can store and retrieve index data.
  */
 @InterfaceAudience.Internal
-public interface DataMap<T extends Blocklet> {
+public interface Index<T extends Blocklet> {
 
   /**
-   * It is called to load the data map to memory or to initialize it.
+   * It is called to load the index to memory or to initialize it.
    */
-  void init(DataMapModel dataMapModel) throws IOException;
+  void init(IndexModel indexModel) throws IOException;
 
   /**
-   * Prune the datamap with resolved filter expression and partition information.
+   * Prune the table with resolved filter expression and partition information.
    * It returns the list of blocklets where these filters can exist.
    */
   List<T> prune(FilterResolverIntf filterExp, SegmentProperties segmentProperties,
@@ -51,7 +51,7 @@ public interface DataMap<T extends Blocklet> {
       throws IOException;
 
   /**
-   * Prune the datamap with filter expression and partition information. It returns the list of
+   * Prune the table with filter expression and partition information. It returns the list of
    * blocklets where these filters can exist.
    */
   List<T> prune(Expression filter, SegmentProperties segmentProperties,
@@ -82,7 +82,7 @@ public interface DataMap<T extends Blocklet> {
   void clear();
 
   /**
-   * clears all the resources for datamaps
+   * clears all the resources
    */
   void finish();
 
diff --git a/core/src/main/java/org/apache/carbondata/core/datamap/dev/DataMapBuilder.java b/core/src/main/java/org/apache/carbondata/core/datamap/dev/IndexBuilder.java
similarity index 81%
rename from core/src/main/java/org/apache/carbondata/core/datamap/dev/DataMapBuilder.java
rename to core/src/main/java/org/apache/carbondata/core/datamap/dev/IndexBuilder.java
index 45f7fa9..4f7aaa6 100644
--- a/core/src/main/java/org/apache/carbondata/core/datamap/dev/DataMapBuilder.java
+++ b/core/src/main/java/org/apache/carbondata/core/datamap/dev/IndexBuilder.java
@@ -22,12 +22,12 @@ import java.io.IOException;
 import org.apache.carbondata.common.annotations.InterfaceAudience;
 
 /**
- * DataMapBuilder is used to implement REBUILD DATAMAP command, it reads all existing
- * data in main table and load them into the DataMap. All existing index data will be deleted
- * if there are existing data in the datamap.
+ * IndexBuilder is used to implement REFRESH INDEX command, it reads all existing
+ * data in main table and load them into the Index. All existing index data will be deleted
+ * if there are existing data in the index.
  */
-@InterfaceAudience.Developer("DataMap")
-public interface DataMapBuilder {
+@InterfaceAudience.Developer("Index")
+public interface IndexBuilder {
   void initialize() throws IOException;
 
   void addRow(int blockletId, int pageId, int rowId, Object[] values) throws IOException;
diff --git a/core/src/main/java/org/apache/carbondata/core/datamap/dev/DataMapFactory.java b/core/src/main/java/org/apache/carbondata/core/datamap/dev/IndexFactory.java
similarity index 58%
rename from core/src/main/java/org/apache/carbondata/core/datamap/dev/DataMapFactory.java
rename to core/src/main/java/org/apache/carbondata/core/datamap/dev/IndexFactory.java
index 711c495..f5ca77c 100644
--- a/core/src/main/java/org/apache/carbondata/core/datamap/dev/DataMapFactory.java
+++ b/core/src/main/java/org/apache/carbondata/core/datamap/dev/IndexFactory.java
@@ -24,14 +24,10 @@ import java.util.List;
 import java.util.Map;
 import java.util.Set;
 
-import org.apache.carbondata.common.exceptions.sql.MalformedDataMapCommandException;
-import org.apache.carbondata.core.datamap.DataMapDistributable;
-import org.apache.carbondata.core.datamap.DataMapFilter;
-import org.apache.carbondata.core.datamap.DataMapLevel;
-import org.apache.carbondata.core.datamap.DataMapMeta;
-import org.apache.carbondata.core.datamap.Segment;
-import org.apache.carbondata.core.datamap.dev.cgdatamap.CoarseGrainDataMap;
-import org.apache.carbondata.core.datamap.dev.expr.DataMapDistributableWrapper;
+import org.apache.carbondata.common.exceptions.sql.MalformedIndexCommandException;
+import org.apache.carbondata.core.datamap.*;
+import org.apache.carbondata.core.datamap.dev.cgdatamap.CoarseGrainIndex;
+import org.apache.carbondata.core.datamap.dev.expr.IndexInputSplitWrapper;
 import org.apache.carbondata.core.datastore.block.SegmentProperties;
 import org.apache.carbondata.core.features.TableOperation;
 import org.apache.carbondata.core.indexstore.PartitionSpec;
@@ -41,17 +37,15 @@ import org.apache.carbondata.core.metadata.schema.table.DataMapSchema;
 import org.apache.carbondata.core.metadata.schema.table.column.CarbonColumn;
 import org.apache.carbondata.events.Event;
 
-import static org.apache.carbondata.core.constants.CarbonCommonConstants.INDEX_COLUMNS;
-
 /**
- * Interface for datamap of index type, it is responsible for creating the datamap.
+ * Factory class for creating the index.
  */
-public abstract class DataMapFactory<T extends DataMap> {
+public abstract class IndexFactory<T extends Index> {
 
   private CarbonTable carbonTable;
   private DataMapSchema dataMapSchema;
 
-  public DataMapFactory(CarbonTable carbonTable, DataMapSchema dataMapSchema) {
+  public IndexFactory(CarbonTable carbonTable, DataMapSchema dataMapSchema) {
     this.carbonTable = carbonTable;
     this.dataMapSchema = dataMapSchema;
   }
@@ -68,7 +62,7 @@ public abstract class DataMapFactory<T extends DataMap> {
     return dataMapSchema;
   }
 
-  public DataMapDistributableWrapper toDistributableSegment(Segment segment,
+  public IndexInputSplitWrapper toDistributableSegment(Segment segment,
       DataMapSchema schema, AbsoluteTableIdentifier identifier, String uniqueId) {
     return null;
   }
@@ -76,63 +70,63 @@ public abstract class DataMapFactory<T extends DataMap> {
   /**
    * Create a new write for this datamap, to write new data into the specified segment and shard
    */
-  public abstract DataMapWriter createWriter(Segment segment, String shardName,
+  public abstract IndexWriter createWriter(Segment segment, String shardName,
       SegmentProperties segmentProperties) throws IOException;
 
   /**
-   * Create a new DataMapBuilder for this datamap, to rebuild the specified
+   * Create a new IndexBuilder for this index, to rebuild the specified
    * segment and shard data in the main table.
-   * TODO: refactor to unify with DataMapWriter
+   * TODO: refactor to unify with IndexWriter
    */
-  public abstract DataMapBuilder createBuilder(Segment segment, String shardName,
+  public abstract IndexBuilder createBuilder(Segment segment, String shardName,
       SegmentProperties segmentProperties) throws IOException;
 
   /**
-   * Get the datamap for all segments
+   * Get the index for all segments
    */
-  public Map<Segment, List<CoarseGrainDataMap>> getDataMaps(List<Segment> segments,
-      DataMapFilter filter) throws IOException {
-    Map<Segment, List<CoarseGrainDataMap>> dataMaps = new HashMap<>();
+  public Map<Segment, List<CoarseGrainIndex>> getIndexes(List<Segment> segments,
+      IndexFilter filter) throws IOException {
+    Map<Segment, List<CoarseGrainIndex>> dataMaps = new HashMap<>();
     for (Segment segment : segments) {
-      dataMaps.put(segment, (List<CoarseGrainDataMap>) this.getDataMaps(segment));
+      dataMaps.put(segment, (List<CoarseGrainIndex>) this.getIndexes(segment));
     }
     return dataMaps;
   }
 
   /**
-   * Get the datamap for all segments with matched partitions. Load datamaps to cache, only if it
+   * Get the index for all segments with matched partitions. Load index to cache, only if it
    * matches the partition.
    */
-  public Map<Segment, List<CoarseGrainDataMap>> getDataMaps(List<Segment> segments,
-      List<PartitionSpec> partitionSpecs, DataMapFilter dataMapFilter) throws IOException {
-    Map<Segment, List<CoarseGrainDataMap>> dataMaps = new HashMap<>();
+  public Map<Segment, List<CoarseGrainIndex>> getIndexes(List<Segment> segments,
+      List<PartitionSpec> partitionSpecs, IndexFilter indexFilter) throws IOException {
+    Map<Segment, List<CoarseGrainIndex>> dataMaps = new HashMap<>();
     for (Segment segment : segments) {
-      dataMaps.put(segment, (List<CoarseGrainDataMap>) this.getDataMaps(segment, partitionSpecs));
+      dataMaps.put(segment, (List<CoarseGrainIndex>) this.getIndexes(segment, partitionSpecs));
     }
     return dataMaps;
   }
 
   /**
-   * Get the datamap for segmentId
+   * Get the index for segmentId
    */
-  public abstract List<T> getDataMaps(Segment segment) throws IOException;
+  public abstract List<T> getIndexes(Segment segment) throws IOException;
 
   /**
-   * Get the datamap for segmentId with matched partitions
+   * Get the index for segmentId with matched partitions
    */
-  public abstract List<T> getDataMaps(Segment segment, List<PartitionSpec> partitions)
+  public abstract List<T> getIndexes(Segment segment, List<PartitionSpec> partitions)
       throws IOException;
 
   /**
-   * Get datamaps for distributable object.
+   * Get index for distributable object.
    */
-  public abstract List<T> getDataMaps(DataMapDistributable distributable) throws IOException;
+  public abstract List<T> getIndexes(IndexInputSplit distributable) throws IOException;
 
   /**
    * Get all distributable objects of a segmentId
    * @return
    */
-  public abstract List<DataMapDistributable> toDistributable(Segment segment);
+  public abstract List<IndexInputSplit> toDistributable(Segment segment);
 
   /**
    *
@@ -141,46 +135,46 @@ public abstract class DataMapFactory<T extends DataMap> {
   public abstract void fireEvent(Event event);
 
   /**
-   * Clear all datamaps for a segment from memory
+   * Clear all index for a segment from memory
    */
   public void clear(String segmentNo) {
 
   }
 
   /**
-   * Clear all datamaps from memory
+   * Clear all index from memory
    */
   public abstract void clear();
 
   /**
-   * Return metadata of this datamap
+   * Return metadata of this index
    */
-  public abstract DataMapMeta getMeta();
+  public abstract IndexMeta getMeta();
 
   /**
-   *  Type of datamap whether it is FG or CG
+   *  Type of index whether it is FG or CG
    */
-  public abstract DataMapLevel getDataMapLevel();
+  public abstract IndexLevel getDataMapLevel();
 
   /**
-   * delete datamap data in the specified segment
+   * delete index data in the specified segment
    */
-  public abstract void deleteDatamapData(Segment segment) throws IOException;
+  public abstract void deleteIndexData(Segment segment) throws IOException;
 
   /**
-   * delete datamap data if any
+   * delete index data if any
    */
-  public abstract void deleteDatamapData();
+  public abstract void deleteIndexData();
 
   /**
-   * delete datamap data if any
+   * delete index data if any
    */
-  public void deleteSegmentDatamapData(String segmentNo) throws IOException {
+  public void deleteSegmentIndexData(String segmentNo) throws IOException {
 
   }
 
   /**
-   * This function should return true is the input operation enum will make the datamap become stale
+   * This function should return true is the input operation enum will make the index become stale
    */
   public abstract boolean willBecomeStale(TableOperation operation);
 
@@ -192,14 +186,14 @@ public abstract class DataMapFactory<T extends DataMap> {
    * 3. INDEX_COLUMNS can't contains duplicate same columns
    * 4. INDEX_COLUMNS should be exists in table columns
    */
-  public void validate() throws MalformedDataMapCommandException {
+  public void validate() throws MalformedIndexCommandException {
     List<CarbonColumn> indexColumns = carbonTable.getIndexedColumns(dataMapSchema);
     Set<String> unique = new HashSet<>();
     for (CarbonColumn indexColumn : indexColumns) {
       unique.add(indexColumn.getColName());
     }
     if (unique.size() != indexColumns.size()) {
-      throw new MalformedDataMapCommandException(INDEX_COLUMNS + " has duplicate column");
+      throw new MalformedIndexCommandException("index column list has duplicate column");
     }
   }
 
diff --git a/core/src/main/java/org/apache/carbondata/core/datamap/dev/DataMapModel.java b/core/src/main/java/org/apache/carbondata/core/datamap/dev/IndexModel.java
similarity index 89%
rename from core/src/main/java/org/apache/carbondata/core/datamap/dev/DataMapModel.java
rename to core/src/main/java/org/apache/carbondata/core/datamap/dev/IndexModel.java
index 5f4d1dd..9a0ab54 100644
--- a/core/src/main/java/org/apache/carbondata/core/datamap/dev/DataMapModel.java
+++ b/core/src/main/java/org/apache/carbondata/core/datamap/dev/IndexModel.java
@@ -20,15 +20,15 @@ package org.apache.carbondata.core.datamap.dev;
 import org.apache.hadoop.conf.Configuration;
 
 /**
- * Information required to build datamap
+ * Information required to build index
  */
-public class DataMapModel {
+public class IndexModel {
 
   private String filePath;
 
   private Configuration configuration;
 
-  public DataMapModel(String filePath, Configuration configuration) {
+  public IndexModel(String filePath, Configuration configuration) {
     this.filePath = filePath;
     this.configuration = configuration;
   }
diff --git a/core/src/main/java/org/apache/carbondata/core/datamap/dev/DataMapWriter.java b/core/src/main/java/org/apache/carbondata/core/datamap/dev/IndexWriter.java
similarity index 95%
rename from core/src/main/java/org/apache/carbondata/core/datamap/dev/DataMapWriter.java
rename to core/src/main/java/org/apache/carbondata/core/datamap/dev/IndexWriter.java
index ec69b19..8381ed9 100644
--- a/core/src/main/java/org/apache/carbondata/core/datamap/dev/DataMapWriter.java
+++ b/core/src/main/java/org/apache/carbondata/core/datamap/dev/IndexWriter.java
@@ -34,9 +34,9 @@ import org.apache.carbondata.core.util.path.CarbonTablePath;
  * Developer should implement this interface to write index files.
  * Writer will be called for every new block/blocklet/page is created when data load is executing.
  */
-@InterfaceAudience.Developer("DataMap")
+@InterfaceAudience.Developer("Index")
 @InterfaceStability.Evolving
-public abstract class DataMapWriter {
+public abstract class IndexWriter {
 
   protected String tablePath;
 
@@ -48,7 +48,7 @@ public abstract class DataMapWriter {
 
   private boolean isWritingFinished;
 
-  public DataMapWriter(String tablePath, String dataMapName, List<CarbonColumn> indexColumns,
+  public IndexWriter(String tablePath, String dataMapName, List<CarbonColumn> indexColumns,
       Segment segment, String shardName) {
     this.tablePath = tablePath;
     this.segmentId = segment.getSegmentNo();
@@ -89,7 +89,7 @@ public abstract class DataMapWriter {
 
   /**
    * Add columnar page data to the datamap, order of field is same as `indexColumns` in
-   * DataMapMeta returned in DataMapFactory.
+   * IndexMeta returned in IndexFactory.
    * Implementation should copy the content of it as needed, because its memory
    * may be freed after this method returns, in case of unsafe memory
    */
diff --git a/core/src/main/java/org/apache/carbondata/core/datamap/dev/cgdatamap/CoarseGrainDataMap.java b/core/src/main/java/org/apache/carbondata/core/datamap/dev/cgdatamap/CoarseGrainIndex.java
similarity index 87%
rename from core/src/main/java/org/apache/carbondata/core/datamap/dev/cgdatamap/CoarseGrainDataMap.java
rename to core/src/main/java/org/apache/carbondata/core/datamap/dev/cgdatamap/CoarseGrainIndex.java
index dd6440d..8a9b753 100644
--- a/core/src/main/java/org/apache/carbondata/core/datamap/dev/cgdatamap/CoarseGrainDataMap.java
+++ b/core/src/main/java/org/apache/carbondata/core/datamap/dev/cgdatamap/CoarseGrainIndex.java
@@ -23,7 +23,7 @@ import java.util.Map;
 import org.apache.carbondata.common.annotations.InterfaceAudience;
 import org.apache.carbondata.common.annotations.InterfaceStability;
 import org.apache.carbondata.core.datamap.Segment;
-import org.apache.carbondata.core.datamap.dev.DataMap;
+import org.apache.carbondata.core.datamap.dev.Index;
 import org.apache.carbondata.core.datastore.block.SegmentProperties;
 import org.apache.carbondata.core.indexstore.Blocklet;
 import org.apache.carbondata.core.indexstore.PartitionSpec;
@@ -32,11 +32,11 @@ import org.apache.carbondata.core.scan.expression.Expression;
 import org.apache.carbondata.core.scan.filter.executer.FilterExecuter;
 
 /**
- * DataMap for Coarse Grain level, see {@link org.apache.carbondata.core.datamap.DataMapLevel#CG}
+ * Index for Coarse Grain level, see {@link IndexLevel#CG}
  */
-@InterfaceAudience.Developer("DataMap")
+@InterfaceAudience.Developer("Index")
 @InterfaceStability.Evolving
-public abstract class CoarseGrainDataMap implements DataMap<Blocklet> {
+public abstract class CoarseGrainIndex implements Index<Blocklet> {
 
   @Override
   public List<Blocklet> prune(Expression expression, SegmentProperties segmentProperties,
@@ -57,7 +57,7 @@ public abstract class CoarseGrainDataMap implements DataMap<Blocklet> {
 
   @Override
   public int getNumberOfEntries() {
-    // keep default, one record in one datamap
+    // keep default, one record in one index
     return 1;
   }
 }
diff --git a/core/src/main/java/org/apache/carbondata/core/datamap/dev/cgdatamap/CoarseGrainDataMapFactory.java b/core/src/main/java/org/apache/carbondata/core/datamap/dev/cgdatamap/CoarseGrainIndexFactory.java
similarity index 76%
rename from core/src/main/java/org/apache/carbondata/core/datamap/dev/cgdatamap/CoarseGrainDataMapFactory.java
rename to core/src/main/java/org/apache/carbondata/core/datamap/dev/cgdatamap/CoarseGrainIndexFactory.java
index 3de923f..c17fd33 100644
--- a/core/src/main/java/org/apache/carbondata/core/datamap/dev/cgdatamap/CoarseGrainDataMapFactory.java
+++ b/core/src/main/java/org/apache/carbondata/core/datamap/dev/cgdatamap/CoarseGrainIndexFactory.java
@@ -19,28 +19,28 @@ package org.apache.carbondata.core.datamap.dev.cgdatamap;
 
 import org.apache.carbondata.common.annotations.InterfaceAudience;
 import org.apache.carbondata.common.annotations.InterfaceStability;
-import org.apache.carbondata.core.datamap.DataMapLevel;
-import org.apache.carbondata.core.datamap.dev.DataMapFactory;
+import org.apache.carbondata.core.datamap.IndexLevel;
+import org.apache.carbondata.core.datamap.dev.IndexFactory;
 import org.apache.carbondata.core.metadata.schema.table.CarbonTable;
 import org.apache.carbondata.core.metadata.schema.table.DataMapSchema;
 
 /**
- *  Factory for {@link CoarseGrainDataMap}
+ *  Factory for {@link CoarseGrainIndex}
  *  1. Any filter query which hits the table with datamap will call prune method of CGdatamap.
  *  2. The prune method of CGDatamap return list Blocklet , these blocklets contain the
  *     information of block and blocklet.
  *  3. Based on the splits scanrdd schedule the tasks.
  */
-@InterfaceAudience.Developer("DataMap")
+@InterfaceAudience.Developer("Index")
 @InterfaceStability.Evolving
-public abstract class CoarseGrainDataMapFactory extends DataMapFactory<CoarseGrainDataMap> {
+public abstract class CoarseGrainIndexFactory extends IndexFactory<CoarseGrainIndex> {
 
-  public CoarseGrainDataMapFactory(CarbonTable carbonTable, DataMapSchema dataMapSchema) {
+  public CoarseGrainIndexFactory(CarbonTable carbonTable, DataMapSchema dataMapSchema) {
     super(carbonTable, dataMapSchema);
   }
 
   @Override
-  public DataMapLevel getDataMapLevel() {
-    return DataMapLevel.CG;
+  public IndexLevel getDataMapLevel() {
+    return IndexLevel.CG;
   }
 }
diff --git a/core/src/main/java/org/apache/carbondata/core/datamap/dev/expr/AndDataMapExprWrapper.java b/core/src/main/java/org/apache/carbondata/core/datamap/dev/expr/AndIndexExprWrapper.java
similarity index 83%
rename from core/src/main/java/org/apache/carbondata/core/datamap/dev/expr/AndDataMapExprWrapper.java
rename to core/src/main/java/org/apache/carbondata/core/datamap/dev/expr/AndIndexExprWrapper.java
index 4ba6844..96ff2ff 100644
--- a/core/src/main/java/org/apache/carbondata/core/datamap/dev/expr/AndDataMapExprWrapper.java
+++ b/core/src/main/java/org/apache/carbondata/core/datamap/dev/expr/AndIndexExprWrapper.java
@@ -21,8 +21,8 @@ import java.io.IOException;
 import java.util.ArrayList;
 import java.util.List;
 
-import org.apache.carbondata.core.datamap.DataMapDistributable;
-import org.apache.carbondata.core.datamap.DataMapLevel;
+import org.apache.carbondata.core.datamap.IndexInputSplit;
+import org.apache.carbondata.core.datamap.IndexLevel;
 import org.apache.carbondata.core.datamap.Segment;
 import org.apache.carbondata.core.indexstore.ExtendedBlocklet;
 import org.apache.carbondata.core.indexstore.PartitionSpec;
@@ -31,15 +31,15 @@ import org.apache.carbondata.core.scan.filter.resolver.FilterResolverIntf;
 /**
  * And expression for datamaps
  */
-public class AndDataMapExprWrapper extends DataMapExprWrapper {
+public class AndIndexExprWrapper extends IndexExprWrapper {
 
-  private DataMapExprWrapper left;
+  private IndexExprWrapper left;
 
-  private DataMapExprWrapper right;
+  private IndexExprWrapper right;
 
   private FilterResolverIntf resolverIntf;
 
-  public AndDataMapExprWrapper(DataMapExprWrapper left, DataMapExprWrapper right,
+  public AndIndexExprWrapper(IndexExprWrapper left, IndexExprWrapper right,
       FilterResolverIntf resolverIntf) {
     this.left = left;
     this.right = right;
@@ -61,7 +61,7 @@ public class AndDataMapExprWrapper extends DataMapExprWrapper {
   }
 
   @Override
-  public List<ExtendedBlocklet> prune(DataMapDistributable distributable,
+  public List<ExtendedBlocklet> prune(IndexInputSplit distributable,
       List<PartitionSpec> partitionsToPrune)
           throws IOException {
     List<ExtendedBlocklet> leftPrune = left.prune(distributable, partitionsToPrune);
@@ -107,26 +107,26 @@ public class AndDataMapExprWrapper extends DataMapExprWrapper {
   }
 
   @Override
-  public List<DataMapDistributableWrapper> toDistributable(List<Segment> segments)
+  public List<IndexInputSplitWrapper> toDistributable(List<Segment> segments)
       throws IOException {
-    List<DataMapDistributableWrapper> wrappers = new ArrayList<>();
+    List<IndexInputSplitWrapper> wrappers = new ArrayList<>();
     wrappers.addAll(left.toDistributable(segments));
     wrappers.addAll(right.toDistributable(segments));
     return wrappers;
   }
 
   @Override
-  public DataMapLevel getDataMapLevel() {
+  public IndexLevel getDataMapLevel() {
     return left.getDataMapLevel();
   }
 
   @Override
-  public DataMapExprWrapper getLeftDataMapWrapper() {
+  public IndexExprWrapper getLeftDataMapWrapper() {
     return left;
   }
 
   @Override
-  public DataMapExprWrapper getRightDataMapWrapprt() {
+  public IndexExprWrapper getRightDataMapWrapprt() {
     return right;
   }
 }
diff --git a/core/src/main/java/org/apache/carbondata/core/datamap/dev/expr/DataMapExprWrapper.java b/core/src/main/java/org/apache/carbondata/core/datamap/dev/expr/IndexExprWrapper.java
similarity index 83%
rename from core/src/main/java/org/apache/carbondata/core/datamap/dev/expr/DataMapExprWrapper.java
rename to core/src/main/java/org/apache/carbondata/core/datamap/dev/expr/IndexExprWrapper.java
index 6de4745..0f3edab 100644
--- a/core/src/main/java/org/apache/carbondata/core/datamap/dev/expr/DataMapExprWrapper.java
+++ b/core/src/main/java/org/apache/carbondata/core/datamap/dev/expr/IndexExprWrapper.java
@@ -21,8 +21,8 @@ import java.io.IOException;
 import java.io.Serializable;
 import java.util.List;
 
-import org.apache.carbondata.core.datamap.DataMapDistributable;
-import org.apache.carbondata.core.datamap.DataMapLevel;
+import org.apache.carbondata.core.datamap.IndexInputSplit;
+import org.apache.carbondata.core.datamap.IndexLevel;
 import org.apache.carbondata.core.datamap.Segment;
 import org.apache.carbondata.core.indexstore.ExtendedBlocklet;
 import org.apache.carbondata.core.indexstore.PartitionSpec;
@@ -32,7 +32,7 @@ import org.apache.carbondata.core.scan.filter.resolver.FilterResolverIntf;
  * It is the wrapper around datamap and related filter expression. By using it user can apply
  * datamaps in expression style.
  */
-public abstract class DataMapExprWrapper implements Serializable {
+public abstract class IndexExprWrapper implements Serializable {
 
   /**
    * It get the blocklets from each leaf node datamap and apply expressions on the blocklets
@@ -49,7 +49,7 @@ public abstract class DataMapExprWrapper implements Serializable {
    * @return the pruned ExtendedBlocklet list
    * @throws IOException
    */
-  public abstract List<ExtendedBlocklet> prune(DataMapDistributable distributable,
+  public abstract List<ExtendedBlocklet> prune(IndexInputSplit distributable,
       List<PartitionSpec> partitionsToPrune) throws IOException;
 
   /**
@@ -75,7 +75,7 @@ public abstract class DataMapExprWrapper implements Serializable {
    * @return
    * @throws IOException
    */
-  public abstract List<DataMapDistributableWrapper> toDistributable(List<Segment> segments)
+  public abstract List<IndexInputSplitWrapper> toDistributable(List<Segment> segments)
       throws IOException;
 
   /**
@@ -89,22 +89,22 @@ public abstract class DataMapExprWrapper implements Serializable {
   /**
    * Get the datamap level.
    */
-  public abstract DataMapLevel getDataMapLevel();
+  public abstract IndexLevel getDataMapLevel();
 
   /**
    * get the left datamap wrapper
    */
-  public abstract DataMapExprWrapper getLeftDataMapWrapper();
+  public abstract IndexExprWrapper getLeftDataMapWrapper();
 
   /**
    * get the right datamap wrapper
    */
-  public abstract DataMapExprWrapper getRightDataMapWrapprt();
+  public abstract IndexExprWrapper getRightDataMapWrapprt();
 
   /**
    * Convert segment to distributable object.
    */
-  public DataMapDistributableWrapper toDistributableSegment(Segment segment)
+  public IndexInputSplitWrapper toDistributableSegment(Segment segment)
       throws IOException {
     return null;
   }
diff --git a/core/src/main/java/org/apache/carbondata/core/datamap/dev/expr/DataMapExprWrapperImpl.java b/core/src/main/java/org/apache/carbondata/core/datamap/dev/expr/IndexExprWrapperImpl.java
similarity index 65%
rename from core/src/main/java/org/apache/carbondata/core/datamap/dev/expr/DataMapExprWrapperImpl.java
rename to core/src/main/java/org/apache/carbondata/core/datamap/dev/expr/IndexExprWrapperImpl.java
index c489d3a..7f558f0 100644
--- a/core/src/main/java/org/apache/carbondata/core/datamap/dev/expr/DataMapExprWrapperImpl.java
+++ b/core/src/main/java/org/apache/carbondata/core/datamap/dev/expr/IndexExprWrapperImpl.java
@@ -22,28 +22,28 @@ import java.util.ArrayList;
 import java.util.List;
 import java.util.UUID;
 
-import org.apache.carbondata.core.datamap.DataMapDistributable;
-import org.apache.carbondata.core.datamap.DataMapFilter;
-import org.apache.carbondata.core.datamap.DataMapLevel;
+import org.apache.carbondata.core.datamap.IndexFilter;
+import org.apache.carbondata.core.datamap.IndexInputSplit;
+import org.apache.carbondata.core.datamap.IndexLevel;
 import org.apache.carbondata.core.datamap.Segment;
-import org.apache.carbondata.core.datamap.TableDataMap;
-import org.apache.carbondata.core.datamap.dev.DataMap;
+import org.apache.carbondata.core.datamap.TableIndex;
+import org.apache.carbondata.core.datamap.dev.Index;
 import org.apache.carbondata.core.indexstore.ExtendedBlocklet;
 import org.apache.carbondata.core.indexstore.PartitionSpec;
 import org.apache.carbondata.core.metadata.schema.table.DataMapSchema;
 import org.apache.carbondata.core.scan.filter.resolver.FilterResolverIntf;
 
-public class DataMapExprWrapperImpl extends DataMapExprWrapper {
+public class IndexExprWrapperImpl extends IndexExprWrapper {
 
   private static final long serialVersionUID = -6240385328696074171L;
 
-  private transient TableDataMap dataMap;
+  private transient TableIndex dataMap;
 
   private FilterResolverIntf expression;
 
   private String uniqueId;
 
-  public DataMapExprWrapperImpl(TableDataMap dataMap, FilterResolverIntf expression) {
+  public IndexExprWrapperImpl(TableIndex dataMap, FilterResolverIntf expression) {
     this.dataMap = dataMap;
     this.expression = expression;
     this.uniqueId = UUID.randomUUID().toString();
@@ -52,14 +52,14 @@ public class DataMapExprWrapperImpl extends DataMapExprWrapper {
   @Override
   public List<ExtendedBlocklet> prune(List<Segment> segments, List<PartitionSpec> partitionsToPrune)
       throws IOException {
-    return dataMap.prune(segments, new DataMapFilter(expression), partitionsToPrune);
+    return dataMap.prune(segments, new IndexFilter(expression), partitionsToPrune);
   }
 
-  public List<ExtendedBlocklet> prune(DataMapDistributable distributable,
+  public List<ExtendedBlocklet> prune(IndexInputSplit distributable,
       List<PartitionSpec> partitionsToPrune)
       throws IOException {
-    List<DataMap> dataMaps = dataMap.getTableDataMaps(distributable);
-    return dataMap.prune(dataMaps, distributable, expression, partitionsToPrune);
+    List<Index> indices = dataMap.getTableDataMaps(distributable);
+    return dataMap.prune(indices, distributable, expression, partitionsToPrune);
   }
 
   @Override
@@ -87,18 +87,18 @@ public class DataMapExprWrapperImpl extends DataMapExprWrapper {
   }
 
   @Override
-  public List<DataMapDistributableWrapper> toDistributable(List<Segment> segments) {
-    List<DataMapDistributable> dataMapDistributables = dataMap.toDistributable(segments);
-    List<DataMapDistributableWrapper> wrappers = new ArrayList<>();
-    for (DataMapDistributable distributable : dataMapDistributables) {
-      wrappers.add(new DataMapDistributableWrapper(uniqueId, distributable));
+  public List<IndexInputSplitWrapper> toDistributable(List<Segment> segments) {
+    List<IndexInputSplit> indexInputSplits = dataMap.toDistributable(segments);
+    List<IndexInputSplitWrapper> wrappers = new ArrayList<>();
+    for (IndexInputSplit distributable : indexInputSplits) {
+      wrappers.add(new IndexInputSplitWrapper(uniqueId, distributable));
     }
     return wrappers;
   }
 
   @Override
-  public DataMapLevel getDataMapLevel() {
-    return dataMap.getDataMapFactory().getDataMapLevel();
+  public IndexLevel getDataMapLevel() {
+    return dataMap.getIndexFactory().getDataMapLevel();
   }
 
   public DataMapSchema getDataMapSchema() {
@@ -106,19 +106,19 @@ public class DataMapExprWrapperImpl extends DataMapExprWrapper {
   }
 
   @Override
-  public DataMapExprWrapper getLeftDataMapWrapper() {
+  public IndexExprWrapper getLeftDataMapWrapper() {
     return null;
   }
 
   @Override
-  public DataMapExprWrapper getRightDataMapWrapprt() {
+  public IndexExprWrapper getRightDataMapWrapprt() {
     return null;
   }
 
   /**
    * Convert segment to distributable object.
    */
-  public DataMapDistributableWrapper toDistributableSegment(Segment segment)
+  public IndexInputSplitWrapper toDistributableSegment(Segment segment)
       throws IOException {
     return dataMap.toDistributableSegment(segment, uniqueId);
   }
diff --git a/core/src/main/java/org/apache/carbondata/core/datamap/dev/expr/DataMapDistributableWrapper.java b/core/src/main/java/org/apache/carbondata/core/datamap/dev/expr/IndexInputSplitWrapper.java
similarity index 79%
rename from core/src/main/java/org/apache/carbondata/core/datamap/dev/expr/DataMapDistributableWrapper.java
rename to core/src/main/java/org/apache/carbondata/core/datamap/dev/expr/IndexInputSplitWrapper.java
index 7ec16b0..87d52f1 100644
--- a/core/src/main/java/org/apache/carbondata/core/datamap/dev/expr/DataMapDistributableWrapper.java
+++ b/core/src/main/java/org/apache/carbondata/core/datamap/dev/expr/IndexInputSplitWrapper.java
@@ -19,17 +19,17 @@ package org.apache.carbondata.core.datamap.dev.expr;
 
 import java.io.Serializable;
 
-import org.apache.carbondata.core.datamap.DataMapDistributable;
+import org.apache.carbondata.core.datamap.IndexInputSplit;
 
 import org.apache.hadoop.mapreduce.InputSplit;
 
-public class DataMapDistributableWrapper extends InputSplit implements Serializable {
+public class IndexInputSplitWrapper extends InputSplit implements Serializable {
 
   private String uniqueId;
 
-  private DataMapDistributable distributable;
+  private IndexInputSplit distributable;
 
-  public DataMapDistributableWrapper(String uniqueId, DataMapDistributable distributable) {
+  public IndexInputSplitWrapper(String uniqueId, IndexInputSplit distributable) {
     this.uniqueId = uniqueId;
     this.distributable = distributable;
   }
@@ -38,7 +38,7 @@ public class DataMapDistributableWrapper extends InputSplit implements Serializa
     return uniqueId;
   }
 
-  public DataMapDistributable getDistributable() {
+  public IndexInputSplit getDistributable() {
     return distributable;
   }
 
diff --git a/core/src/main/java/org/apache/carbondata/core/datamap/dev/expr/DataMapWrapperSimpleInfo.java b/core/src/main/java/org/apache/carbondata/core/datamap/dev/expr/IndexWrapperSimpleInfo.java
similarity index 64%
rename from core/src/main/java/org/apache/carbondata/core/datamap/dev/expr/DataMapWrapperSimpleInfo.java
rename to core/src/main/java/org/apache/carbondata/core/datamap/dev/expr/IndexWrapperSimpleInfo.java
index 331e77e..f9646b2 100644
--- a/core/src/main/java/org/apache/carbondata/core/datamap/dev/expr/DataMapWrapperSimpleInfo.java
+++ b/core/src/main/java/org/apache/carbondata/core/datamap/dev/expr/IndexWrapperSimpleInfo.java
@@ -22,10 +22,10 @@ import org.apache.carbondata.core.metadata.schema.table.DataMapSchema;
 /**
  * schema for datamap wrapper.
  * Currently a DataMapWrapper contains more than one datamap, this class is used to describe its
- * schema. For example a AndDataMapExprWrapper contains BloomFilter in its left and Lucene in
+ * schema. For example a AndIndexExprWrapper contains BloomFilter in its left and Lucene in
  * its right, then its schema would be AND(BloomFilter, Lucene)
  */
-public class DataMapWrapperSimpleInfo {
+public class IndexWrapperSimpleInfo {
   enum WrapperType {
     PRIMITIVE,
     AND,
@@ -33,34 +33,34 @@ public class DataMapWrapperSimpleInfo {
   }
 
   private WrapperType wrapperType;
-  private DataMapWrapperSimpleInfo left;
-  private DataMapWrapperSimpleInfo right;
+  private IndexWrapperSimpleInfo left;
+  private IndexWrapperSimpleInfo right;
   private DataMapSchema schema;
 
-  private DataMapWrapperSimpleInfo(WrapperType wrapperType, DataMapWrapperSimpleInfo left,
-      DataMapWrapperSimpleInfo right) {
+  private IndexWrapperSimpleInfo(WrapperType wrapperType, IndexWrapperSimpleInfo left,
+      IndexWrapperSimpleInfo right) {
     this.wrapperType = wrapperType;
     this.left = left;
     this.right = right;
   }
 
-  private DataMapWrapperSimpleInfo(DataMapSchema schema) {
+  private IndexWrapperSimpleInfo(DataMapSchema schema) {
     this.wrapperType = WrapperType.PRIMITIVE;
     this.schema = schema;
   }
 
-  public static DataMapWrapperSimpleInfo fromDataMapWrapper(DataMapExprWrapper dataMapExprWrapper) {
-    if (dataMapExprWrapper instanceof DataMapExprWrapperImpl) {
-      return new DataMapWrapperSimpleInfo(
-          ((DataMapExprWrapperImpl) dataMapExprWrapper).getDataMapSchema());
-    } else if (dataMapExprWrapper instanceof AndDataMapExprWrapper) {
-      return new DataMapWrapperSimpleInfo(WrapperType.AND,
-          fromDataMapWrapper(dataMapExprWrapper.getLeftDataMapWrapper()),
-          fromDataMapWrapper(dataMapExprWrapper.getRightDataMapWrapprt()));
+  public static IndexWrapperSimpleInfo fromDataMapWrapper(IndexExprWrapper indexExprWrapper) {
+    if (indexExprWrapper instanceof IndexExprWrapperImpl) {
+      return new IndexWrapperSimpleInfo(
+          ((IndexExprWrapperImpl) indexExprWrapper).getDataMapSchema());
+    } else if (indexExprWrapper instanceof AndIndexExprWrapper) {
+      return new IndexWrapperSimpleInfo(WrapperType.AND,
+          fromDataMapWrapper(indexExprWrapper.getLeftDataMapWrapper()),
+          fromDataMapWrapper(indexExprWrapper.getRightDataMapWrapprt()));
     } else {
-      return new DataMapWrapperSimpleInfo(WrapperType.OR,
-          fromDataMapWrapper(dataMapExprWrapper.getLeftDataMapWrapper()),
-          fromDataMapWrapper(dataMapExprWrapper.getRightDataMapWrapprt()));
+      return new IndexWrapperSimpleInfo(WrapperType.OR,
+          fromDataMapWrapper(indexExprWrapper.getLeftDataMapWrapper()),
+          fromDataMapWrapper(indexExprWrapper.getRightDataMapWrapprt()));
     }
   }
 
diff --git a/core/src/main/java/org/apache/carbondata/core/datamap/dev/expr/OrDataMapExprWrapper.java b/core/src/main/java/org/apache/carbondata/core/datamap/dev/expr/OrIndexExprWrapper.java
similarity index 83%
rename from core/src/main/java/org/apache/carbondata/core/datamap/dev/expr/OrDataMapExprWrapper.java
rename to core/src/main/java/org/apache/carbondata/core/datamap/dev/expr/OrIndexExprWrapper.java
index 4bfdf4a..f2eddc6 100644
--- a/core/src/main/java/org/apache/carbondata/core/datamap/dev/expr/OrDataMapExprWrapper.java
+++ b/core/src/main/java/org/apache/carbondata/core/datamap/dev/expr/OrIndexExprWrapper.java
@@ -23,8 +23,8 @@ import java.util.HashSet;
 import java.util.List;
 import java.util.Set;
 
-import org.apache.carbondata.core.datamap.DataMapDistributable;
-import org.apache.carbondata.core.datamap.DataMapLevel;
+import org.apache.carbondata.core.datamap.IndexInputSplit;
+import org.apache.carbondata.core.datamap.IndexLevel;
 import org.apache.carbondata.core.datamap.Segment;
 import org.apache.carbondata.core.indexstore.ExtendedBlocklet;
 import org.apache.carbondata.core.indexstore.PartitionSpec;
@@ -33,15 +33,15 @@ import org.apache.carbondata.core.scan.filter.resolver.FilterResolverIntf;
 /**
  * Or expression for datamaps
  */
-public class OrDataMapExprWrapper extends DataMapExprWrapper {
+public class OrIndexExprWrapper extends IndexExprWrapper {
 
-  private DataMapExprWrapper left;
+  private IndexExprWrapper left;
 
-  private DataMapExprWrapper right;
+  private IndexExprWrapper right;
 
   private FilterResolverIntf resolverIntf;
 
-  public OrDataMapExprWrapper(DataMapExprWrapper left, DataMapExprWrapper right,
+  public OrIndexExprWrapper(IndexExprWrapper left, IndexExprWrapper right,
       FilterResolverIntf resolverIntf) {
     this.left = left;
     this.right = right;
@@ -60,7 +60,7 @@ public class OrDataMapExprWrapper extends DataMapExprWrapper {
   }
 
   @Override
-  public List<ExtendedBlocklet> prune(DataMapDistributable distributable,
+  public List<ExtendedBlocklet> prune(IndexInputSplit distributable,
       List<PartitionSpec> partitionsToPrune)
           throws IOException {
     List<ExtendedBlocklet> leftPrune = left.prune(distributable, partitionsToPrune);
@@ -83,9 +83,9 @@ public class OrDataMapExprWrapper extends DataMapExprWrapper {
   }
 
   @Override
-  public List<DataMapDistributableWrapper> toDistributable(List<Segment> segments)
+  public List<IndexInputSplitWrapper> toDistributable(List<Segment> segments)
       throws IOException {
-    List<DataMapDistributableWrapper> wrappers = new ArrayList<>();
+    List<IndexInputSplitWrapper> wrappers = new ArrayList<>();
     wrappers.addAll(left.toDistributable(segments));
     wrappers.addAll(right.toDistributable(segments));
     return wrappers;
@@ -109,17 +109,17 @@ public class OrDataMapExprWrapper extends DataMapExprWrapper {
   }
 
   @Override
-  public DataMapLevel getDataMapLevel() {
+  public IndexLevel getDataMapLevel() {
     return left.getDataMapLevel();
   }
 
   @Override
-  public DataMapExprWrapper getLeftDataMapWrapper() {
+  public IndexExprWrapper getLeftDataMapWrapper() {
     return left;
   }
 
   @Override
-  public DataMapExprWrapper getRightDataMapWrapprt() {
+  public IndexExprWrapper getRightDataMapWrapprt() {
     return right;
   }
 }
diff --git a/core/src/main/java/org/apache/carbondata/core/datamap/dev/fgdatamap/FineGrainBlocklet.java b/core/src/main/java/org/apache/carbondata/core/datamap/dev/fgdatamap/FineGrainBlocklet.java
index 1162d2f..8387130 100644
--- a/core/src/main/java/org/apache/carbondata/core/datamap/dev/fgdatamap/FineGrainBlocklet.java
+++ b/core/src/main/java/org/apache/carbondata/core/datamap/dev/fgdatamap/FineGrainBlocklet.java
@@ -34,9 +34,9 @@ import org.apache.carbondata.core.metadata.schema.table.Writable;
 import org.apache.carbondata.core.util.BitSetGroup;
 
 /**
- * Used for returning matched rows after pruned by {@link FineGrainDataMap}
+ * Used for returning matched rows after pruned by {@link FineGrainIndex}
  */
-@InterfaceAudience.Developer("DataMap")
+@InterfaceAudience.Developer("Index")
 @InterfaceStability.Evolving
 public class FineGrainBlocklet extends Blocklet implements Serializable {
 
diff --git a/core/src/main/java/org/apache/carbondata/core/datamap/dev/fgdatamap/FineGrainDataMap.java b/core/src/main/java/org/apache/carbondata/core/datamap/dev/fgdatamap/FineGrainIndex.java
similarity index 89%
rename from core/src/main/java/org/apache/carbondata/core/datamap/dev/fgdatamap/FineGrainDataMap.java
rename to core/src/main/java/org/apache/carbondata/core/datamap/dev/fgdatamap/FineGrainIndex.java
index d9fbb1f..529f6f0 100644
--- a/core/src/main/java/org/apache/carbondata/core/datamap/dev/fgdatamap/FineGrainDataMap.java
+++ b/core/src/main/java/org/apache/carbondata/core/datamap/dev/fgdatamap/FineGrainIndex.java
@@ -23,7 +23,7 @@ import java.util.Map;
 import org.apache.carbondata.common.annotations.InterfaceAudience;
 import org.apache.carbondata.common.annotations.InterfaceStability;
 import org.apache.carbondata.core.datamap.Segment;
-import org.apache.carbondata.core.datamap.dev.DataMap;
+import org.apache.carbondata.core.datamap.dev.Index;
 import org.apache.carbondata.core.datastore.block.SegmentProperties;
 import org.apache.carbondata.core.indexstore.PartitionSpec;
 import org.apache.carbondata.core.metadata.schema.table.CarbonTable;
@@ -31,11 +31,11 @@ import org.apache.carbondata.core.scan.expression.Expression;
 import org.apache.carbondata.core.scan.filter.executer.FilterExecuter;
 
 /**
- * DataMap for Fine Grain level, see {@link org.apache.carbondata.core.datamap.DataMapLevel#FG}
+ * Index for Fine Grain level, see {@link IndexLevel#FG}
  */
-@InterfaceAudience.Developer("DataMap")
+@InterfaceAudience.Developer("Index")
 @InterfaceStability.Evolving
-public abstract class FineGrainDataMap implements DataMap<FineGrainBlocklet> {
+public abstract class FineGrainIndex implements Index<FineGrainBlocklet> {
 
   @Override
   public List<FineGrainBlocklet> prune(Expression filter, SegmentProperties segmentProperties,
diff --git a/core/src/main/java/org/apache/carbondata/core/datamap/dev/fgdatamap/FineGrainDataMapFactory.java b/core/src/main/java/org/apache/carbondata/core/datamap/dev/fgdatamap/FineGrainIndexFactory.java
similarity index 80%
rename from core/src/main/java/org/apache/carbondata/core/datamap/dev/fgdatamap/FineGrainDataMapFactory.java
rename to core/src/main/java/org/apache/carbondata/core/datamap/dev/fgdatamap/FineGrainIndexFactory.java
index 1ea21df..053c4de 100644
--- a/core/src/main/java/org/apache/carbondata/core/datamap/dev/fgdatamap/FineGrainDataMapFactory.java
+++ b/core/src/main/java/org/apache/carbondata/core/datamap/dev/fgdatamap/FineGrainIndexFactory.java
@@ -19,13 +19,13 @@ package org.apache.carbondata.core.datamap.dev.fgdatamap;
 
 import org.apache.carbondata.common.annotations.InterfaceAudience;
 import org.apache.carbondata.common.annotations.InterfaceStability;
-import org.apache.carbondata.core.datamap.DataMapLevel;
-import org.apache.carbondata.core.datamap.dev.DataMapFactory;
+import org.apache.carbondata.core.datamap.IndexLevel;
+import org.apache.carbondata.core.datamap.dev.IndexFactory;
 import org.apache.carbondata.core.metadata.schema.table.CarbonTable;
 import org.apache.carbondata.core.metadata.schema.table.DataMapSchema;
 
 /**
- *  Factory for {@link FineGrainDataMap}
+ *  Factory for {@link FineGrainIndex}
  *
  *  1. Any filter query which hits the table with datamap will call prune method of FGdatamap.
  *  2. The prune method of FGDatamap return list FineGrainBlocklet , these blocklets contain the
@@ -36,16 +36,16 @@ import org.apache.carbondata.core.metadata.schema.table.DataMapSchema;
  *  5. In filterscanner we check the datamapwriterpath from split and reNoteads the
  *     bitset if exists. And pass this bitset as input to it.
  */
-@InterfaceAudience.Developer("DataMap")
+@InterfaceAudience.Developer("Index")
 @InterfaceStability.Evolving
-public abstract class FineGrainDataMapFactory extends DataMapFactory<FineGrainDataMap> {
+public abstract class FineGrainIndexFactory extends IndexFactory<FineGrainIndex> {
 
-  public FineGrainDataMapFactory(CarbonTable carbonTable, DataMapSchema dataMapSchema) {
+  public FineGrainIndexFactory(CarbonTable carbonTable, DataMapSchema dataMapSchema) {
     super(carbonTable, dataMapSchema);
   }
 
   @Override
-  public DataMapLevel getDataMapLevel() {
-    return DataMapLevel.FG;
+  public IndexLevel getDataMapLevel() {
+    return IndexLevel.FG;
   }
 }
diff --git a/core/src/main/java/org/apache/carbondata/core/datamap/status/DataMapStatus.java b/core/src/main/java/org/apache/carbondata/core/datamap/status/DataMapStatus.java
index fce16c8..36bec6e 100644
--- a/core/src/main/java/org/apache/carbondata/core/datamap/status/DataMapStatus.java
+++ b/core/src/main/java/org/apache/carbondata/core/datamap/status/DataMapStatus.java
@@ -18,7 +18,7 @@
 package org.apache.carbondata.core.datamap.status;
 
 /**
- * DataMap status
+ * Index status
  */
 public enum DataMapStatus {
   ENABLED, DISABLED, DROPPED
diff --git a/core/src/main/java/org/apache/carbondata/core/datamap/status/DatabaseDataMapStatusProvider.java b/core/src/main/java/org/apache/carbondata/core/datamap/status/DatabaseDataMapStatusProvider.java
index 838eb1d..e15bf73 100644
--- a/core/src/main/java/org/apache/carbondata/core/datamap/status/DatabaseDataMapStatusProvider.java
+++ b/core/src/main/java/org/apache/carbondata/core/datamap/status/DatabaseDataMapStatusProvider.java
@@ -25,7 +25,7 @@ import org.apache.carbondata.core.metadata.schema.table.DataMapSchema;
  * It saves/serializes the array of {{@link DataMapStatusDetail}} to database folder.
  * It ensures the data consistance while concurrent write through write lock. It saves the status
  * to the datamapstatus under the database folder.
- * Now the implement not finished, it used to disable DataMap in multi-tenant scenario.
+ * Now the implement not finished, it used to disable Index in multi-tenant scenario.
  */
 public class DatabaseDataMapStatusProvider implements DataMapStatusStorageProvider {
 
diff --git a/core/src/main/java/org/apache/carbondata/core/datamap/status/DiskBasedDataMapStatusProvider.java b/core/src/main/java/org/apache/carbondata/core/datamap/status/DiskBasedDataMapStatusProvider.java
index bb1b188..66e99d7 100644
--- a/core/src/main/java/org/apache/carbondata/core/datamap/status/DiskBasedDataMapStatusProvider.java
+++ b/core/src/main/java/org/apache/carbondata/core/datamap/status/DiskBasedDataMapStatusProvider.java
@@ -114,7 +114,7 @@ public class DiskBasedDataMapStatusProvider implements DataMapStatusStorageProvi
       locked = carbonTableStatusLock.lockWithRetries();
       if (locked) {
         LOG.info("Datamap status lock has been successfully acquired.");
-        if (dataMapStatus == DataMapStatus.ENABLED && !dataMapSchemas.get(0).isIndexDataMap()) {
+        if (dataMapStatus == DataMapStatus.ENABLED && !dataMapSchemas.get(0).isIndex()) {
           // Enable datamap only if datamap tables and main table are in sync
           if (!DataMapSyncStatus.canDataMapBeEnabled(dataMapSchemas.get(0))) {
             return;
diff --git a/core/src/main/java/org/apache/carbondata/core/datamap/status/DataMapSegmentStatusUtil.java b/core/src/main/java/org/apache/carbondata/core/datamap/status/MVSegmentStatusUtil.java
similarity index 98%
rename from core/src/main/java/org/apache/carbondata/core/datamap/status/DataMapSegmentStatusUtil.java
rename to core/src/main/java/org/apache/carbondata/core/datamap/status/MVSegmentStatusUtil.java
index 9e00104..967d04d 100644
--- a/core/src/main/java/org/apache/carbondata/core/datamap/status/DataMapSegmentStatusUtil.java
+++ b/core/src/main/java/org/apache/carbondata/core/datamap/status/MVSegmentStatusUtil.java
@@ -32,7 +32,7 @@ import com.google.gson.Gson;
 /**
  * Utility class to get updated segment mapping for datamap table
  */
-public class DataMapSegmentStatusUtil {
+public class MVSegmentStatusUtil {
 
   /**
    * This method will convert string to map and return segmentMap
diff --git a/core/src/main/java/org/apache/carbondata/core/datastore/filesystem/LocalCarbonFile.java b/core/src/main/java/org/apache/carbondata/core/datastore/filesystem/LocalCarbonFile.java
index b8333f0..1bb1c99 100644
--- a/core/src/main/java/org/apache/carbondata/core/datastore/filesystem/LocalCarbonFile.java
+++ b/core/src/main/java/org/apache/carbondata/core/datastore/filesystem/LocalCarbonFile.java
@@ -478,11 +478,11 @@ public class LocalCarbonFile implements CarbonFile {
       return false;
     }
     LocalCarbonFile that = (LocalCarbonFile) o;
-    return Objects.equals(file, that.file);
+    return Objects.equals(file.getAbsolutePath(), that.file.getAbsolutePath());
   }
 
   @Override
   public int hashCode() {
-    return Objects.hash(file);
+    return Objects.hash(file.getAbsolutePath());
   }
 }
diff --git a/core/src/main/java/org/apache/carbondata/core/datastore/impl/FileFactory.java b/core/src/main/java/org/apache/carbondata/core/datastore/impl/FileFactory.java
index 7b6aa34..d2dc7b3 100644
--- a/core/src/main/java/org/apache/carbondata/core/datastore/impl/FileFactory.java
+++ b/core/src/main/java/org/apache/carbondata/core/datastore/impl/FileFactory.java
@@ -689,4 +689,39 @@ public final class FileFactory {
       CarbonUtil.closeStreams(fileReader, bufferedReader);
     }
   }
+
+  public static void touchFile(CarbonFile file) throws IOException {
+    if (file.exists()) {
+      return;
+    }
+    touchDirectory(file.getParentFile());
+    file.createNewFile();
+  }
+
+  public static void touchFile(CarbonFile file, FsPermission permission) throws IOException {
+    if (file.exists()) {
+      return;
+    }
+    touchDirectory(file.getParentFile(), permission);
+    file.createNewFile(permission);
+  }
+
+  public static void touchDirectory(CarbonFile directory)
+      throws IOException {
+    if (directory.exists()) {
+      return;
+    }
+    touchDirectory(directory.getParentFile());
+    directory.mkdirs();
+  }
+
+  public static void touchDirectory(CarbonFile directory, FsPermission permission)
+      throws IOException {
+    if (directory.exists()) {
+      return;
+    }
+    touchDirectory(directory.getParentFile(), permission);
+    FileFactory.createDirectoryAndSetPermission(directory.getCanonicalPath(), permission);
+  }
+
 }
diff --git a/core/src/main/java/org/apache/carbondata/core/indexstore/AbstractMemoryDMStore.java b/core/src/main/java/org/apache/carbondata/core/indexstore/AbstractMemoryDMStore.java
index 1be72f0..f113b57 100644
--- a/core/src/main/java/org/apache/carbondata/core/indexstore/AbstractMemoryDMStore.java
+++ b/core/src/main/java/org/apache/carbondata/core/indexstore/AbstractMemoryDMStore.java
@@ -19,12 +19,12 @@ package org.apache.carbondata.core.indexstore;
 
 import java.io.Serializable;
 
-import org.apache.carbondata.core.indexstore.row.DataMapRow;
+import org.apache.carbondata.core.indexstore.row.IndexRow;
 import org.apache.carbondata.core.indexstore.schema.CarbonRowSchema;
 import org.apache.carbondata.core.util.ThreadLocalTaskInfo;
 
 /**
- * Store the data map row @{@link DataMapRow}
+ * Store the data map row @{@link IndexRow}
  */
 public abstract class AbstractMemoryDMStore implements Serializable {
 
@@ -34,9 +34,9 @@ public abstract class AbstractMemoryDMStore implements Serializable {
 
   protected final String taskId = ThreadLocalTaskInfo.getCarbonTaskInfo().getTaskId();
 
-  public abstract void addIndexRow(CarbonRowSchema[] schema, DataMapRow indexRow);
+  public abstract void addIndexRow(CarbonRowSchema[] schema, IndexRow indexRow);
 
-  public abstract DataMapRow getDataMapRow(CarbonRowSchema[] schema, int index);
+  public abstract IndexRow getDataMapRow(CarbonRowSchema[] schema, int index);
 
   public abstract void freeMemory();
 
diff --git a/core/src/main/java/org/apache/carbondata/core/indexstore/Blocklet.java b/core/src/main/java/org/apache/carbondata/core/indexstore/Blocklet.java
index 215752f..a733bdd 100644
--- a/core/src/main/java/org/apache/carbondata/core/indexstore/Blocklet.java
+++ b/core/src/main/java/org/apache/carbondata/core/indexstore/Blocklet.java
@@ -38,7 +38,7 @@ public class Blocklet implements Writable, Serializable {
   /**
    * flag to specify whether to consider blocklet Id in equals and hashcode comparison. This is
    * because when CACHE_LEVEL='BLOCK' which is default value, the blocklet ID returned by
-   * BlockDataMap pruning will always be -1 and other datamaps will give the the correct blocklet
+   * BlockIndex pruning will always be -1 and other datamaps will give the the correct blocklet
    * ID. Therefore if we compare -1 with correct blocklet ID the comparison will become wrong and
    * always false will be returned resulting in incorrect result. Default value for flag is true.
    */
diff --git a/core/src/main/java/org/apache/carbondata/core/indexstore/BlockletDetailInfo.java b/core/src/main/java/org/apache/carbondata/core/indexstore/BlockletDetailInfo.java
index 667b271..8c40ef9 100644
--- a/core/src/main/java/org/apache/carbondata/core/indexstore/BlockletDetailInfo.java
+++ b/core/src/main/java/org/apache/carbondata/core/indexstore/BlockletDetailInfo.java
@@ -28,7 +28,7 @@ import java.util.List;
 import org.apache.carbondata.common.logging.LogServiceFactory;
 import org.apache.carbondata.core.metadata.blocklet.BlockletInfo;
 import org.apache.carbondata.core.metadata.schema.table.column.ColumnSchema;
-import org.apache.carbondata.core.util.BlockletDataMapUtil;
+import org.apache.carbondata.core.util.BlockletIndexUtil;
 
 import org.apache.hadoop.io.Writable;
 import org.apache.log4j.Logger;
@@ -218,13 +218,13 @@ public class BlockletDetailInfo implements Serializable, Writable {
    */
   public void readColumnSchema(byte[] schemaArray) throws IOException {
     if (null != schemaArray) {
-      columnSchemas = BlockletDataMapUtil.readColumnSchema(schemaArray);
+      columnSchemas = BlockletIndexUtil.readColumnSchema(schemaArray);
     }
   }
 
   private void convertColumnSchemaToBinary() throws IOException {
     if (null != columnSchemas) {
-      columnSchemaBinary = BlockletDataMapUtil.convertSchemaToBinary(columnSchemas);
+      columnSchemaBinary = BlockletIndexUtil.convertSchemaToBinary(columnSchemas);
     }
   }
 
diff --git a/core/src/main/java/org/apache/carbondata/core/indexstore/BlockletDataMapIndexStore.java b/core/src/main/java/org/apache/carbondata/core/indexstore/BlockletIndexStore.java
similarity index 79%
rename from core/src/main/java/org/apache/carbondata/core/indexstore/BlockletDataMapIndexStore.java
rename to core/src/main/java/org/apache/carbondata/core/indexstore/BlockletIndexStore.java
index 97eeab8..7f1cf26 100644
--- a/core/src/main/java/org/apache/carbondata/core/indexstore/BlockletDataMapIndexStore.java
+++ b/core/src/main/java/org/apache/carbondata/core/indexstore/BlockletIndexStore.java
@@ -30,15 +30,15 @@ import org.apache.carbondata.common.logging.LogServiceFactory;
 import org.apache.carbondata.core.cache.Cache;
 import org.apache.carbondata.core.cache.CarbonLRUCache;
 import org.apache.carbondata.core.constants.CarbonCommonConstants;
-import org.apache.carbondata.core.datamap.dev.DataMap;
+import org.apache.carbondata.core.datamap.dev.Index;
 import org.apache.carbondata.core.datastore.block.SegmentPropertiesAndSchemaHolder;
-import org.apache.carbondata.core.indexstore.blockletindex.BlockDataMap;
-import org.apache.carbondata.core.indexstore.blockletindex.BlockletDataMapFactory;
-import org.apache.carbondata.core.indexstore.blockletindex.BlockletDataMapModel;
+import org.apache.carbondata.core.indexstore.blockletindex.BlockIndex;
+import org.apache.carbondata.core.indexstore.blockletindex.BlockletIndexFactory;
+import org.apache.carbondata.core.indexstore.blockletindex.BlockletIndexModel;
 import org.apache.carbondata.core.indexstore.blockletindex.SegmentIndexFileStore;
 import org.apache.carbondata.core.metadata.blocklet.DataFileFooter;
 import org.apache.carbondata.core.metadata.schema.table.CarbonTable;
-import org.apache.carbondata.core.util.BlockletDataMapUtil;
+import org.apache.carbondata.core.util.BlockletIndexUtil;
 import org.apache.carbondata.core.util.CarbonUtil;
 
 import org.apache.hadoop.conf.Configuration;
@@ -48,10 +48,10 @@ import org.apache.log4j.Logger;
  * Class to handle loading, unloading,clearing,storing of the table
  * blocks
  */
-public class BlockletDataMapIndexStore
-    implements Cache<TableBlockIndexUniqueIdentifierWrapper, BlockletDataMapIndexWrapper> {
+public class BlockletIndexStore
+    implements Cache<TableBlockIndexUniqueIdentifierWrapper, BlockletIndexWrapper> {
   private static final Logger LOGGER =
-      LogServiceFactory.getLogService(BlockletDataMapIndexStore.class.getName());
+      LogServiceFactory.getLogService(BlockletIndexStore.class.getName());
   /**
    * CarbonLRU cache
    */
@@ -70,25 +70,25 @@ public class BlockletDataMapIndexStore
    *
    * @param lruCache
    */
-  public BlockletDataMapIndexStore(CarbonLRUCache lruCache) {
+  public BlockletIndexStore(CarbonLRUCache lruCache) {
     this.lruCache = lruCache;
     segmentLockMap = new ConcurrentHashMap<String, Object>();
   }
 
   @Override
-  public BlockletDataMapIndexWrapper get(TableBlockIndexUniqueIdentifierWrapper identifierWrapper) {
+  public BlockletIndexWrapper get(TableBlockIndexUniqueIdentifierWrapper identifierWrapper) {
     return get(identifierWrapper, null);
   }
 
-  public BlockletDataMapIndexWrapper get(TableBlockIndexUniqueIdentifierWrapper identifierWrapper,
+  public BlockletIndexWrapper get(TableBlockIndexUniqueIdentifierWrapper identifierWrapper,
       Map<String, Map<String, BlockMetaInfo>> segInfoCache) {
     TableBlockIndexUniqueIdentifier identifier =
         identifierWrapper.getTableBlockIndexUniqueIdentifier();
     String lruCacheKey = identifier.getUniqueTableSegmentIdentifier();
-    BlockletDataMapIndexWrapper blockletDataMapIndexWrapper =
-        (BlockletDataMapIndexWrapper) lruCache.get(lruCacheKey);
-    List<BlockDataMap> dataMaps = new ArrayList<>();
-    if (blockletDataMapIndexWrapper == null) {
+    BlockletIndexWrapper blockletIndexWrapper =
+        (BlockletIndexWrapper) lruCache.get(lruCacheKey);
+    List<BlockIndex> dataMaps = new ArrayList<>();
+    if (blockletIndexWrapper == null) {
       try {
         SegmentIndexFileStore indexFileStore =
             new SegmentIndexFileStore(identifierWrapper.getConfiguration());
@@ -101,17 +101,17 @@ public class BlockletDataMapIndexStore
             segInfoCache.get(segmentFilePath);
         if (carbonDataFileBlockMetaInfoMapping == null) {
           carbonDataFileBlockMetaInfoMapping =
-              BlockletDataMapUtil.createCarbonDataFileBlockMetaInfoMapping(segmentFilePath,
+              BlockletIndexUtil.createCarbonDataFileBlockMetaInfoMapping(segmentFilePath,
                   identifierWrapper.getConfiguration());
           segInfoCache.put(segmentFilePath, carbonDataFileBlockMetaInfoMapping);
         }
         // if the identifier is not a merge file we can directly load the datamaps
         if (identifier.getMergeIndexFileName() == null) {
           List<DataFileFooter> indexInfos = new ArrayList<>();
-          Map<String, BlockMetaInfo> blockMetaInfoMap = BlockletDataMapUtil
+          Map<String, BlockMetaInfo> blockMetaInfoMap = BlockletIndexUtil
               .getBlockMetaInfoMap(identifierWrapper, indexFileStore, filesRead,
                   carbonDataFileBlockMetaInfoMapping, indexInfos);
-          BlockDataMap blockletDataMap =
+          BlockIndex blockletDataMap =
               loadAndGetDataMap(identifier, indexFileStore, blockMetaInfoMap,
                   identifierWrapper.getCarbonTable(),
                   identifierWrapper.isAddToUnsafe(),
@@ -119,21 +119,21 @@ public class BlockletDataMapIndexStore
                   identifierWrapper.isSerializeDmStore(),
                   indexInfos);
           dataMaps.add(blockletDataMap);
-          blockletDataMapIndexWrapper =
-              new BlockletDataMapIndexWrapper(identifier.getSegmentId(), dataMaps);
+          blockletIndexWrapper =
+              new BlockletIndexWrapper(identifier.getSegmentId(), dataMaps);
         } else {
           // if the identifier is a merge file then collect the index files and load the datamaps
           List<TableBlockIndexUniqueIdentifier> tableBlockIndexUniqueIdentifiers =
-              BlockletDataMapUtil.getIndexFileIdentifiersFromMergeFile(identifier, indexFileStore);
+              BlockletIndexUtil.getIndexFileIdentifiersFromMergeFile(identifier, indexFileStore);
           for (TableBlockIndexUniqueIdentifier blockIndexUniqueIdentifier :
               tableBlockIndexUniqueIdentifiers) {
             List<DataFileFooter> indexInfos = new ArrayList<>();
-            Map<String, BlockMetaInfo> blockMetaInfoMap = BlockletDataMapUtil.getBlockMetaInfoMap(
+            Map<String, BlockMetaInfo> blockMetaInfoMap = BlockletIndexUtil.getBlockMetaInfoMap(
                 new TableBlockIndexUniqueIdentifierWrapper(blockIndexUniqueIdentifier,
                     identifierWrapper.getCarbonTable()), indexFileStore, filesRead,
                 carbonDataFileBlockMetaInfoMapping, indexInfos);
             if (!blockMetaInfoMap.isEmpty()) {
-              BlockDataMap blockletDataMap =
+              BlockIndex blockletDataMap =
                   loadAndGetDataMap(blockIndexUniqueIdentifier, indexFileStore, blockMetaInfoMap,
                       identifierWrapper.getCarbonTable(),
                       identifierWrapper.isAddToUnsafe(),
@@ -143,66 +143,66 @@ public class BlockletDataMapIndexStore
               dataMaps.add(blockletDataMap);
             }
           }
-          blockletDataMapIndexWrapper =
-              new BlockletDataMapIndexWrapper(identifier.getSegmentId(), dataMaps);
+          blockletIndexWrapper =
+              new BlockletIndexWrapper(identifier.getSegmentId(), dataMaps);
         }
         if (identifierWrapper.isAddTableBlockToUnsafeAndLRUCache()) {
           long expiration_time = CarbonUtil.getExpiration_time(identifierWrapper.getCarbonTable());
-          lruCache.put(identifier.getUniqueTableSegmentIdentifier(), blockletDataMapIndexWrapper,
-              blockletDataMapIndexWrapper.getMemorySize(), expiration_time);
+          lruCache.put(identifier.getUniqueTableSegmentIdentifier(), blockletIndexWrapper,
+                  blockletIndexWrapper.getMemorySize(), expiration_time);
         }
       } catch (Throwable e) {
         // clear all the memory used by datamaps loaded
-        for (DataMap dataMap : dataMaps) {
-          dataMap.clear();
+        for (Index index : dataMaps) {
+          index.clear();
         }
         LOGGER.error("memory exception when loading datamap: " + e.getMessage(), e);
         throw new RuntimeException(e);
       }
     }
-    return blockletDataMapIndexWrapper;
+    return blockletIndexWrapper;
   }
 
   @Override
-  public List<BlockletDataMapIndexWrapper> getAll(
+  public List<BlockletIndexWrapper> getAll(
       List<TableBlockIndexUniqueIdentifierWrapper> tableSegmentUniqueIdentifiers)
       throws IOException {
     Map<String, Map<String, BlockMetaInfo>> segInfoCache =
         new HashMap<String, Map<String, BlockMetaInfo>>();
 
-    List<BlockletDataMapIndexWrapper> blockletDataMapIndexWrappers =
+    List<BlockletIndexWrapper> blockletIndexWrappers =
         new ArrayList<>(tableSegmentUniqueIdentifiers.size());
     List<TableBlockIndexUniqueIdentifierWrapper> missedIdentifiersWrapper = new ArrayList<>();
-    BlockletDataMapIndexWrapper blockletDataMapIndexWrapper = null;
+    BlockletIndexWrapper blockletIndexWrapper = null;
     // Get the datamaps for each indexfile from cache.
     try {
       for (TableBlockIndexUniqueIdentifierWrapper
                identifierWrapper : tableSegmentUniqueIdentifiers) {
-        BlockletDataMapIndexWrapper dataMapIndexWrapper =
+        BlockletIndexWrapper dataMapIndexWrapper =
             getIfPresent(identifierWrapper);
         if (dataMapIndexWrapper != null) {
-          blockletDataMapIndexWrappers.add(dataMapIndexWrapper);
+          blockletIndexWrappers.add(dataMapIndexWrapper);
         } else {
           missedIdentifiersWrapper.add(identifierWrapper);
         }
       }
       if (missedIdentifiersWrapper.size() > 0) {
         for (TableBlockIndexUniqueIdentifierWrapper identifierWrapper : missedIdentifiersWrapper) {
-          blockletDataMapIndexWrapper = get(identifierWrapper, segInfoCache);
-          blockletDataMapIndexWrappers.add(blockletDataMapIndexWrapper);
+          blockletIndexWrapper = get(identifierWrapper, segInfoCache);
+          blockletIndexWrappers.add(blockletIndexWrapper);
         }
       }
     } catch (Throwable e) {
-      if (null != blockletDataMapIndexWrapper) {
-        List<BlockDataMap> dataMaps = blockletDataMapIndexWrapper.getDataMaps();
-        for (DataMap dataMap : dataMaps) {
-          dataMap.clear();
+      if (null != blockletIndexWrapper) {
+        List<BlockIndex> dataMaps = blockletIndexWrapper.getDataMaps();
+        for (Index index : dataMaps) {
+          index.clear();
         }
       }
       throw new IOException("Problem in loading segment blocks: " + e.getMessage(), e);
     }
 
-    return blockletDataMapIndexWrappers;
+    return blockletIndexWrappers;
   }
 
   /**
@@ -212,9 +212,9 @@ public class BlockletDataMapIndexStore
    * @return
    */
   @Override
-  public BlockletDataMapIndexWrapper getIfPresent(
+  public BlockletIndexWrapper getIfPresent(
       TableBlockIndexUniqueIdentifierWrapper tableSegmentUniqueIdentifierWrapper) {
-    return (BlockletDataMapIndexWrapper) lruCache.get(
+    return (BlockletIndexWrapper) lruCache.get(
         tableSegmentUniqueIdentifierWrapper.getTableBlockIndexUniqueIdentifier()
             .getUniqueTableSegmentIdentifier());
   }
@@ -227,11 +227,11 @@ public class BlockletDataMapIndexStore
   @Override
   public void invalidate(
       TableBlockIndexUniqueIdentifierWrapper tableSegmentUniqueIdentifierWrapper) {
-    BlockletDataMapIndexWrapper blockletDataMapIndexWrapper =
+    BlockletIndexWrapper blockletIndexWrapper =
         getIfPresent(tableSegmentUniqueIdentifierWrapper);
-    if (null != blockletDataMapIndexWrapper) {
+    if (null != blockletIndexWrapper) {
       // clear the segmentProperties cache
-      List<BlockDataMap> dataMaps = blockletDataMapIndexWrapper.getDataMaps();
+      List<BlockIndex> dataMaps = blockletIndexWrapper.getDataMaps();
       if (null != dataMaps && !dataMaps.isEmpty()) {
         String segmentId =
             tableSegmentUniqueIdentifierWrapper.getTableBlockIndexUniqueIdentifier().getSegmentId();
@@ -248,15 +248,15 @@ public class BlockletDataMapIndexStore
 
   @Override
   public void put(TableBlockIndexUniqueIdentifierWrapper tableBlockIndexUniqueIdentifierWrapper,
-      BlockletDataMapIndexWrapper wrapper) throws IOException {
+      BlockletIndexWrapper wrapper) throws IOException {
     // As dataMap will use unsafe memory, it is not recommended to overwrite an existing entry
     // as in that case clearing unsafe memory need to be taken card. If at all datamap entry
     // in the cache need to be overwritten then use the invalidate interface
     // and then use the put interface
     if (null == getIfPresent(tableBlockIndexUniqueIdentifierWrapper)) {
-      List<BlockDataMap> dataMaps = wrapper.getDataMaps();
+      List<BlockIndex> dataMaps = wrapper.getDataMaps();
       try {
-        for (BlockDataMap blockletDataMap : dataMaps) {
+        for (BlockIndex blockletDataMap : dataMaps) {
           blockletDataMap.convertToUnsafeDMStore();
         }
         // get cacheExpirationTime for table from tableProperties
@@ -268,8 +268,8 @@ public class BlockletDataMapIndexStore
             .getUniqueTableSegmentIdentifier(), wrapper, wrapper.getMemorySize(), expiration_time);
       } catch (Throwable e) {
         // clear all the memory acquired by data map in case of any failure
-        for (DataMap blockletDataMap : dataMaps) {
-          blockletDataMap.clear();
+        for (Index blockletIndex : dataMaps) {
+          blockletIndex.clear();
         }
         throw new IOException("Problem in adding datamap to cache.", e);
       }
@@ -285,7 +285,7 @@ public class BlockletDataMapIndexStore
    * @return map of taks id to segment mapping
    * @throws IOException
    */
-  private BlockDataMap loadAndGetDataMap(TableBlockIndexUniqueIdentifier identifier,
+  private BlockIndex loadAndGetDataMap(TableBlockIndexUniqueIdentifier identifier,
       SegmentIndexFileStore indexFileStore, Map<String, BlockMetaInfo> blockMetaInfoMap,
       CarbonTable carbonTable, boolean addTableBlockToUnsafe, Configuration configuration,
       boolean serializeDmStore, List<DataFileFooter> indexInfos) throws IOException {
@@ -295,10 +295,10 @@ public class BlockletDataMapIndexStore
     if (lock == null) {
       lock = addAndGetSegmentLock(uniqueTableSegmentIdentifier);
     }
-    BlockDataMap dataMap;
+    BlockIndex dataMap;
     synchronized (lock) {
-      dataMap = (BlockDataMap) BlockletDataMapFactory.createDataMap(carbonTable);
-      final BlockletDataMapModel blockletDataMapModel = new BlockletDataMapModel(carbonTable,
+      dataMap = (BlockIndex) BlockletIndexFactory.createDataMap(carbonTable);
+      final BlockletIndexModel blockletDataMapModel = new BlockletIndexModel(carbonTable,
           identifier.getIndexFilePath() + CarbonCommonConstants.FILE_SEPARATOR + identifier
               .getIndexFileName(), indexFileStore.getFileData(identifier.getIndexFileName()),
           blockMetaInfoMap, identifier.getSegmentId(), addTableBlockToUnsafe, configuration,
@@ -336,7 +336,7 @@ public class BlockletDataMapIndexStore
       List<TableBlockIndexUniqueIdentifierWrapper> tableSegmentUniqueIdentifiersWrapper) {
     for (TableBlockIndexUniqueIdentifierWrapper
              identifierWrapper : tableSegmentUniqueIdentifiersWrapper) {
-      BlockDataMap cacheable = (BlockDataMap) lruCache.get(
+      BlockIndex cacheable = (BlockIndex) lruCache.get(
           identifierWrapper.getTableBlockIndexUniqueIdentifier().getUniqueTableSegmentIdentifier());
       cacheable.clear();
     }
diff --git a/core/src/main/java/org/apache/carbondata/core/indexstore/BlockletDataMapIndexWrapper.java b/core/src/main/java/org/apache/carbondata/core/indexstore/BlockletIndexWrapper.java
similarity index 80%
rename from core/src/main/java/org/apache/carbondata/core/indexstore/BlockletDataMapIndexWrapper.java
rename to core/src/main/java/org/apache/carbondata/core/indexstore/BlockletIndexWrapper.java
index b74f61f..747b884 100644
--- a/core/src/main/java/org/apache/carbondata/core/indexstore/BlockletDataMapIndexWrapper.java
+++ b/core/src/main/java/org/apache/carbondata/core/indexstore/BlockletIndexWrapper.java
@@ -21,29 +21,29 @@ import java.io.Serializable;
 import java.util.List;
 
 import org.apache.carbondata.core.cache.Cacheable;
-import org.apache.carbondata.core.datamap.dev.DataMap;
-import org.apache.carbondata.core.indexstore.blockletindex.BlockDataMap;
+import org.apache.carbondata.core.datamap.dev.Index;
+import org.apache.carbondata.core.indexstore.blockletindex.BlockIndex;
 
 /**
  * A cacheable wrapper of datamaps
  */
-public class BlockletDataMapIndexWrapper implements Cacheable, Serializable {
+public class BlockletIndexWrapper implements Cacheable, Serializable {
 
   private static final long serialVersionUID = -2859075086955465810L;
 
-  private List<BlockDataMap> dataMaps;
+  private List<BlockIndex> dataMaps;
 
   private String segmentId;
 
   // size of the wrapper. basically the total size of the datamaps this wrapper is holding
   private long wrapperSize;
 
-  public BlockletDataMapIndexWrapper(String segmentId, List<BlockDataMap> dataMaps) {
+  public BlockletIndexWrapper(String segmentId, List<BlockIndex> dataMaps) {
     this.dataMaps = dataMaps;
     this.wrapperSize = 0L;
     this.segmentId = segmentId;
     // add the size of each and every datamap in this wrapper
-    for (BlockDataMap dataMap : dataMaps) {
+    for (BlockIndex dataMap : dataMaps) {
       this.wrapperSize += dataMap.getMemorySize();
     }
   }
@@ -60,12 +60,12 @@ public class BlockletDataMapIndexWrapper implements Cacheable, Serializable {
 
   @Override
   public void invalidate() {
-    for (DataMap dataMap : dataMaps) {
-      dataMap.clear();
+    for (Index index : dataMaps) {
+      index.clear();
     }
   }
 
-  public List<BlockDataMap> getDataMaps() {
+  public List<BlockIndex> getDataMaps() {
     return dataMaps;
   }
 
diff --git a/core/src/main/java/org/apache/carbondata/core/indexstore/ExtendedBlocklet.java b/core/src/main/java/org/apache/carbondata/core/indexstore/ExtendedBlocklet.java
index 5d61776..37075c8 100644
--- a/core/src/main/java/org/apache/carbondata/core/indexstore/ExtendedBlocklet.java
+++ b/core/src/main/java/org/apache/carbondata/core/indexstore/ExtendedBlocklet.java
@@ -25,7 +25,7 @@ import java.util.List;
 import java.util.Map;
 
 import org.apache.carbondata.core.datamap.Segment;
-import org.apache.carbondata.core.indexstore.row.DataMapRow;
+import org.apache.carbondata.core.indexstore.row.IndexRow;
 import org.apache.carbondata.core.metadata.ColumnarFormatVersion;
 import org.apache.carbondata.core.metadata.schema.table.column.ColumnSchema;
 import org.apache.carbondata.core.stream.ExtendedByteArrayOutputStream;
@@ -62,8 +62,8 @@ public class ExtendedBlocklet extends Blocklet {
     return this.inputSplit.getDetailInfo();
   }
 
-  public void setDataMapRow(DataMapRow dataMapRow) {
-    this.inputSplit.setDataMapRow(dataMapRow);
+  public void setDataMapRow(IndexRow indexRow) {
+    this.inputSplit.setIndexRow(indexRow);
   }
 
   public String[] getLocations() {
diff --git a/core/src/main/java/org/apache/carbondata/core/indexstore/SafeMemoryDMStore.java b/core/src/main/java/org/apache/carbondata/core/indexstore/SafeMemoryDMStore.java
index 70ae7b2..914ea5a 100644
--- a/core/src/main/java/org/apache/carbondata/core/indexstore/SafeMemoryDMStore.java
+++ b/core/src/main/java/org/apache/carbondata/core/indexstore/SafeMemoryDMStore.java
@@ -21,46 +21,46 @@ import java.util.ArrayList;
 import java.util.List;
 
 import org.apache.carbondata.core.constants.CarbonCommonConstants;
-import org.apache.carbondata.core.indexstore.row.DataMapRow;
+import org.apache.carbondata.core.indexstore.row.IndexRow;
 import org.apache.carbondata.core.indexstore.schema.CarbonRowSchema;
 import org.apache.carbondata.core.util.DataTypeUtil;
 
 /**
- * Store the data map row @{@link DataMapRow} data to memory.
+ * Store the data map row @{@link IndexRow} data to memory.
  */
 public class SafeMemoryDMStore extends AbstractMemoryDMStore {
 
   /**
    * holds all blocklets metadata in memory
    */
-  private List<DataMapRow> dataMapRows =
+  private List<IndexRow> indexRows =
       new ArrayList<>(CarbonCommonConstants.DEFAULT_COLLECTION_SIZE);
 
   private int runningLength;
 
   /**
-   * Add the index row to dataMapRows, basically to in memory.
+   * Add the index row to indexRows, basically to in memory.
    *
    * @param indexRow
    * @return
    */
   @Override
-  public void addIndexRow(CarbonRowSchema[] schema, DataMapRow indexRow) {
-    dataMapRows.add(indexRow);
+  public void addIndexRow(CarbonRowSchema[] schema, IndexRow indexRow) {
+    indexRows.add(indexRow);
     runningLength += indexRow.getTotalSizeInBytes();
   }
 
   @Override
-  public DataMapRow getDataMapRow(CarbonRowSchema[] schema, int index) {
-    assert (index < dataMapRows.size());
-    return dataMapRows.get(index);
+  public IndexRow getDataMapRow(CarbonRowSchema[] schema, int index) {
+    assert (index < indexRows.size());
+    return indexRows.get(index);
   }
 
   @Override
   public void freeMemory() {
     if (!isMemoryFreed) {
-      if (null != dataMapRows) {
-        dataMapRows.clear();
+      if (null != indexRows) {
+        indexRows.clear();
       }
       isMemoryFreed = true;
     }
@@ -73,16 +73,16 @@ public class SafeMemoryDMStore extends AbstractMemoryDMStore {
 
   @Override
   public int getRowCount() {
-    return dataMapRows.size();
+    return indexRows.size();
   }
 
   @Override
   public UnsafeMemoryDMStore convertToUnsafeDMStore(CarbonRowSchema[] schema) {
     setSchemaDataType(schema);
     UnsafeMemoryDMStore unsafeMemoryDMStore = new UnsafeMemoryDMStore();
-    for (DataMapRow dataMapRow : dataMapRows) {
-      dataMapRow.setSchemas(schema);
-      unsafeMemoryDMStore.addIndexRow(schema, dataMapRow);
+    for (IndexRow indexRow : indexRows) {
+      indexRow.setSchemas(schema);
+      unsafeMemoryDMStore.addIndexRow(schema, indexRow);
     }
     unsafeMemoryDMStore.finishWriting();
     return unsafeMemoryDMStore;
diff --git a/core/src/main/java/org/apache/carbondata/core/indexstore/SegmentPropertiesFetcher.java b/core/src/main/java/org/apache/carbondata/core/indexstore/SegmentPropertiesFetcher.java
index b4ac3e9..70db70c 100644
--- a/core/src/main/java/org/apache/carbondata/core/indexstore/SegmentPropertiesFetcher.java
+++ b/core/src/main/java/org/apache/carbondata/core/indexstore/SegmentPropertiesFetcher.java
@@ -21,7 +21,7 @@ import java.io.IOException;
 import java.util.List;
 
 import org.apache.carbondata.core.datamap.Segment;
-import org.apache.carbondata.core.datamap.dev.DataMap;
+import org.apache.carbondata.core.datamap.dev.Index;
 import org.apache.carbondata.core.datastore.block.SegmentProperties;
 
 /**
@@ -41,5 +41,5 @@ public interface SegmentPropertiesFetcher {
   SegmentProperties getSegmentProperties(Segment segment, List<PartitionSpec> partitionSpecs)
       throws IOException;
 
-  SegmentProperties getSegmentPropertiesFromDataMap(DataMap coarseGrainDataMap);
+  SegmentProperties getSegmentPropertiesFromDataMap(Index coarseGrainIndex);
 }
diff --git a/core/src/main/java/org/apache/carbondata/core/indexstore/UnsafeMemoryDMStore.java b/core/src/main/java/org/apache/carbondata/core/indexstore/UnsafeMemoryDMStore.java
index 0dc8b36..15b67d7 100644
--- a/core/src/main/java/org/apache/carbondata/core/indexstore/UnsafeMemoryDMStore.java
+++ b/core/src/main/java/org/apache/carbondata/core/indexstore/UnsafeMemoryDMStore.java
@@ -18,8 +18,8 @@
 package org.apache.carbondata.core.indexstore;
 
 import org.apache.carbondata.core.constants.CarbonCommonConstants;
-import org.apache.carbondata.core.indexstore.row.DataMapRow;
-import org.apache.carbondata.core.indexstore.row.UnsafeDataMapRow;
+import org.apache.carbondata.core.indexstore.row.IndexRow;
+import org.apache.carbondata.core.indexstore.row.UnsafeIndexRow;
 import org.apache.carbondata.core.indexstore.schema.CarbonRowSchema;
 import org.apache.carbondata.core.memory.CarbonUnsafe;
 import org.apache.carbondata.core.memory.MemoryBlock;
@@ -32,7 +32,7 @@ import static org.apache.carbondata.core.memory.CarbonUnsafe.BYTE_ARRAY_OFFSET;
 import static org.apache.carbondata.core.memory.CarbonUnsafe.getUnsafe;
 
 /**
- * Store the data map row @{@link DataMapRow} data to unsafe.
+ * Store the data map row @{@link IndexRow} data to unsafe.
  */
 public class UnsafeMemoryDMStore extends AbstractMemoryDMStore {
 
@@ -116,7 +116,7 @@ public class UnsafeMemoryDMStore extends AbstractMemoryDMStore {
    *
    * @param indexRow
    */
-  public void addIndexRow(CarbonRowSchema[] schema, DataMapRow indexRow) {
+  public void addIndexRow(CarbonRowSchema[] schema, IndexRow indexRow) {
     // First calculate the required memory to keep the row in unsafe
     int rowSize = indexRow.getTotalSizeInBytes();
     // Check whether allocated memory is sufficient or not.
@@ -151,7 +151,7 @@ public class UnsafeMemoryDMStore extends AbstractMemoryDMStore {
         case STRUCT:
           CarbonRowSchema[] childSchemas =
               ((CarbonRowSchema.StructCarbonRowSchema) schema[i]).getChildSchemas();
-          DataMapRow row = indexRow.getRow(i);
+          IndexRow row = indexRow.getRow(i);
           for (int j = 0; j < childSchemas.length; j++) {
             currentPosition = addToUnsafe(childSchemas[j], row, j, pointer, varColPosition);
             if (currentPosition > 0) {
@@ -177,7 +177,7 @@ public class UnsafeMemoryDMStore extends AbstractMemoryDMStore {
     pointers[rowCount++] = pointer;
   }
 
-  private int addToUnsafe(CarbonRowSchema schema, DataMapRow row, int index, int startOffset,
+  private int addToUnsafe(CarbonRowSchema schema, IndexRow row, int index, int startOffset,
       int varPosition) {
     switch (schema.getSchemaType()) {
       case FIXED:
@@ -246,9 +246,9 @@ public class UnsafeMemoryDMStore extends AbstractMemoryDMStore {
     }
   }
 
-  public DataMapRow getDataMapRow(CarbonRowSchema[] schema, int index) {
+  public IndexRow getDataMapRow(CarbonRowSchema[] schema, int index) {
     assert (index < rowCount);
-    return new UnsafeDataMapRow(schema, memoryBlock, pointers[index]);
+    return new UnsafeIndexRow(schema, memoryBlock, pointers[index]);
   }
 
   public void finishWriting() {
diff --git a/core/src/main/java/org/apache/carbondata/core/indexstore/blockletindex/BlockDataMap.java b/core/src/main/java/org/apache/carbondata/core/indexstore/blockletindex/BlockIndex.java
similarity index 89%
rename from core/src/main/java/org/apache/carbondata/core/indexstore/blockletindex/BlockDataMap.java
rename to core/src/main/java/org/apache/carbondata/core/indexstore/blockletindex/BlockIndex.java
index 47a2ca3..9198b6b 100644
--- a/core/src/main/java/org/apache/carbondata/core/indexstore/blockletindex/BlockDataMap.java
+++ b/core/src/main/java/org/apache/carbondata/core/indexstore/blockletindex/BlockIndex.java
@@ -30,10 +30,10 @@ import java.util.Map;
 
 import org.apache.carbondata.common.logging.LogServiceFactory;
 import org.apache.carbondata.core.constants.CarbonCommonConstants;
-import org.apache.carbondata.core.datamap.DataMapFilter;
+import org.apache.carbondata.core.datamap.IndexFilter;
 import org.apache.carbondata.core.datamap.Segment;
-import org.apache.carbondata.core.datamap.dev.DataMapModel;
-import org.apache.carbondata.core.datamap.dev.cgdatamap.CoarseGrainDataMap;
+import org.apache.carbondata.core.datamap.dev.IndexModel;
+import org.apache.carbondata.core.datamap.dev.cgdatamap.CoarseGrainIndex;
 import org.apache.carbondata.core.datastore.block.SegmentProperties;
 import org.apache.carbondata.core.datastore.block.SegmentPropertiesAndSchemaHolder;
 import org.apache.carbondata.core.datastore.block.TableBlockInfo;
@@ -45,8 +45,8 @@ import org.apache.carbondata.core.indexstore.ExtendedBlocklet;
 import org.apache.carbondata.core.indexstore.PartitionSpec;
 import org.apache.carbondata.core.indexstore.SafeMemoryDMStore;
 import org.apache.carbondata.core.indexstore.UnsafeMemoryDMStore;
-import org.apache.carbondata.core.indexstore.row.DataMapRow;
-import org.apache.carbondata.core.indexstore.row.DataMapRowImpl;
+import org.apache.carbondata.core.indexstore.row.IndexRow;
+import org.apache.carbondata.core.indexstore.row.IndexRowImpl;
 import org.apache.carbondata.core.indexstore.schema.CarbonRowSchema;
 import org.apache.carbondata.core.metadata.ColumnarFormatVersion;
 import org.apache.carbondata.core.metadata.blocklet.DataFileFooter;
@@ -61,7 +61,7 @@ import org.apache.carbondata.core.scan.filter.FilterUtil;
 import org.apache.carbondata.core.scan.filter.executer.FilterExecuter;
 import org.apache.carbondata.core.scan.filter.executer.ImplicitColumnFilterExecutor;
 import org.apache.carbondata.core.scan.filter.resolver.FilterResolverIntf;
-import org.apache.carbondata.core.util.BlockletDataMapUtil;
+import org.apache.carbondata.core.util.BlockletIndexUtil;
 import org.apache.carbondata.core.util.ByteUtil;
 import org.apache.carbondata.core.util.DataFileFooterConverter;
 import org.apache.carbondata.core.util.path.CarbonTablePath;
@@ -74,11 +74,11 @@ import org.apache.log4j.Logger;
 /**
  * Datamap implementation for block.
  */
-public class BlockDataMap extends CoarseGrainDataMap
-    implements BlockletDataMapRowIndexes, Serializable {
+public class BlockIndex extends CoarseGrainIndex
+    implements BlockletIndexRowIndexes, Serializable {
 
   private static final Logger LOGGER =
-      LogServiceFactory.getLogService(BlockDataMap.class.getName());
+      LogServiceFactory.getLogService(BlockIndex.class.getName());
 
   protected static final long serialVersionUID = -2170289352240810993L;
   /**
@@ -109,12 +109,12 @@ public class BlockDataMap extends CoarseGrainDataMap
   protected boolean isPartitionTable;
 
   @Override
-  public void init(DataMapModel dataMapModel) throws IOException {
+  public void init(IndexModel indexModel) throws IOException {
     long startTime = System.currentTimeMillis();
-    assert (dataMapModel instanceof BlockletDataMapModel);
-    BlockletDataMapModel blockletDataMapInfo = (BlockletDataMapModel) dataMapModel;
+    assert (indexModel instanceof BlockletIndexModel);
+    BlockletIndexModel blockletDataMapInfo = (BlockletIndexModel) indexModel;
     DataFileFooterConverter fileFooterConverter =
-        new DataFileFooterConverter(dataMapModel.getConfiguration());
+        new DataFileFooterConverter(indexModel.getConfiguration());
     List<DataFileFooter> indexInfo = null;
     if (blockletDataMapInfo.getIndexInfos() == null || blockletDataMapInfo.getIndexInfos()
         .isEmpty()) {
@@ -153,22 +153,22 @@ public class BlockDataMap extends CoarseGrainDataMap
       createSummaryDMStore(blockletDataMapInfo);
       CarbonRowSchema[] taskSummarySchema = getTaskSummarySchema();
       // check for legacy store and load the metadata
-      DataMapRowImpl summaryRow =
+      IndexRowImpl summaryRow =
           loadMetadata(taskSummarySchema, segmentProperties, blockletDataMapInfo, indexInfo);
       finishWriting(taskSummarySchema, filePath, fileName, segmentId, summaryRow);
-      if (((BlockletDataMapModel) dataMapModel).isSerializeDmStore()) {
+      if (((BlockletIndexModel) indexModel).isSerializeDmStore()) {
         serializeDmStore();
       }
     }
     if (LOGGER.isDebugEnabled()) {
       LOGGER.debug(
-          "Time taken to load blocklet datamap from file : " + dataMapModel.getFilePath() + " is "
+          "Time taken to load blocklet datamap from file : " + indexModel.getFilePath() + " is "
               + (System.currentTimeMillis() - startTime));
     }
   }
 
   private void finishWriting(CarbonRowSchema[] taskSummarySchema, byte[] filePath, byte[] fileName,
-      byte[] segmentId, DataMapRowImpl summaryRow) {
+      byte[] segmentId, IndexRowImpl summaryRow) {
     if (memoryDMStore != null) {
       memoryDMStore.finishWriting();
     }
@@ -195,8 +195,8 @@ public class BlockDataMap extends CoarseGrainDataMap
    * @param blockletDataMapInfo
    * @param indexInfo
    */
-  protected DataMapRowImpl loadMetadata(CarbonRowSchema[] taskSummarySchema,
-      SegmentProperties segmentProperties, BlockletDataMapModel blockletDataMapInfo,
+  protected IndexRowImpl loadMetadata(CarbonRowSchema[] taskSummarySchema,
+      SegmentProperties segmentProperties, BlockletIndexModel blockletDataMapInfo,
       List<DataFileFooter> indexInfo) {
     return loadBlockMetaInfo(taskSummarySchema, segmentProperties, blockletDataMapInfo, indexInfo);
   }
@@ -207,7 +207,7 @@ public class BlockDataMap extends CoarseGrainDataMap
    * @param fileFooter
    * @throws IOException
    */
-  private SegmentProperties initSegmentProperties(BlockletDataMapModel blockletDataMapInfo,
+  private SegmentProperties initSegmentProperties(BlockletIndexModel blockletDataMapInfo,
       DataFileFooter fileFooter) {
     List<ColumnSchema> columnInTable = fileFooter.getColumnInTable();
     segmentPropertiesWrapper = SegmentPropertiesAndSchemaHolder.getInstance()
@@ -216,11 +216,11 @@ public class BlockDataMap extends CoarseGrainDataMap
     return segmentPropertiesWrapper.getSegmentProperties();
   }
 
-  protected void setMinMaxFlagForTaskSummary(DataMapRow summaryRow,
+  protected void setMinMaxFlagForTaskSummary(IndexRow summaryRow,
       CarbonRowSchema[] taskSummarySchema, SegmentProperties segmentProperties,
       boolean[] minMaxFlag) {
     // add min max flag for all the dimension columns
-    boolean[] minMaxFlagValuesForColumnsToBeCached = BlockletDataMapUtil
+    boolean[] minMaxFlagValuesForColumnsToBeCached = BlockletIndexUtil
         .getMinMaxFlagValuesForColumnsToBeCached(segmentProperties, getMinMaxCacheColumns(),
             minMaxFlag);
     addMinMaxFlagValues(summaryRow, taskSummarySchema[TASK_MIN_MAX_FLAG],
@@ -233,15 +233,15 @@ public class BlockDataMap extends CoarseGrainDataMap
    * @param blockletDataMapInfo
    * @param indexInfo
    */
-  private DataMapRowImpl loadBlockMetaInfo(CarbonRowSchema[] taskSummarySchema,
-      SegmentProperties segmentProperties, BlockletDataMapModel blockletDataMapInfo,
+  private IndexRowImpl loadBlockMetaInfo(CarbonRowSchema[] taskSummarySchema,
+      SegmentProperties segmentProperties, BlockletIndexModel blockletDataMapInfo,
       List<DataFileFooter> indexInfo) {
     String tempFilePath = null;
     DataFileFooter previousDataFileFooter = null;
     int footerCounter = 0;
     byte[][] blockMinValues = null;
     byte[][] blockMaxValues = null;
-    DataMapRowImpl summaryRow = null;
+    IndexRowImpl summaryRow = null;
     List<Short> blockletCountInEachBlock = new ArrayList<>(indexInfo.size());
     short totalBlockletsInOneBlock = 0;
     boolean isLastFileFooterEntryNeedToBeAdded = false;
@@ -259,7 +259,7 @@ public class BlockDataMap extends CoarseGrainDataMap
           blockletDataMapInfo.getBlockMetaInfoMap().get(blockInfo.getFilePath());
       footerCounter++;
       if (blockMetaInfo != null) {
-        // this variable will be used for adding the DataMapRow entry every time a unique block
+        // this variable will be used for adding the IndexRow entry every time a unique block
         // path is encountered
         if (null == tempFilePath) {
           tempFilePath = blockInfo.getFilePath();
@@ -339,7 +339,7 @@ public class BlockDataMap extends CoarseGrainDataMap
   }
 
   protected void updateMinMaxFlag(DataFileFooter fileFooter, boolean[] minMaxFlag) {
-    BlockletDataMapUtil
+    BlockletIndexUtil
         .updateMinMaxFlag(fileFooter.getBlockletIndex().getMinMaxIndex(), minMaxFlag);
   }
 
@@ -353,7 +353,7 @@ public class BlockDataMap extends CoarseGrainDataMap
     return byteBuffer.array();
   }
 
-  protected void setLocations(String[] locations, DataMapRow row, int ordinal)
+  protected void setLocations(String[] locations, IndexRow row, int ordinal)
       throws UnsupportedEncodingException {
     // Add location info
     String locationStr = StringUtils.join(locations, ',');
@@ -365,27 +365,27 @@ public class BlockDataMap extends CoarseGrainDataMap
    * where blocklet information is not available in index file. So load only block information
    * and read blocklet information in executor.
    */
-  protected DataMapRowImpl loadToUnsafeBlock(CarbonRowSchema[] schema,
+  protected IndexRowImpl loadToUnsafeBlock(CarbonRowSchema[] schema,
       CarbonRowSchema[] taskSummarySchema, DataFileFooter fileFooter,
       SegmentProperties segmentProperties, List<CarbonColumn> minMaxCacheColumns, String filePath,
-      DataMapRowImpl summaryRow, BlockMetaInfo blockMetaInfo, byte[][] minValues,
+      IndexRowImpl summaryRow, BlockMetaInfo blockMetaInfo, byte[][] minValues,
       byte[][] maxValues, boolean[] minMaxFlag) {
     // Add one row to maintain task level min max for segment pruning
     if (summaryRow == null) {
-      summaryRow = new DataMapRowImpl(taskSummarySchema);
+      summaryRow = new IndexRowImpl(taskSummarySchema);
     }
-    DataMapRow row = new DataMapRowImpl(schema);
+    IndexRow row = new IndexRowImpl(schema);
     int ordinal = 0;
     int taskMinMaxOrdinal = 1;
     // get min max values for columns to be cached
-    byte[][] minValuesForColumnsToBeCached = BlockletDataMapUtil
+    byte[][] minValuesForColumnsToBeCached = BlockletIndexUtil
         .getMinMaxForColumnsToBeCached(segmentProperties, minMaxCacheColumns, minValues);
-    byte[][] maxValuesForColumnsToBeCached = BlockletDataMapUtil
+    byte[][] maxValuesForColumnsToBeCached = BlockletIndexUtil
         .getMinMaxForColumnsToBeCached(segmentProperties, minMaxCacheColumns, maxValues);
-    boolean[] minMaxFlagValuesForColumnsToBeCached = BlockletDataMapUtil
+    boolean[] minMaxFlagValuesForColumnsToBeCached = BlockletIndexUtil
         .getMinMaxFlagValuesForColumnsToBeCached(segmentProperties, minMaxCacheColumns, minMaxFlag);
-    DataMapRow dataMapRow = addMinMax(schema[ordinal], minValuesForColumnsToBeCached);
-    row.setRow(dataMapRow, ordinal);
+    IndexRow indexRow = addMinMax(schema[ordinal], minValuesForColumnsToBeCached);
+    row.setRow(indexRow, ordinal);
     // compute and set task level min values
     addTaskMinMaxValues(summaryRow, taskSummarySchema, taskMinMaxOrdinal,
         minValuesForColumnsToBeCached, TASK_MIN_VALUES_INDEX, true);
@@ -424,11 +424,11 @@ public class BlockDataMap extends CoarseGrainDataMap
     return summaryRow;
   }
 
-  protected void addMinMaxFlagValues(DataMapRow row, CarbonRowSchema carbonRowSchema,
+  protected void addMinMaxFlagValues(IndexRow row, CarbonRowSchema carbonRowSchema,
       boolean[] minMaxFlag, int ordinal) {
     CarbonRowSchema[] minMaxFlagSchema =
         ((CarbonRowSchema.StructCarbonRowSchema) carbonRowSchema).getChildSchemas();
-    DataMapRow minMaxFlagRow = new DataMapRowImpl(minMaxFlagSchema);
+    IndexRow minMaxFlagRow = new IndexRowImpl(minMaxFlagSchema);
     int flagOrdinal = 0;
     // min value adding
     for (int i = 0; i < minMaxFlag.length; i++) {
@@ -447,15 +447,15 @@ public class BlockDataMap extends CoarseGrainDataMap
     return CarbonTablePath.getSegmentPath(tablePath, segmentId);
   }
 
-  protected String getFileNameWithFilePath(DataMapRow dataMapRow, String filePath) {
+  protected String getFileNameWithFilePath(IndexRow indexRow, String filePath) {
     String fileName = filePath + CarbonCommonConstants.FILE_SEPARATOR + new String(
-        dataMapRow.getByteArray(FILE_PATH_INDEX), CarbonCommonConstants.DEFAULT_CHARSET_CLASS)
+        indexRow.getByteArray(FILE_PATH_INDEX), CarbonCommonConstants.DEFAULT_CHARSET_CLASS)
         + CarbonTablePath.getCarbonDataExtension();
     return FileFactory.getUpdatedFilePath(fileName);
   }
 
   private void addTaskSummaryRowToUnsafeMemoryStore(CarbonRowSchema[] taskSummarySchema,
-      DataMapRow summaryRow, byte[] filePath, byte[] fileName, byte[] segmentId) {
+      IndexRow summaryRow, byte[] filePath, byte[] fileName, byte[] segmentId) {
     // write the task summary info to unsafe memory store
     if (null != summaryRow) {
       summaryRow.setByteArray(fileName, SUMMARY_INDEX_FILE_NAME);
@@ -471,11 +471,11 @@ public class BlockDataMap extends CoarseGrainDataMap
     }
   }
 
-  protected DataMapRow addMinMax(CarbonRowSchema carbonRowSchema,
+  protected IndexRow addMinMax(CarbonRowSchema carbonRowSchema,
       byte[][] minValues) {
     CarbonRowSchema[] minSchemas =
         ((CarbonRowSchema.StructCarbonRowSchema) carbonRowSchema).getChildSchemas();
-    DataMapRow minRow = new DataMapRowImpl(minSchemas);
+    IndexRow minRow = new IndexRowImpl(minSchemas);
     int minOrdinal = 0;
     // min value adding
     for (int i = 0; i < minValues.length; i++) {
@@ -494,15 +494,15 @@ public class BlockDataMap extends CoarseGrainDataMap
    * @param ordinal
    * @param isMinValueComparison
    */
-  protected void addTaskMinMaxValues(DataMapRow taskMinMaxRow, CarbonRowSchema[] carbonRowSchema,
+  protected void addTaskMinMaxValues(IndexRow taskMinMaxRow, CarbonRowSchema[] carbonRowSchema,
       int taskMinMaxOrdinal, byte[][] minMaxValue, int ordinal, boolean isMinValueComparison) {
-    DataMapRow row = taskMinMaxRow.getRow(ordinal);
+    IndexRow row = taskMinMaxRow.getRow(ordinal);
     byte[][] updatedMinMaxValues = null;
     if (null == row) {
       CarbonRowSchema[] minSchemas =
           ((CarbonRowSchema.StructCarbonRowSchema) carbonRowSchema[taskMinMaxOrdinal])
               .getChildSchemas();
-      row = new DataMapRowImpl(minSchemas);
+      row = new IndexRowImpl(minSchemas);
       updatedMinMaxValues = minMaxValue;
     } else {
       byte[][] existingMinMaxValues = getMinMaxValue(taskMinMaxRow, ordinal);
@@ -543,7 +543,7 @@ public class BlockDataMap extends CoarseGrainDataMap
     return updatedMinMaxValues;
   }
 
-  protected void createMemorySchema(BlockletDataMapModel blockletDataMapModel) {
+  protected void createMemorySchema(BlockletIndexModel blockletDataMapModel) {
     memoryDMStore = getMemoryDMStore(blockletDataMapModel.isAddToUnsafe());
   }
 
@@ -553,7 +553,7 @@ public class BlockDataMap extends CoarseGrainDataMap
    * datamap
    *
    */
-  protected void createSummaryDMStore(BlockletDataMapModel blockletDataMapModel) {
+  protected void createSummaryDMStore(BlockletIndexModel blockletDataMapModel) {
     taskSummaryDMStore = getMemoryDMStore(blockletDataMapModel.isAddToUnsafe());
   }
 
@@ -561,7 +561,7 @@ public class BlockDataMap extends CoarseGrainDataMap
   public boolean isScanRequired(FilterResolverIntf filterExp) {
     FilterExecuter filterExecuter = FilterUtil.getFilterExecuterTree(
         filterExp, getSegmentProperties(), null, getMinMaxCacheColumns(), false);
-    DataMapRow unsafeRow = taskSummaryDMStore
+    IndexRow unsafeRow = taskSummaryDMStore
         .getDataMapRow(getTaskSummarySchema(), taskSummaryDMStore.getRowCount() - 1);
     boolean isScanRequired = FilterExpressionProcessor
         .isScanRequired(filterExecuter, getMinMaxValue(unsafeRow, TASK_MAX_VALUES_INDEX),
@@ -640,10 +640,10 @@ public class BlockDataMap extends CoarseGrainDataMap
     CarbonRowSchema[] schema = getFileFooterEntrySchema();
     int numEntries = memoryDMStore.getRowCount();
     for (int i = 0; i < numEntries; i++) {
-      DataMapRow dataMapRow = memoryDMStore.getDataMapRow(schema, i);
-      String fileName = new String(dataMapRow.getByteArray(FILE_PATH_INDEX),
+      IndexRow indexRow = memoryDMStore.getDataMapRow(schema, i);
+      String fileName = new String(indexRow.getByteArray(FILE_PATH_INDEX),
           CarbonCommonConstants.DEFAULT_CHARSET_CLASS) + CarbonTablePath.getCarbonDataExtension();
-      int rowCount = dataMapRow.getInt(ROW_COUNT_INDEX);
+      int rowCount = indexRow.getInt(ROW_COUNT_INDEX);
       // prepend segment number with the blocklet file path
       String blockletMapKey = segment.getSegmentNo() + "," + fileName;
       Long existingCount = blockletToRowCountMap.get(blockletMapKey);
@@ -672,16 +672,16 @@ public class BlockDataMap extends CoarseGrainDataMap
     int hitBlocklets = 0;
     if (filterExp == null) {
       for (int i = 0; i < numEntries; i++) {
-        DataMapRow dataMapRow = memoryDMStore.getDataMapRow(schema, i);
-        blocklets.add(createBlocklet(dataMapRow, getFileNameWithFilePath(dataMapRow, filePath),
-            getBlockletId(dataMapRow), false));
+        IndexRow indexRow = memoryDMStore.getDataMapRow(schema, i);
+        blocklets.add(createBlocklet(indexRow, getFileNameWithFilePath(indexRow, filePath),
+            getBlockletId(indexRow), false));
       }
       hitBlocklets = totalBlocklets;
     } else {
       // Remove B-tree jump logic as start and end key prepared is not
       // correct for old store scenarios
       int entryIndex = 0;
-      // flag to be used for deciding whether use min/max in executor pruning for BlockletDataMap
+      // flag to be used for deciding whether use min/max in executor pruning for BlockletIndex
       boolean useMinMaxForPruning = useMinMaxForExecutorPruning(filterExp);
       if (!validateSegmentProperties(segmentProperties)) {
         filterExecuter = FilterUtil
@@ -690,7 +690,7 @@ public class BlockDataMap extends CoarseGrainDataMap
       }
       // min and max for executor pruning
       while (entryIndex < numEntries) {
-        DataMapRow row = memoryDMStore.getDataMapRow(schema, entryIndex);
+        IndexRow row = memoryDMStore.getDataMapRow(schema, entryIndex);
         boolean[] minMaxFlag = getMinMaxFlag(row, BLOCK_MIN_MAX_FLAG);
         String fileName = getFileNameWithFilePath(row, filePath);
         short blockletId = getBlockletId(row);
@@ -722,7 +722,7 @@ public class BlockDataMap extends CoarseGrainDataMap
   @Override
   public List<Blocklet> prune(Expression expression, SegmentProperties properties,
       List<PartitionSpec> partitions, CarbonTable carbonTable, FilterExecuter filterExecuter) {
-    return prune(new DataMapFilter(properties, carbonTable, expression).getResolver(), properties,
+    return prune(new IndexFilter(properties, carbonTable, expression).getResolver(), properties,
         partitions, filterExecuter, carbonTable);
   }
 
@@ -735,7 +735,7 @@ public class BlockDataMap extends CoarseGrainDataMap
     // Prune with filters if the partitions are existed in this datamap
     // changed segmentProperties to this.segmentProperties to make sure the pruning with its own
     // segmentProperties.
-    // Its a temporary fix. The Interface DataMap.prune(FilterResolverIntf filterExp,
+    // Its a temporary fix. The Interface Index.prune(FilterResolverIntf filterExp,
     // SegmentProperties segmentProperties, List<PartitionSpec> partitions) should be corrected
     return prune(filterExp, filterExecuter, segmentProperties);
   }
@@ -850,7 +850,7 @@ public class BlockDataMap extends CoarseGrainDataMap
         rowIndex++;
       }
     }
-    DataMapRow row =
+    IndexRow row =
         memoryDMStore.getDataMapRow(getFileFooterEntrySchema(), rowIndex);
     String filePath = getFilePath();
     return createBlocklet(row, getFileNameWithFilePath(row, filePath), relativeBlockletId,
@@ -871,7 +871,7 @@ public class BlockDataMap extends CoarseGrainDataMap
    * @return
    */
   public String getTableTaskInfo(int index) {
-    DataMapRow unsafeRow = taskSummaryDMStore.getDataMapRow(getTaskSummarySchema(), 0);
+    IndexRow unsafeRow = taskSummaryDMStore.getDataMapRow(getTaskSummarySchema(), 0);
     try {
       return new String(unsafeRow.getByteArray(index), CarbonCommonConstants.DEFAULT_CHARSET);
     } catch (UnsupportedEncodingException e) {
@@ -880,8 +880,8 @@ public class BlockDataMap extends CoarseGrainDataMap
     }
   }
 
-  private byte[][] getMinMaxValue(DataMapRow row, int index) {
-    DataMapRow minMaxRow = row.getRow(index);
+  private byte[][] getMinMaxValue(IndexRow row, int index) {
+    IndexRow minMaxRow = row.getRow(index);
     byte[][] minMax = new byte[minMaxRow.getColumnCount()][];
     for (int i = 0; i < minMax.length; i++) {
       minMax[i] = minMaxRow.getByteArray(i);
@@ -889,8 +889,8 @@ public class BlockDataMap extends CoarseGrainDataMap
     return minMax;
   }
 
-  private boolean[] getMinMaxFlag(DataMapRow row, int index) {
-    DataMapRow minMaxFlagRow = row.getRow(index);
+  private boolean[] getMinMaxFlag(IndexRow row, int index) {
+    IndexRow minMaxFlagRow = row.getRow(index);
     boolean[] minMaxFlag = new boolean[minMaxFlagRow.getColumnCount()];
     for (int i = 0; i < minMaxFlag.length; i++) {
       minMaxFlag[i] = minMaxFlagRow.getBoolean(i);
@@ -898,11 +898,11 @@ public class BlockDataMap extends CoarseGrainDataMap
     return minMaxFlag;
   }
 
-  protected short getBlockletId(DataMapRow dataMapRow) {
+  protected short getBlockletId(IndexRow indexRow) {
     return BLOCK_DEFAULT_BLOCKLET_ID;
   }
 
-  protected ExtendedBlocklet createBlocklet(DataMapRow row, String fileName, short blockletId,
+  protected ExtendedBlocklet createBlocklet(IndexRow row, String fileName, short blockletId,
       boolean useMinMaxForPruning) {
     short versionNumber = row.getShort(VERSION_INDEX);
     ExtendedBlocklet blocklet = new ExtendedBlocklet(fileName, blockletId + "", false,
@@ -915,7 +915,7 @@ public class BlockDataMap extends CoarseGrainDataMap
   private String[] getFileDetails() {
     try {
       String[] fileDetails = new String[3];
-      DataMapRow unsafeRow = taskSummaryDMStore.getDataMapRow(getTaskSummarySchema(), 0);
+      IndexRow unsafeRow = taskSummaryDMStore.getDataMapRow(getTaskSummarySchema(), 0);
       fileDetails[0] = new String(unsafeRow.getByteArray(SUMMARY_INDEX_PATH),
           CarbonCommonConstants.DEFAULT_CHARSET);
       fileDetails[1] = new String(unsafeRow.getByteArray(SUMMARY_INDEX_FILE_NAME),
diff --git a/core/src/main/java/org/apache/carbondata/core/indexstore/blockletindex/BlockletDataMap.java b/core/src/main/java/org/apache/carbondata/core/indexstore/blockletindex/BlockletIndex.java
similarity index 85%
rename from core/src/main/java/org/apache/carbondata/core/indexstore/blockletindex/BlockletDataMap.java
rename to core/src/main/java/org/apache/carbondata/core/indexstore/blockletindex/BlockletIndex.java
index 9c21748..5ccbf2c 100644
--- a/core/src/main/java/org/apache/carbondata/core/indexstore/blockletindex/BlockletDataMap.java
+++ b/core/src/main/java/org/apache/carbondata/core/indexstore/blockletindex/BlockletIndex.java
@@ -26,13 +26,13 @@ import java.util.Arrays;
 import java.util.List;
 
 import org.apache.carbondata.core.constants.CarbonCommonConstants;
-import org.apache.carbondata.core.datamap.dev.DataMapModel;
+import org.apache.carbondata.core.datamap.dev.IndexModel;
 import org.apache.carbondata.core.datastore.block.SegmentProperties;
 import org.apache.carbondata.core.datastore.block.TableBlockInfo;
 import org.apache.carbondata.core.indexstore.BlockMetaInfo;
 import org.apache.carbondata.core.indexstore.ExtendedBlocklet;
-import org.apache.carbondata.core.indexstore.row.DataMapRow;
-import org.apache.carbondata.core.indexstore.row.DataMapRowImpl;
+import org.apache.carbondata.core.indexstore.row.IndexRow;
+import org.apache.carbondata.core.indexstore.row.IndexRowImpl;
 import org.apache.carbondata.core.indexstore.schema.CarbonRowSchema;
 import org.apache.carbondata.core.metadata.ColumnarFormatVersion;
 import org.apache.carbondata.core.metadata.blocklet.BlockletInfo;
@@ -40,21 +40,21 @@ import org.apache.carbondata.core.metadata.blocklet.DataFileFooter;
 import org.apache.carbondata.core.metadata.blocklet.index.BlockletMinMaxIndex;
 import org.apache.carbondata.core.metadata.schema.table.column.CarbonColumn;
 import org.apache.carbondata.core.scan.filter.resolver.FilterResolverIntf;
-import org.apache.carbondata.core.util.BlockletDataMapUtil;
+import org.apache.carbondata.core.util.BlockletIndexUtil;
 import org.apache.carbondata.core.util.path.CarbonTablePath;
 
 /**
  * Datamap implementation for blocklet.
  */
-public class BlockletDataMap extends BlockDataMap implements Serializable {
+public class BlockletIndex extends BlockIndex implements Serializable {
 
   private static final long serialVersionUID = -2170289352240810993L;
   // total block number in this datamap
   private int blockNum = 0;
 
   @Override
-  public void init(DataMapModel dataMapModel) throws IOException {
-    super.init(dataMapModel);
+  public void init(IndexModel indexModel) throws IOException {
+    super.init(indexModel);
   }
 
   /**
@@ -64,8 +64,8 @@ public class BlockletDataMap extends BlockDataMap implements Serializable {
    * @param indexInfo
    */
   @Override
-  protected DataMapRowImpl loadMetadata(CarbonRowSchema[] taskSummarySchema,
-      SegmentProperties segmentProperties, BlockletDataMapModel blockletDataMapInfo,
+  protected IndexRowImpl loadMetadata(CarbonRowSchema[] taskSummarySchema,
+      SegmentProperties segmentProperties, BlockletIndexModel blockletDataMapInfo,
       List<DataFileFooter> indexInfo) {
     return loadBlockletMetaInfo(taskSummarySchema, segmentProperties, blockletDataMapInfo,
         indexInfo);
@@ -87,11 +87,11 @@ public class BlockletDataMap extends BlockDataMap implements Serializable {
    * @param blockletDataMapInfo
    * @param indexInfo
    */
-  private DataMapRowImpl loadBlockletMetaInfo(CarbonRowSchema[] taskSummarySchema,
-      SegmentProperties segmentProperties, BlockletDataMapModel blockletDataMapInfo,
+  private IndexRowImpl loadBlockletMetaInfo(CarbonRowSchema[] taskSummarySchema,
+      SegmentProperties segmentProperties, BlockletIndexModel blockletDataMapInfo,
       List<DataFileFooter> indexInfo) {
     String tempFilePath = null;
-    DataMapRowImpl summaryRow = null;
+    IndexRowImpl summaryRow = null;
     CarbonRowSchema[] schema = getFileFooterEntrySchema();
     boolean[] summaryRowMinMaxFlag = new boolean[segmentProperties.getNumberOfColumns()];
     Arrays.fill(summaryRowMinMaxFlag, true);
@@ -129,30 +129,30 @@ public class BlockletDataMap extends BlockDataMap implements Serializable {
     return summaryRow;
   }
 
-  private DataMapRowImpl loadToUnsafe(CarbonRowSchema[] schema, CarbonRowSchema[] taskSummarySchema,
+  private IndexRowImpl loadToUnsafe(CarbonRowSchema[] schema, CarbonRowSchema[] taskSummarySchema,
       DataFileFooter fileFooter, SegmentProperties segmentProperties,
-      List<CarbonColumn> minMaxCacheColumns, String filePath, DataMapRowImpl summaryRow,
+      List<CarbonColumn> minMaxCacheColumns, String filePath, IndexRowImpl summaryRow,
       BlockMetaInfo blockMetaInfo, int relativeBlockletId) {
     List<BlockletInfo> blockletList = fileFooter.getBlockletList();
     // Add one row to maintain task level min max for segment pruning
     if (!blockletList.isEmpty() && summaryRow == null) {
-      summaryRow = new DataMapRowImpl(taskSummarySchema);
+      summaryRow = new IndexRowImpl(taskSummarySchema);
     }
     for (int index = 0; index < blockletList.size(); index++) {
-      DataMapRow row = new DataMapRowImpl(schema);
+      IndexRow row = new IndexRowImpl(schema);
       int ordinal = 0;
       int taskMinMaxOrdinal = 1;
       BlockletInfo blockletInfo = blockletList.get(index);
       blockletInfo.setSorted(fileFooter.isSorted());
       BlockletMinMaxIndex minMaxIndex = blockletInfo.getBlockletIndex().getMinMaxIndex();
       // get min max values for columns to be cached
-      byte[][] minValuesForColumnsToBeCached = BlockletDataMapUtil
+      byte[][] minValuesForColumnsToBeCached = BlockletIndexUtil
           .getMinMaxForColumnsToBeCached(segmentProperties, minMaxCacheColumns,
               minMaxIndex.getMinValues());
-      byte[][] maxValuesForColumnsToBeCached = BlockletDataMapUtil
+      byte[][] maxValuesForColumnsToBeCached = BlockletIndexUtil
           .getMinMaxForColumnsToBeCached(segmentProperties, minMaxCacheColumns,
               minMaxIndex.getMaxValues());
-      boolean[] minMaxFlagValuesForColumnsToBeCached = BlockletDataMapUtil
+      boolean[] minMaxFlagValuesForColumnsToBeCached = BlockletIndexUtil
           .getMinMaxFlagValuesForColumnsToBeCached(segmentProperties, minMaxCacheColumns,
               fileFooter.getBlockletIndex().getMinMaxIndex().getIsMinMaxSet());
       row.setRow(addMinMax(schema[ordinal], minValuesForColumnsToBeCached), ordinal);
@@ -207,7 +207,7 @@ public class BlockletDataMap extends BlockDataMap implements Serializable {
   @Override
   public ExtendedBlocklet getDetailedBlocklet(String blockletId) {
     int absoluteBlockletId = Integer.parseInt(blockletId);
-    DataMapRow row = memoryDMStore.getDataMapRow(getFileFooterEntrySchema(), absoluteBlockletId);
+    IndexRow row = memoryDMStore.getDataMapRow(getFileFooterEntrySchema(), absoluteBlockletId);
     short relativeBlockletId = row.getShort(BLOCKLET_ID_INDEX);
     String filePath = getFilePath();
     return createBlocklet(row, getFileNameWithFilePath(row, filePath), relativeBlockletId,
@@ -215,17 +215,17 @@ public class BlockletDataMap extends BlockDataMap implements Serializable {
   }
 
   @Override
-  protected short getBlockletId(DataMapRow dataMapRow) {
-    return dataMapRow.getShort(BLOCKLET_ID_INDEX);
+  protected short getBlockletId(IndexRow indexRow) {
+    return indexRow.getShort(BLOCKLET_ID_INDEX);
   }
 
   protected boolean useMinMaxForExecutorPruning(FilterResolverIntf filterResolverIntf) {
-    return BlockletDataMapUtil
+    return BlockletIndexUtil
         .useMinMaxForBlockletPruning(filterResolverIntf, getMinMaxCacheColumns());
   }
 
   @Override
-  protected ExtendedBlocklet createBlocklet(DataMapRow row, String fileName, short blockletId,
+  protected ExtendedBlocklet createBlocklet(IndexRow row, String fileName, short blockletId,
       boolean useMinMaxForPruning) {
     short versionNumber = row.getShort(VERSION_INDEX);
     ExtendedBlocklet blocklet = new ExtendedBlocklet(fileName, blockletId + "",
diff --git a/core/src/main/java/org/apache/carbondata/core/indexstore/blockletindex/BlockletDataMapFactory.java b/core/src/main/java/org/apache/carbondata/core/indexstore/blockletindex/BlockletIndexFactory.java
similarity index 80%
rename from core/src/main/java/org/apache/carbondata/core/indexstore/blockletindex/BlockletDataMapFactory.java
rename to core/src/main/java/org/apache/carbondata/core/indexstore/blockletindex/BlockletIndexFactory.java
index 44cc6da..46d7080 100644
--- a/core/src/main/java/org/apache/carbondata/core/indexstore/blockletindex/BlockletDataMapFactory.java
+++ b/core/src/main/java/org/apache/carbondata/core/indexstore/blockletindex/BlockletIndexFactory.java
@@ -33,26 +33,26 @@ import org.apache.carbondata.core.cache.Cache;
 import org.apache.carbondata.core.cache.CacheProvider;
 import org.apache.carbondata.core.cache.CacheType;
 import org.apache.carbondata.core.constants.CarbonCommonConstants;
-import org.apache.carbondata.core.datamap.DataMapDistributable;
-import org.apache.carbondata.core.datamap.DataMapFilter;
-import org.apache.carbondata.core.datamap.DataMapMeta;
+import org.apache.carbondata.core.datamap.IndexFilter;
+import org.apache.carbondata.core.datamap.IndexInputSplit;
+import org.apache.carbondata.core.datamap.IndexMeta;
 import org.apache.carbondata.core.datamap.Segment;
-import org.apache.carbondata.core.datamap.dev.CacheableDataMap;
-import org.apache.carbondata.core.datamap.dev.DataMap;
-import org.apache.carbondata.core.datamap.dev.DataMapBuilder;
-import org.apache.carbondata.core.datamap.dev.DataMapWriter;
-import org.apache.carbondata.core.datamap.dev.cgdatamap.CoarseGrainDataMap;
-import org.apache.carbondata.core.datamap.dev.cgdatamap.CoarseGrainDataMapFactory;
-import org.apache.carbondata.core.datamap.dev.expr.DataMapDistributableWrapper;
-import org.apache.carbondata.core.datamap.dev.expr.DataMapExprWrapper;
+import org.apache.carbondata.core.datamap.dev.CacheableIndex;
+import org.apache.carbondata.core.datamap.dev.Index;
+import org.apache.carbondata.core.datamap.dev.IndexBuilder;
+import org.apache.carbondata.core.datamap.dev.IndexWriter;
+import org.apache.carbondata.core.datamap.dev.cgdatamap.CoarseGrainIndex;
+import org.apache.carbondata.core.datamap.dev.cgdatamap.CoarseGrainIndexFactory;
+import org.apache.carbondata.core.datamap.dev.expr.IndexExprWrapper;
+import org.apache.carbondata.core.datamap.dev.expr.IndexInputSplitWrapper;
 import org.apache.carbondata.core.datastore.block.SegmentProperties;
 import org.apache.carbondata.core.datastore.block.SegmentPropertiesAndSchemaHolder;
 import org.apache.carbondata.core.datastore.filesystem.CarbonFile;
 import org.apache.carbondata.core.datastore.impl.FileFactory;
 import org.apache.carbondata.core.features.TableOperation;
 import org.apache.carbondata.core.indexstore.Blocklet;
-import org.apache.carbondata.core.indexstore.BlockletDataMapIndexWrapper;
 import org.apache.carbondata.core.indexstore.BlockletDetailsFetcher;
+import org.apache.carbondata.core.indexstore.BlockletIndexWrapper;
 import org.apache.carbondata.core.indexstore.ExtendedBlocklet;
 import org.apache.carbondata.core.indexstore.PartitionSpec;
 import org.apache.carbondata.core.indexstore.SegmentBlockIndexInfo;
@@ -68,16 +68,16 @@ import org.apache.carbondata.core.scan.filter.executer.FilterExecuter;
 import org.apache.carbondata.core.scan.filter.resolver.FilterResolverIntf;
 import org.apache.carbondata.core.segmentmeta.SegmentColumnMetaDataInfo;
 import org.apache.carbondata.core.segmentmeta.SegmentMetaDataInfo;
-import org.apache.carbondata.core.util.BlockletDataMapUtil;
+import org.apache.carbondata.core.util.BlockletIndexUtil;
 import org.apache.carbondata.core.util.CarbonProperties;
 import org.apache.carbondata.core.util.path.CarbonTablePath;
 import org.apache.carbondata.events.Event;
 
 /**
- * Table map for blocklet
+ * Index for blocklet
  */
-public class BlockletDataMapFactory extends CoarseGrainDataMapFactory
-    implements BlockletDetailsFetcher, SegmentPropertiesFetcher, CacheableDataMap {
+public class BlockletIndexFactory extends CoarseGrainIndexFactory
+    implements BlockletDetailsFetcher, SegmentPropertiesFetcher, CacheableIndex {
 
   private static final String NAME = "clustered.btree.blocklet";
   /**
@@ -86,16 +86,16 @@ public class BlockletDataMapFactory extends CoarseGrainDataMapFactory
   public static final String CACHE_LEVEL_BLOCKLET = "BLOCKLET";
 
   public static final DataMapSchema DATA_MAP_SCHEMA =
-      new DataMapSchema(NAME, BlockletDataMapFactory.class.getName());
+      new DataMapSchema(NAME, BlockletIndexFactory.class.getName());
 
   private AbsoluteTableIdentifier identifier;
 
   // segmentId -> list of index file
   private Map<String, SegmentBlockIndexInfo> segmentMap = new ConcurrentHashMap<>();
 
-  private Cache<TableBlockIndexUniqueIdentifierWrapper, BlockletDataMapIndexWrapper> cache;
+  private Cache<TableBlockIndexUniqueIdentifierWrapper, BlockletIndexWrapper> cache;
 
-  public BlockletDataMapFactory(CarbonTable carbonTable, DataMapSchema dataMapSchema) {
+  public BlockletIndexFactory(CarbonTable carbonTable, DataMapSchema dataMapSchema) {
     super(carbonTable, dataMapSchema);
     this.identifier = carbonTable.getAbsoluteTableIdentifier();
     cache = CacheProvider.getInstance()
@@ -108,25 +108,25 @@ public class BlockletDataMapFactory extends CoarseGrainDataMapFactory
    * @param carbonTable
    * @return
    */
-  public static DataMap createDataMap(CarbonTable carbonTable) {
-    boolean cacheLevelBlock = BlockletDataMapUtil.isCacheLevelBlock(carbonTable);
+  public static Index createDataMap(CarbonTable carbonTable) {
+    boolean cacheLevelBlock = BlockletIndexUtil.isCacheLevelBlock(carbonTable);
     if (cacheLevelBlock) {
       // case1: when CACHE_LEVEL = BLOCK
-      return new BlockDataMap();
+      return new BlockIndex();
     } else {
       // case2: when CACHE_LEVEL = BLOCKLET
-      return new BlockletDataMap();
+      return new BlockletIndex();
     }
   }
 
   @Override
-  public DataMapWriter createWriter(Segment segment, String shardName,
+  public IndexWriter createWriter(Segment segment, String shardName,
       SegmentProperties segmentProperties) {
     throw new UnsupportedOperationException("not implemented");
   }
 
   @Override
-  public DataMapBuilder createBuilder(Segment segment, String shardName,
+  public IndexBuilder createBuilder(Segment segment, String shardName,
       SegmentProperties segmentProperties) {
     throw new UnsupportedOperationException("not implemented");
   }
@@ -134,19 +134,19 @@ public class BlockletDataMapFactory extends CoarseGrainDataMapFactory
   /**
    * Get the datamap for all segments
    */
-  public Map<Segment, List<CoarseGrainDataMap>> getDataMaps(List<Segment> segments,
-      DataMapFilter filter) throws IOException {
-    return getDataMaps(segments, null, filter);
+  public Map<Segment, List<CoarseGrainIndex>> getIndexes(List<Segment> segments,
+      IndexFilter filter) throws IOException {
+    return getIndexes(segments, null, filter);
   }
 
   /**
    * Get the datamap for all segments
    */
-  public Map<Segment, List<CoarseGrainDataMap>> getDataMaps(List<Segment> segments,
-      List<PartitionSpec> partitionsToPrune, DataMapFilter filter) throws IOException {
+  public Map<Segment, List<CoarseGrainIndex>> getIndexes(List<Segment> segments,
+      List<PartitionSpec> partitionsToPrune, IndexFilter filter) throws IOException {
     List<TableBlockIndexUniqueIdentifierWrapper> tableBlockIndexUniqueIdentifierWrappers =
         new ArrayList<>();
-    Map<Segment, List<CoarseGrainDataMap>> dataMaps = new HashMap<>();
+    Map<Segment, List<CoarseGrainIndex>> dataMaps = new HashMap<>();
     Map<String, Segment> segmentMap = new HashMap<>();
     for (Segment segment : segments) {
       segmentMap.put(segment.getSegmentNo(), segment);
@@ -175,13 +175,13 @@ public class BlockletDataMapFactory extends CoarseGrainDataMapFactory
         }
       }
     }
-    List<BlockletDataMapIndexWrapper> blockletDataMapIndexWrappers =
+    List<BlockletIndexWrapper> blockletIndexWrappers =
         cache.getAll(tableBlockIndexUniqueIdentifierWrappers);
-    for (BlockletDataMapIndexWrapper wrapper : blockletDataMapIndexWrappers) {
+    for (BlockletIndexWrapper wrapper : blockletIndexWrappers) {
       Segment segment = segmentMap.get(wrapper.getSegmentId());
-      List<CoarseGrainDataMap> datamapList = dataMaps.get(segment);
+      List<CoarseGrainIndex> datamapList = dataMaps.get(segment);
       if (null == datamapList) {
-        datamapList = new ArrayList<CoarseGrainDataMap>();
+        datamapList = new ArrayList<CoarseGrainIndex>();
       }
       datamapList.addAll(wrapper.getDataMaps());
       dataMaps.put(segment, datamapList);
@@ -227,7 +227,7 @@ public class BlockletDataMapFactory extends CoarseGrainDataMapFactory
    * @param tableBlockIndexUniqueIdentifierWrappers to add tableBlockIndexUniqueIdentifiers
    */
   private void getTableBlockIndexUniqueIdentifierUsingSegmentMinMax(Segment segment,
-      SegmentMetaDataInfo segmentMetaDataInfo, DataMapFilter filter,
+      SegmentMetaDataInfo segmentMetaDataInfo, IndexFilter filter,
       Set<TableBlockIndexUniqueIdentifier> identifiers,
       List<TableBlockIndexUniqueIdentifierWrapper> tableBlockIndexUniqueIdentifierWrappers) {
     boolean isScanRequired = false;
@@ -288,7 +288,7 @@ public class BlockletDataMapFactory extends CoarseGrainDataMapFactory
         .getSegmentProperties();
 
     FilterResolverIntf resolver =
-        new DataMapFilter(segmentProperties, this.getCarbonTable(), filter.getExpression())
+        new IndexFilter(segmentProperties, this.getCarbonTable(), filter.getExpression())
             .getResolver();
     // prepare filter executer using datmapFilter resolver
     FilterExecuter filterExecuter =
@@ -327,23 +327,23 @@ public class BlockletDataMapFactory extends CoarseGrainDataMapFactory
   }
 
   @Override
-  public List<CoarseGrainDataMap> getDataMaps(Segment segment) throws IOException {
-    return getDataMaps(segment, null);
+  public List<CoarseGrainIndex> getIndexes(Segment segment) throws IOException {
+    return getIndexes(segment, null);
   }
 
   @Override
-  public List<CoarseGrainDataMap> getDataMaps(Segment segment,
+  public List<CoarseGrainIndex> getIndexes(Segment segment,
       List<PartitionSpec> partitionsToPrune) throws IOException {
-    List<CoarseGrainDataMap> dataMaps = new ArrayList<>();
+    List<CoarseGrainIndex> dataMaps = new ArrayList<>();
     Set<TableBlockIndexUniqueIdentifier> identifiers =
         getTableBlockIndexUniqueIdentifiers(segment);
     List<TableBlockIndexUniqueIdentifierWrapper> tableBlockIndexUniqueIdentifierWrappers =
         new ArrayList<>(identifiers.size());
     getTableBlockUniqueIdentifierWrappers(partitionsToPrune,
         tableBlockIndexUniqueIdentifierWrappers, identifiers);
-    List<BlockletDataMapIndexWrapper> blockletDataMapIndexWrappers =
+    List<BlockletIndexWrapper> blockletIndexWrappers =
         cache.getAll(tableBlockIndexUniqueIdentifierWrappers);
-    for (BlockletDataMapIndexWrapper wrapper : blockletDataMapIndexWrappers) {
+    for (BlockletIndexWrapper wrapper : blockletIndexWrappers) {
       dataMaps.addAll(wrapper.getDataMaps());
     }
     return dataMaps;
@@ -359,7 +359,7 @@ public class BlockletDataMapFactory extends CoarseGrainDataMapFactory
       return segmentBlockIndexInfo.getTableBlockIndexUniqueIdentifiers();
     } else {
       tableBlockIndexUniqueIdentifiers =
-          BlockletDataMapUtil.getTableBlockUniqueIdentifiers(segment);
+          BlockletIndexUtil.getTableBlockUniqueIdentifiers(segment);
       if (tableBlockIndexUniqueIdentifiers.size() > 0) {
         segmentMap.put(segment.getSegmentNo(),
             new SegmentBlockIndexInfo(tableBlockIndexUniqueIdentifiers,
@@ -371,7 +371,7 @@ public class BlockletDataMapFactory extends CoarseGrainDataMapFactory
 
   /**
    * Get the blocklet detail information based on blockletid, blockid and segmentId. This method is
-   * exclusively for BlockletDataMapFactory as detail information is only available in this
+   * exclusively for BlockletIndexFactory as detail information is only available in this
    * default datamap.
    */
   @Override
@@ -428,13 +428,13 @@ public class BlockletDataMapFactory extends CoarseGrainDataMapFactory
       Set<TableBlockIndexUniqueIdentifierWrapper> identifiersWrapper, Blocklet blocklet)
       throws IOException {
     for (TableBlockIndexUniqueIdentifierWrapper identifierWrapper : identifiersWrapper) {
-      BlockletDataMapIndexWrapper wrapper = cache.get(identifierWrapper);
-      List<BlockDataMap> dataMaps = wrapper.getDataMaps();
-      for (DataMap dataMap : dataMaps) {
-        if (((BlockDataMap) dataMap)
-            .getTableTaskInfo(BlockletDataMapRowIndexes.SUMMARY_INDEX_FILE_NAME)
+      BlockletIndexWrapper wrapper = cache.get(identifierWrapper);
+      List<BlockIndex> dataMaps = wrapper.getDataMaps();
+      for (Index index : dataMaps) {
+        if (((BlockIndex) index)
+            .getTableTaskInfo(BlockletIndexRowIndexes.SUMMARY_INDEX_FILE_NAME)
             .startsWith(blocklet.getFilePath())) {
-          return ((BlockDataMap) dataMap).getDetailedBlocklet(blocklet.getBlockletId());
+          return ((BlockIndex) index).getDetailedBlocklet(blocklet.getBlockletId());
         }
       }
     }
@@ -442,15 +442,15 @@ public class BlockletDataMapFactory extends CoarseGrainDataMapFactory
   }
 
   @Override
-  public List<DataMapDistributable> toDistributable(Segment segment) {
-    List<DataMapDistributable> distributables = new ArrayList<>();
+  public List<IndexInputSplit> toDistributable(Segment segment) {
+    List<IndexInputSplit> distributables = new ArrayList<>();
     try {
-      BlockletDataMapDistributable distributable = new BlockletDataMapDistributable();
+      BlockletIndexInputSplit distributable = new BlockletIndexInputSplit();
       distributable.setSegment(segment);
       distributable.setDataMapSchema(DATA_MAP_SCHEMA);
       distributable.setSegmentPath(CarbonTablePath.getSegmentPath(identifier.getTablePath(),
           segment.getSegmentNo()));
-      distributables.add(new DataMapDistributableWrapper(UUID.randomUUID().toString(),
+      distributables.add(new IndexInputSplitWrapper(UUID.randomUUID().toString(),
           distributable).getDistributable());
     } catch (Exception e) {
       throw new RuntimeException(e);
@@ -474,13 +474,13 @@ public class BlockletDataMapFactory extends CoarseGrainDataMapFactory
       for (TableBlockIndexUniqueIdentifier blockIndex : blockIndexes) {
         TableBlockIndexUniqueIdentifierWrapper blockIndexWrapper =
             new TableBlockIndexUniqueIdentifierWrapper(blockIndex, this.getCarbonTable());
-        BlockletDataMapIndexWrapper wrapper = cache.getIfPresent(blockIndexWrapper);
+        BlockletIndexWrapper wrapper = cache.getIfPresent(blockIndexWrapper);
         if (null != wrapper) {
-          List<BlockDataMap> dataMaps = wrapper.getDataMaps();
-          for (DataMap dataMap : dataMaps) {
-            if (dataMap != null) {
+          List<BlockIndex> dataMaps = wrapper.getDataMaps();
+          for (Index index : dataMaps) {
+            if (index != null) {
               cache.invalidate(blockIndexWrapper);
-              dataMap.clear();
+              index.clear();
             }
           }
         }
@@ -503,12 +503,12 @@ public class BlockletDataMapFactory extends CoarseGrainDataMapFactory
     int numOfIndexFiles = 0;
     for (Map.Entry<String, SegmentBlockIndexInfo> entry : segmentMap.entrySet()) {
       for (TableBlockIndexUniqueIdentifier tableBlockIndexUniqueIdentifier : entry.getValue()
-          .getTableBlockIndexUniqueIdentifiers()) {
-        BlockletDataMapIndexWrapper blockletDataMapIndexWrapper = cache.getIfPresent(
+              .getTableBlockIndexUniqueIdentifiers()) {
+        BlockletIndexWrapper blockletIndexWrapper = cache.getIfPresent(
             new TableBlockIndexUniqueIdentifierWrapper(tableBlockIndexUniqueIdentifier,
                 getCarbonTable()));
-        if (blockletDataMapIndexWrapper != null) {
-          sum += blockletDataMapIndexWrapper.getMemorySize();
+        if (blockletIndexWrapper != null) {
+          sum += blockletIndexWrapper.getMemorySize();
           numOfIndexFiles++;
         }
       }
@@ -517,9 +517,9 @@ public class BlockletDataMapFactory extends CoarseGrainDataMapFactory
   }
 
   @Override
-  public List<CoarseGrainDataMap> getDataMaps(DataMapDistributable distributable)
+  public List<CoarseGrainIndex> getIndexes(IndexInputSplit distributable)
       throws IOException {
-    BlockletDataMapDistributable mapDistributable = (BlockletDataMapDistributable) distributable;
+    BlockletIndexInputSplit mapDistributable = (BlockletIndexInputSplit) distributable;
     List<TableBlockIndexUniqueIdentifierWrapper> identifiersWrapper;
     String segmentNo = mapDistributable.getSegment().getSegmentNo();
     if (mapDistributable.getSegmentPath() != null) {
@@ -528,10 +528,10 @@ public class BlockletDataMapFactory extends CoarseGrainDataMapFactory
       identifiersWrapper =
           getTableBlockIndexUniqueIdentifier(mapDistributable.getFilePath(), segmentNo);
     }
-    List<CoarseGrainDataMap> dataMaps = new ArrayList<>();
+    List<CoarseGrainIndex> dataMaps = new ArrayList<>();
     try {
-      List<BlockletDataMapIndexWrapper> wrappers = cache.getAll(identifiersWrapper);
-      for (BlockletDataMapIndexWrapper wrapper : wrappers) {
+      List<BlockletIndexWrapper> wrappers = cache.getAll(identifiersWrapper);
+      for (BlockletIndexWrapper wrapper : wrappers) {
         dataMaps.addAll(wrapper.getDataMaps());
       }
     } catch (IOException e) {
@@ -541,7 +541,7 @@ public class BlockletDataMapFactory extends CoarseGrainDataMapFactory
   }
 
   private List<TableBlockIndexUniqueIdentifierWrapper> getTableBlockIndexUniqueIdentifier(
-      DataMapDistributable distributable) throws IOException {
+      IndexInputSplit distributable) throws IOException {
     List<TableBlockIndexUniqueIdentifierWrapper> identifiersWrapper = new ArrayList<>();
     SegmentBlockIndexInfo segmentBlockIndexInfo =
         segmentMap.get(distributable.getSegment().getSegmentNo());
@@ -610,18 +610,18 @@ public class BlockletDataMapFactory extends CoarseGrainDataMapFactory
   }
 
   @Override
-  public DataMapMeta getMeta() {
+  public IndexMeta getMeta() {
     // TODO: pass SORT_COLUMNS into this class
     return null;
   }
 
   @Override
-  public void deleteDatamapData(Segment segment) {
+  public void deleteIndexData(Segment segment) {
 
   }
 
   @Override
-  public void deleteDatamapData() {
+  public void deleteIndexData() {
 
   }
 
@@ -633,18 +633,18 @@ public class BlockletDataMapFactory extends CoarseGrainDataMapFactory
   @Override
   public SegmentProperties getSegmentProperties(Segment segment, List<PartitionSpec> partitions)
       throws IOException {
-    List<CoarseGrainDataMap> dataMaps = getDataMaps(segment, partitions);
+    List<CoarseGrainIndex> dataMaps = getIndexes(segment, partitions);
     assert (dataMaps.size() > 0);
-    CoarseGrainDataMap coarseGrainDataMap = dataMaps.get(0);
-    assert (coarseGrainDataMap instanceof BlockDataMap);
-    BlockDataMap dataMap = (BlockDataMap) coarseGrainDataMap;
+    CoarseGrainIndex coarseGrainIndex = dataMaps.get(0);
+    assert (coarseGrainIndex instanceof BlockIndex);
+    BlockIndex dataMap = (BlockIndex) coarseGrainIndex;
     return dataMap.getSegmentProperties();
   }
 
   @Override
-  public SegmentProperties getSegmentPropertiesFromDataMap(DataMap coarseGrainDataMap) {
-    assert (coarseGrainDataMap instanceof BlockDataMap);
-    BlockDataMap dataMap = (BlockDataMap) coarseGrainDataMap;
+  public SegmentProperties getSegmentPropertiesFromDataMap(Index coarseGrainIndex) {
+    assert (coarseGrainIndex instanceof BlockIndex);
+    BlockIndex dataMap = (BlockIndex) coarseGrainIndex;
     return dataMap.getSegmentProperties();
   }
 
@@ -652,8 +652,8 @@ public class BlockletDataMapFactory extends CoarseGrainDataMapFactory
   public List<Blocklet> getAllBlocklets(Segment segment, List<PartitionSpec> partitions)
       throws IOException {
     List<Blocklet> blocklets = new ArrayList<>();
-    List<CoarseGrainDataMap> dataMaps = getDataMaps(segment, partitions);
-    for (CoarseGrainDataMap dataMap : dataMaps) {
+    List<CoarseGrainIndex> dataMaps = getIndexes(segment, partitions);
+    for (CoarseGrainIndex dataMap : dataMaps) {
       blocklets.addAll(dataMap
           .prune((FilterResolverIntf) null, getSegmentProperties(segment, partitions), partitions,
               null, this.getCarbonTable()));
@@ -668,25 +668,25 @@ public class BlockletDataMapFactory extends CoarseGrainDataMapFactory
 
   @Override
   public void cache(TableBlockIndexUniqueIdentifierWrapper tableBlockIndexUniqueIdentifierWrapper,
-      BlockletDataMapIndexWrapper blockletDataMapIndexWrapper) throws IOException {
-    cache.put(tableBlockIndexUniqueIdentifierWrapper, blockletDataMapIndexWrapper);
+      BlockletIndexWrapper blockletIndexWrapper) throws IOException {
+    cache.put(tableBlockIndexUniqueIdentifierWrapper, blockletIndexWrapper);
   }
 
   @Override
-  public List<DataMapDistributable> getAllUncachedDistributables(
-      List<DataMapDistributable> distributables) throws IOException {
-    List<DataMapDistributable> distributablesToBeLoaded = new ArrayList<>(distributables.size());
-    for (DataMapDistributable distributable : distributables) {
+  public List<IndexInputSplit> getAllUncachedDistributables(
+      List<IndexInputSplit> distributables) throws IOException {
+    List<IndexInputSplit> distributablesToBeLoaded = new ArrayList<>(distributables.size());
+    for (IndexInputSplit distributable : distributables) {
       Segment segment = distributable.getSegment();
       Set<TableBlockIndexUniqueIdentifier> tableBlockIndexUniqueIdentifiers =
           getTableBlockIndexUniqueIdentifiers(segment);
       // filter out the tableBlockIndexUniqueIdentifiers based on distributable
-      TableBlockIndexUniqueIdentifier validIdentifier = BlockletDataMapUtil
+      TableBlockIndexUniqueIdentifier validIdentifier = BlockletIndexUtil
           .filterIdentifiersBasedOnDistributable(tableBlockIndexUniqueIdentifiers,
-              (BlockletDataMapDistributable) distributable);
+              (BlockletIndexInputSplit) distributable);
       if (null == cache.getIfPresent(
           new TableBlockIndexUniqueIdentifierWrapper(validIdentifier, this.getCarbonTable()))) {
-        ((BlockletDataMapDistributable) distributable)
+        ((BlockletIndexInputSplit) distributable)
             .setTableBlockIndexUniqueIdentifier(validIdentifier);
         distributablesToBeLoaded.add(distributable);
       }
@@ -699,7 +699,7 @@ public class BlockletDataMapFactory extends CoarseGrainDataMapFactory
     Set<TableBlockIndexUniqueIdentifier> tableBlockIndexUniqueIdentifiers =
         segmentMap.get(segment.getSegmentNo()).getTableBlockIndexUniqueIdentifiers();
     if (tableBlockIndexUniqueIdentifiers == null) {
-      tableBlockIndexUniqueIdentifiers = BlockletDataMapUtil.getSegmentUniqueIdentifiers(segment);
+      tableBlockIndexUniqueIdentifiers = BlockletIndexUtil.getSegmentUniqueIdentifiers(segment);
     }
     return tableBlockIndexUniqueIdentifiers;
   }
@@ -713,21 +713,21 @@ public class BlockletDataMapFactory extends CoarseGrainDataMapFactory
   }
 
   @Override
-  public List<DataMapDistributable> getAllUncachedDistributables(List<Segment> validSegments,
-      DataMapExprWrapper dataMapExprWrapper) throws IOException {
-    List<DataMapDistributable> distributablesToBeLoaded = new ArrayList<>();
+  public List<IndexInputSplit> getAllUncachedDistributables(List<Segment> validSegments,
+      IndexExprWrapper indexExprWrapper) throws IOException {
+    List<IndexInputSplit> distributablesToBeLoaded = new ArrayList<>();
     for (Segment segment : validSegments) {
-      DataMapDistributableWrapper dataMapDistributableWrappers =
-          dataMapExprWrapper.toDistributableSegment(segment);
+      IndexInputSplitWrapper indexInputSplitWrappers =
+          indexExprWrapper.toDistributableSegment(segment);
       Set<TableBlockIndexUniqueIdentifier> tableBlockIndexUniqueIdentifiers =
           getTableSegmentUniqueIdentifiers(segment);
       for (TableBlockIndexUniqueIdentifier identifier : tableBlockIndexUniqueIdentifiers) {
-        BlockletDataMapIndexWrapper blockletDataMapIndexWrapper = cache.getIfPresent(
+        BlockletIndexWrapper blockletIndexWrapper = cache.getIfPresent(
             new TableBlockIndexUniqueIdentifierWrapper(identifier, this.getCarbonTable()));
-        if (identifier.getIndexFilePath() == null || blockletDataMapIndexWrapper == null) {
-          ((BlockletDataMapDistributable) dataMapDistributableWrappers.getDistributable())
+        if (identifier.getIndexFilePath() == null || blockletIndexWrapper == null) {
+          ((BlockletIndexInputSplit) indexInputSplitWrappers.getDistributable())
               .setTableBlockIndexUniqueIdentifier(identifier);
-          distributablesToBeLoaded.add(dataMapDistributableWrappers.getDistributable());
+          distributablesToBeLoaded.add(indexInputSplitWrappers.getDistributable());
         }
       }
     }
@@ -735,16 +735,16 @@ public class BlockletDataMapFactory extends CoarseGrainDataMapFactory
   }
 
   @Override
-  public DataMapDistributableWrapper toDistributableSegment(Segment segment,
+  public IndexInputSplitWrapper toDistributableSegment(Segment segment,
       DataMapSchema schema, AbsoluteTableIdentifier identifier, String uniqueId) {
     try {
-      BlockletDataMapDistributable distributable = new BlockletDataMapDistributable();
+      BlockletIndexInputSplit distributable = new BlockletIndexInputSplit();
       distributable.setDataMapSchema(schema);
       distributable.setSegment(segment);
       distributable.setSegmentPath(
           CarbonTablePath.getSegmentPath(identifier.getTablePath(), segment.getSegmentNo()));
       distributable.setTablePath(identifier.getTablePath());
-      return new DataMapDistributableWrapper(uniqueId, distributable);
+      return new IndexInputSplitWrapper(uniqueId, distributable);
     } catch (Exception e) {
       throw new RuntimeException(e);
     }
diff --git a/core/src/main/java/org/apache/carbondata/core/indexstore/blockletindex/BlockletDataMapDistributable.java b/core/src/main/java/org/apache/carbondata/core/indexstore/blockletindex/BlockletIndexInputSplit.java
similarity index 89%
rename from core/src/main/java/org/apache/carbondata/core/indexstore/blockletindex/BlockletDataMapDistributable.java
rename to core/src/main/java/org/apache/carbondata/core/indexstore/blockletindex/BlockletIndexInputSplit.java
index f1937de..a4a2bd8 100644
--- a/core/src/main/java/org/apache/carbondata/core/indexstore/blockletindex/BlockletDataMapDistributable.java
+++ b/core/src/main/java/org/apache/carbondata/core/indexstore/blockletindex/BlockletIndexInputSplit.java
@@ -17,7 +17,7 @@
 
 package org.apache.carbondata.core.indexstore.blockletindex;
 
-import org.apache.carbondata.core.datamap.DataMapDistributable;
+import org.apache.carbondata.core.datamap.IndexInputSplit;
 import org.apache.carbondata.core.indexstore.TableBlockIndexUniqueIdentifier;
 
 /**
@@ -26,7 +26,7 @@ import org.apache.carbondata.core.indexstore.TableBlockIndexUniqueIdentifier;
  * Using this object job like spark/MR can be launched and execute each distributable object as
  * one datamap task.
  */
-public class BlockletDataMapDistributable extends DataMapDistributable {
+public class BlockletIndexInputSplit extends IndexInputSplit {
 
   /**
    * Relative file path from the segment folder.
@@ -37,11 +37,11 @@ public class BlockletDataMapDistributable extends DataMapDistributable {
 
   private TableBlockIndexUniqueIdentifier tableBlockIndexUniqueIdentifier;
 
-  public BlockletDataMapDistributable() {
+  public BlockletIndexInputSplit() {
 
   }
 
-  public BlockletDataMapDistributable(String indexFilePath) {
+  public BlockletIndexInputSplit(String indexFilePath) {
     this.filePath = indexFilePath;
   }
 
diff --git a/core/src/main/java/org/apache/carbondata/core/indexstore/blockletindex/BlockletDataMapModel.java b/core/src/main/java/org/apache/carbondata/core/indexstore/blockletindex/BlockletIndexModel.java
similarity index 90%
rename from core/src/main/java/org/apache/carbondata/core/indexstore/blockletindex/BlockletDataMapModel.java
rename to core/src/main/java/org/apache/carbondata/core/indexstore/blockletindex/BlockletIndexModel.java
index 5d8a235..598f7d9 100644
--- a/core/src/main/java/org/apache/carbondata/core/indexstore/blockletindex/BlockletDataMapModel.java
+++ b/core/src/main/java/org/apache/carbondata/core/indexstore/blockletindex/BlockletIndexModel.java
@@ -20,7 +20,7 @@ package org.apache.carbondata.core.indexstore.blockletindex;
 import java.util.List;
 import java.util.Map;
 
-import org.apache.carbondata.core.datamap.dev.DataMapModel;
+import org.apache.carbondata.core.datamap.dev.IndexModel;
 import org.apache.carbondata.core.indexstore.BlockMetaInfo;
 import org.apache.carbondata.core.metadata.blocklet.DataFileFooter;
 import org.apache.carbondata.core.metadata.schema.table.CarbonTable;
@@ -28,9 +28,9 @@ import org.apache.carbondata.core.metadata.schema.table.CarbonTable;
 import org.apache.hadoop.conf.Configuration;
 
 /**
- * It is the model object to keep the information to build or initialize BlockletDataMap.
+ * It is the model object to keep the information to build or initialize BlockletIndex.
  */
-public class BlockletDataMapModel extends DataMapModel {
+public class BlockletIndexModel extends IndexModel {
 
   private byte[] fileData;
 
@@ -48,7 +48,7 @@ public class BlockletDataMapModel extends DataMapModel {
 
   private boolean serializeDmStore = false;
 
-  public BlockletDataMapModel(CarbonTable carbonTable, String filePath, byte[] fileData,
+  public BlockletIndexModel(CarbonTable carbonTable, String filePath, byte[] fileData,
       Map<String, BlockMetaInfo> blockMetaInfoMap, String segmentId, Configuration configuration) {
     super(filePath, configuration);
     this.fileData = fileData;
@@ -57,7 +57,7 @@ public class BlockletDataMapModel extends DataMapModel {
     this.carbonTable = carbonTable;
   }
 
-  public BlockletDataMapModel(CarbonTable carbonTable, String filePath,
+  public BlockletIndexModel(CarbonTable carbonTable, String filePath,
       byte[] fileData, Map<String, BlockMetaInfo> blockMetaInfoMap, String segmentId,
       boolean addToUnsafe, Configuration configuration, boolean serializeDmStore) {
     this(carbonTable, filePath, fileData, blockMetaInfoMap, segmentId, configuration);
diff --git a/core/src/main/java/org/apache/carbondata/core/indexstore/blockletindex/BlockletDataMapRowIndexes.java b/core/src/main/java/org/apache/carbondata/core/indexstore/blockletindex/BlockletIndexRowIndexes.java
similarity index 90%
rename from core/src/main/java/org/apache/carbondata/core/indexstore/blockletindex/BlockletDataMapRowIndexes.java
rename to core/src/main/java/org/apache/carbondata/core/indexstore/blockletindex/BlockletIndexRowIndexes.java
index 2872ab6..52bbd66 100644
--- a/core/src/main/java/org/apache/carbondata/core/indexstore/blockletindex/BlockletDataMapRowIndexes.java
+++ b/core/src/main/java/org/apache/carbondata/core/indexstore/blockletindex/BlockletIndexRowIndexes.java
@@ -18,11 +18,11 @@
 package org.apache.carbondata.core.indexstore.blockletindex;
 
 /**
- * holder for blocklet info indexes in a DataMap row
+ * holder for blocklet info indexes in a Index row
  */
-public interface BlockletDataMapRowIndexes {
+public interface BlockletIndexRowIndexes {
 
-  // Each DataMapRow Indexes for blocklet and block dataMap
+  // Each IndexRow Indexes for blocklet and block dataMap
   int MIN_VALUES_INDEX = 0;
 
   int MAX_VALUES_INDEX = 1;
diff --git a/core/src/main/java/org/apache/carbondata/core/indexstore/row/DataMapRow.java b/core/src/main/java/org/apache/carbondata/core/indexstore/row/IndexRow.java
similarity index 92%
rename from core/src/main/java/org/apache/carbondata/core/indexstore/row/DataMapRow.java
rename to core/src/main/java/org/apache/carbondata/core/indexstore/row/IndexRow.java
index eb93505..cf66f97 100644
--- a/core/src/main/java/org/apache/carbondata/core/indexstore/row/DataMapRow.java
+++ b/core/src/main/java/org/apache/carbondata/core/indexstore/row/IndexRow.java
@@ -25,22 +25,22 @@ import org.apache.carbondata.core.indexstore.schema.CarbonRowSchema;
  * It is just a normal row to store data. Implementation classes could be safe and unsafe.
  * TODO move this class a global row and use across loading after DataType is changed class
  */
-public abstract class DataMapRow implements Serializable {
+public abstract class IndexRow implements Serializable {
 
   /**
    * This is made transient as it is temporary and should not be serialized
    */
   protected transient CarbonRowSchema[] schemas;
 
-  public DataMapRow(CarbonRowSchema[] schemas) {
+  public IndexRow(CarbonRowSchema[] schemas) {
     this.schemas = schemas;
   }
 
   public abstract byte[] getByteArray(int ordinal);
 
-  public abstract DataMapRow getRow(int ordinal);
+  public abstract IndexRow getRow(int ordinal);
 
-  public abstract void setRow(DataMapRow row, int ordinal);
+  public abstract void setRow(IndexRow row, int ordinal);
 
   public abstract void setByteArray(byte[] byteArray, int ordinal);
 
@@ -92,7 +92,7 @@ public abstract class DataMapRow implements Serializable {
       case VARIABLE_INT:
         return getLengthInBytes(ordinal) + 4;
       case STRUCT:
-        DataMapRow row = getRow(ordinal);
+        IndexRow row = getRow(ordinal);
         CarbonRowSchema[] childSchemas =
             ((CarbonRowSchema.StructCarbonRowSchema) schemas[ordinal]).getChildSchemas();
         // set the child schema. Because schema is transient it can be null
diff --git a/core/src/main/java/org/apache/carbondata/core/indexstore/row/DataMapRowImpl.java b/core/src/main/java/org/apache/carbondata/core/indexstore/row/IndexRowImpl.java
similarity index 92%
rename from core/src/main/java/org/apache/carbondata/core/indexstore/row/DataMapRowImpl.java
rename to core/src/main/java/org/apache/carbondata/core/indexstore/row/IndexRowImpl.java
index 22eca8a..2d0770d 100644
--- a/core/src/main/java/org/apache/carbondata/core/indexstore/row/DataMapRowImpl.java
+++ b/core/src/main/java/org/apache/carbondata/core/indexstore/row/IndexRowImpl.java
@@ -23,13 +23,13 @@ import org.apache.carbondata.core.metadata.datatype.DataTypes;
 /**
  * Data map row.
  */
-public class DataMapRowImpl extends DataMapRow {
+public class IndexRowImpl extends IndexRow {
 
   private Object[] data;
 
   private int totalLengthInBytes;
 
-  public DataMapRowImpl(CarbonRowSchema[] schemas) {
+  public IndexRowImpl(CarbonRowSchema[] schemas) {
     super(schemas);
     this.data = new Object[schemas.length];
   }
@@ -42,7 +42,7 @@ public class DataMapRowImpl extends DataMapRow {
   @Override
   public int getLengthInBytes(int ordinal) {
     // if the segment data is written in tablepath
-    // then the data[BlockletDataMapRowIndexes.SUMMARY_INDEX_PATH] will be null.
+    // then the data[BlockletIndexRowIndexes.SUMMARY_INDEX_PATH] will be null.
     if (data[ordinal] == null) {
       return 0;
     }
@@ -61,8 +61,8 @@ public class DataMapRowImpl extends DataMapRow {
   }
 
   @Override
-  public DataMapRow getRow(int ordinal) {
-    return (DataMapRow) data[ordinal];
+  public IndexRow getRow(int ordinal) {
+    return (IndexRow) data[ordinal];
   }
 
   @Override
@@ -133,7 +133,7 @@ public class DataMapRowImpl extends DataMapRow {
   }
 
   @Override
-  public void setRow(DataMapRow row, int ordinal) {
+  public void setRow(IndexRow row, int ordinal) {
     assert (DataTypes.isStructType(schemas[ordinal].getDataType()));
     data[ordinal] = row;
   }
diff --git a/core/src/main/java/org/apache/carbondata/core/indexstore/row/UnsafeDataMapRow.java b/core/src/main/java/org/apache/carbondata/core/indexstore/row/UnsafeIndexRow.java
similarity index 95%
rename from core/src/main/java/org/apache/carbondata/core/indexstore/row/UnsafeDataMapRow.java
rename to core/src/main/java/org/apache/carbondata/core/indexstore/row/UnsafeIndexRow.java
index eca20aa..b6ebc14 100644
--- a/core/src/main/java/org/apache/carbondata/core/indexstore/row/UnsafeDataMapRow.java
+++ b/core/src/main/java/org/apache/carbondata/core/indexstore/row/UnsafeIndexRow.java
@@ -26,7 +26,7 @@ import static org.apache.carbondata.core.memory.CarbonUnsafe.getUnsafe;
 /**
  * Unsafe implementation of data map row.
  */
-public class UnsafeDataMapRow extends DataMapRow {
+public class UnsafeIndexRow extends IndexRow {
 
   private static final long serialVersionUID = -1156704133552046321L;
 
@@ -37,7 +37,7 @@ public class UnsafeDataMapRow extends DataMapRow {
 
   private int pointer;
 
-  public UnsafeDataMapRow(CarbonRowSchema[] schemas, MemoryBlock block, int pointer) {
+  public UnsafeIndexRow(CarbonRowSchema[] schemas, MemoryBlock block, int pointer) {
     super(schemas);
     this.block = block;
     this.pointer = pointer;
@@ -99,10 +99,10 @@ public class UnsafeDataMapRow extends DataMapRow {
   }
 
   @Override
-  public DataMapRow getRow(int ordinal) {
+  public IndexRow getRow(int ordinal) {
     CarbonRowSchema[] childSchemas =
         ((CarbonRowSchema.StructCarbonRowSchema) schemas[ordinal]).getChildSchemas();
-    return new UnsafeDataMapRow(childSchemas, block, pointer);
+    return new UnsafeIndexRow(childSchemas, block, pointer);
   }
 
   @Override
@@ -177,7 +177,7 @@ public class UnsafeDataMapRow extends DataMapRow {
   }
 
   @Override
-  public void setRow(DataMapRow row, int ordinal) {
+  public void setRow(IndexRow row, int ordinal) {
     throw new UnsupportedOperationException("Not supported to set on unsafe row");
   }
 }
diff --git a/core/src/main/java/org/apache/carbondata/core/indexstore/schema/SchemaGenerator.java b/core/src/main/java/org/apache/carbondata/core/indexstore/schema/SchemaGenerator.java
index de40efc..857ac05 100644
--- a/core/src/main/java/org/apache/carbondata/core/indexstore/schema/SchemaGenerator.java
+++ b/core/src/main/java/org/apache/carbondata/core/indexstore/schema/SchemaGenerator.java
@@ -24,10 +24,10 @@ import org.apache.carbondata.core.constants.CarbonCommonConstants;
 import org.apache.carbondata.core.datastore.block.SegmentProperties;
 import org.apache.carbondata.core.metadata.datatype.DataTypes;
 import org.apache.carbondata.core.metadata.schema.table.column.CarbonColumn;
-import org.apache.carbondata.core.util.BlockletDataMapUtil;
+import org.apache.carbondata.core.util.BlockletIndexUtil;
 
 /**
- * class for creating schema for a given DataMap
+ * class for creating schema for a given Index
  */
 public class SchemaGenerator {
 
@@ -285,7 +285,7 @@ public class SchemaGenerator {
       minMaxLen = new int[minMaxCacheColumns.size()];
       int counter = 0;
       for (CarbonColumn column : minMaxCacheColumns) {
-        minMaxLen[counter++] = segmentProperties.createColumnValueLength()[BlockletDataMapUtil
+        minMaxLen[counter++] = segmentProperties.createColumnValueLength()[BlockletIndexUtil
             .getColumnOrdinal(segmentProperties, column)];
       }
     } else {
@@ -309,7 +309,7 @@ public class SchemaGenerator {
       int counter = 0;
       for (CarbonColumn column : minMaxCacheColumns) {
         columnOrdinalsTOAccess[counter++] =
-            BlockletDataMapUtil.getColumnOrdinal(segmentProperties, column);
+            BlockletIndexUtil.getColumnOrdinal(segmentProperties, column);
       }
     } else {
       // when columns to cache is not specified then column access order will be same as the array
diff --git a/core/src/main/java/org/apache/carbondata/core/locks/CarbonLockUtil.java b/core/src/main/java/org/apache/carbondata/core/locks/CarbonLockUtil.java
index bcf15ee..5dd12d4 100644
--- a/core/src/main/java/org/apache/carbondata/core/locks/CarbonLockUtil.java
+++ b/core/src/main/java/org/apache/carbondata/core/locks/CarbonLockUtil.java
@@ -54,7 +54,7 @@ public class CarbonLockUtil {
       } else if (locktype.equals(LockUsage.DELETE_SEGMENT_LOCK)) {
         LOGGER.info("Delete segments lock has been successfully released");
       } else if (locktype.equals(LockUsage.DATAMAP_STATUS_LOCK)) {
-        LOGGER.info("DataMap status lock has been successfully released");
+        LOGGER.info("Index status lock has been successfully released");
       }
     } else {
       if (locktype.equals(LockUsage.METADATA_LOCK)) {
diff --git a/core/src/main/java/org/apache/carbondata/core/locks/LockUsage.java b/core/src/main/java/org/apache/carbondata/core/locks/LockUsage.java
index e0c04df..30b1829 100644
--- a/core/src/main/java/org/apache/carbondata/core/locks/LockUsage.java
+++ b/core/src/main/java/org/apache/carbondata/core/locks/LockUsage.java
@@ -35,6 +35,7 @@ public class LockUsage {
   public static final String DROP_TABLE_LOCK = "droptable.lock";
   public static final String STREAMING_LOCK = "streaming.lock";
   public static final String DATAMAP_STATUS_LOCK = "datamapstatus.lock";
+  public static final String MATERIALIZED_VIEW_STATUS_LOCK = "mvstatus.lock";
   public static final String CONCURRENT_LOAD_LOCK = "concurrentload.lock";
   public static final String UPDATE_LOCK = "update.lock";
   public static final String INGEST_LOCK = "ingest.lock";
diff --git a/core/src/main/java/org/apache/carbondata/core/metadata/SegmentFileStore.java b/core/src/main/java/org/apache/carbondata/core/metadata/SegmentFileStore.java
index af5724b..afb76bd 100644
--- a/core/src/main/java/org/apache/carbondata/core/metadata/SegmentFileStore.java
+++ b/core/src/main/java/org/apache/carbondata/core/metadata/SegmentFileStore.java
@@ -39,7 +39,7 @@ import org.apache.carbondata.common.logging.LogServiceFactory;
 import org.apache.carbondata.core.constants.CarbonCommonConstants;
 import org.apache.carbondata.core.datamap.DataMapStoreManager;
 import org.apache.carbondata.core.datamap.Segment;
-import org.apache.carbondata.core.datamap.TableDataMap;
+import org.apache.carbondata.core.datamap.TableIndex;
 import org.apache.carbondata.core.datastore.filesystem.CarbonFile;
 import org.apache.carbondata.core.datastore.filesystem.CarbonFileFilter;
 import org.apache.carbondata.core.datastore.impl.FileFactory;
@@ -577,11 +577,11 @@ public class SegmentFileStore {
    * @param segmentId
    */
   public static void clearBlockDataMapCache(CarbonTable carbonTable, String segmentId) {
-    TableDataMap defaultDataMap = DataMapStoreManager.getInstance().getDefaultDataMap(carbonTable);
+    TableIndex defaultDataMap = DataMapStoreManager.getInstance().getDefaultIndex(carbonTable);
     LOGGER.info(
         "clearing cache while updating segment file entry in table status file for segmentId: "
             + segmentId);
-    defaultDataMap.getDataMapFactory().clear(segmentId);
+    defaultDataMap.getIndexFactory().clear(segmentId);
   }
 
   private static CarbonFile[] getSegmentFiles(String segmentPath) {
@@ -936,7 +936,7 @@ public class SegmentFileStore {
     if (toBeDeleteSegments.size() > 0 || toBeUpdatedSegments.size() > 0) {
       Set<Segment> segmentSet = new HashSet<>(
           new SegmentStatusManager(carbonTable.getAbsoluteTableIdentifier())
-              .getValidAndInvalidSegments(carbonTable.isChildTableForMV()).getValidSegments());
+              .getValidAndInvalidSegments(carbonTable.isMV()).getValidSegments());
       CarbonUpdateUtil.updateTableMetadataStatus(segmentSet, carbonTable, uniqueId, true,
           Segment.toSegmentList(toBeDeleteSegments, null),
           Segment.toSegmentList(toBeUpdatedSegments, null), uuid);
diff --git a/core/src/main/java/org/apache/carbondata/core/metadata/schema/datamap/DataMapClassProvider.java b/core/src/main/java/org/apache/carbondata/core/metadata/schema/datamap/DataMapClassProvider.java
index 4095857..70da523 100644
--- a/core/src/main/java/org/apache/carbondata/core/metadata/schema/datamap/DataMapClassProvider.java
+++ b/core/src/main/java/org/apache/carbondata/core/metadata/schema/datamap/DataMapClassProvider.java
@@ -18,26 +18,20 @@
 package org.apache.carbondata.core.metadata.schema.datamap;
 
 /**
- * type for create datamap
- * The syntax of datamap creation is as follows.
- * CREATE DATAMAP IF NOT EXISTS dataMapName ON TABLE tableName USING 'DataMapClassProvider'
- * DMPROPERTIES('KEY'='VALUE') AS SELECT COUNT(COL1) FROM tableName
- *
  * Please refer {{org.apache.spark.sql.parser.CarbonSpark2SqlParser}}
  */
-
 public enum DataMapClassProvider {
-  LUCENE("org.apache.carbondata.datamap.lucene.LuceneFineGrainDataMapFactory", "lucene"),
-  BLOOMFILTER("org.apache.carbondata.datamap.bloom.BloomCoarseGrainDataMapFactory", "bloomfilter"),
+  LUCENE("org.apache.carbondata.datamap.lucene.LuceneFineGrainIndexFactory", "lucene"),
+  BLOOMFILTER("org.apache.carbondata.datamap.bloom.BloomCoarseGrainIndexFactory", "bloomfilter"),
   MV("org.apache.carbondata.core.datamap.MVDataMap", "mv");
 
   /**
-   * Fully qualified class name of datamap
+   * Fully qualified class name of index
    */
   private String className;
 
   /**
-   * Short name representation of datamap
+   * Short name representation of index
    */
   private String shortName;
 
@@ -57,16 +51,16 @@ public enum DataMapClassProvider {
   private boolean isEqual(String dataMapClass) {
     return (dataMapClass != null &&
         (dataMapClass.equals(className) ||
-        dataMapClass.equalsIgnoreCase(shortName)));
+            dataMapClass.equalsIgnoreCase(shortName)));
   }
 
-  public static DataMapClassProvider getDataMapProviderOnName(String dataMapShortname) {
-    if (LUCENE.isEqual(dataMapShortname)) {
+  public static DataMapClassProvider get(String indexProviderName) {
+    if (LUCENE.isEqual(indexProviderName)) {
       return LUCENE;
-    } else if (BLOOMFILTER.isEqual(dataMapShortname)) {
+    } else if (BLOOMFILTER.isEqual(indexProviderName)) {
       return BLOOMFILTER;
     } else {
-      throw new UnsupportedOperationException("Unknown datamap provider" + dataMapShortname);
+      throw new UnsupportedOperationException("Unknown index provider" + indexProviderName);
     }
   }
 }
diff --git a/core/src/main/java/org/apache/carbondata/core/metadata/schema/datamap/DataMapProperty.java b/core/src/main/java/org/apache/carbondata/core/metadata/schema/datamap/DataMapProperty.java
index f706b96..be62129 100644
--- a/core/src/main/java/org/apache/carbondata/core/metadata/schema/datamap/DataMapProperty.java
+++ b/core/src/main/java/org/apache/carbondata/core/metadata/schema/datamap/DataMapProperty.java
@@ -20,7 +20,7 @@ package org.apache.carbondata.core.metadata.schema.datamap;
 import org.apache.carbondata.common.annotations.InterfaceAudience;
 
 /**
- * Property that can be specified when creating DataMap
+ * Property that can be specified when creating Index
  */
 @InterfaceAudience.Internal
 public class DataMapProperty {
diff --git a/core/src/main/java/org/apache/carbondata/core/datamap/status/DataMapStatus.java b/core/src/main/java/org/apache/carbondata/core/metadata/schema/datamap/MVProviderName.java
similarity index 85%
copy from core/src/main/java/org/apache/carbondata/core/datamap/status/DataMapStatus.java
copy to core/src/main/java/org/apache/carbondata/core/metadata/schema/datamap/MVProviderName.java
index fce16c8..8af5f35 100644
--- a/core/src/main/java/org/apache/carbondata/core/datamap/status/DataMapStatus.java
+++ b/core/src/main/java/org/apache/carbondata/core/metadata/schema/datamap/MVProviderName.java
@@ -15,11 +15,8 @@
  * limitations under the License.
  */
 
-package org.apache.carbondata.core.datamap.status;
+package org.apache.carbondata.core.metadata.schema.datamap;
 
-/**
- * DataMap status
- */
-public enum DataMapStatus {
-  ENABLED, DISABLED, DROPPED
+public class MVProviderName {
+  public static final String NAME = "mv";
 }
diff --git a/core/src/main/java/org/apache/carbondata/core/metadata/schema/table/CarbonTable.java b/core/src/main/java/org/apache/carbondata/core/metadata/schema/table/CarbonTable.java
index e852f5f..3fb0596 100644
--- a/core/src/main/java/org/apache/carbondata/core/metadata/schema/table/CarbonTable.java
+++ b/core/src/main/java/org/apache/carbondata/core/metadata/schema/table/CarbonTable.java
@@ -30,14 +30,14 @@ import java.util.List;
 import java.util.Locale;
 import java.util.Map;
 
-import org.apache.carbondata.common.exceptions.sql.MalformedDataMapCommandException;
+import org.apache.carbondata.common.exceptions.sql.MalformedIndexCommandException;
 import org.apache.carbondata.common.logging.LogServiceFactory;
 import org.apache.carbondata.core.constants.CarbonCommonConstants;
 import org.apache.carbondata.core.constants.CarbonLoadOptionConstants;
 import org.apache.carbondata.core.constants.SortScopeOptions;
 import org.apache.carbondata.core.datamap.DataMapStoreManager;
-import org.apache.carbondata.core.datamap.TableDataMap;
-import org.apache.carbondata.core.datamap.dev.DataMapFactory;
+import org.apache.carbondata.core.datamap.TableIndex;
+import org.apache.carbondata.core.datamap.dev.IndexFactory;
 import org.apache.carbondata.core.datastore.block.SegmentProperties;
 import org.apache.carbondata.core.features.TableOperation;
 import org.apache.carbondata.core.metadata.AbsoluteTableIdentifier;
@@ -879,6 +879,15 @@ public class CarbonTable implements Serializable, Writable {
   /**
    * Return true if this table is a MV table (child table of other table)
    */
+  public boolean isMVTable() {
+    String parentTables = tableInfo.getFactTable().getTableProperties()
+        .get(CarbonCommonConstants.PARENT_TABLES);
+    return null != parentTables && !parentTables.isEmpty();
+  }
+
+  /**
+   * Return true if this table is a MV table (child table of other table)
+   */
   public boolean isChildTableForMV() {
     return null != tableInfo.getFactTable().getTableProperties()
         .get(CarbonCommonConstants.PARENT_TABLES) && !tableInfo.getFactTable().getTableProperties()
@@ -886,6 +895,16 @@ public class CarbonTable implements Serializable, Writable {
   }
 
   /**
+   * Return true if this table is a MV table (child table of other table)
+   */
+  public boolean isMV() {
+    return tableInfo.getFactTable().getTableProperties()
+        .get(CarbonCommonConstants.MV_RELATED_TABLES) != null &&
+        !tableInfo.getFactTable().getTableProperties()
+        .get(CarbonCommonConstants.MV_RELATED_TABLES).isEmpty();
+  }
+
+  /**
    * Return true if this is an external table (table with property "_external"="true", this is
    * an internal table property set during table creation)
    */
@@ -940,15 +959,15 @@ public class CarbonTable implements Serializable, Writable {
    */
   public boolean canAllow(CarbonTable carbonTable, TableOperation operation, Object... targets) {
     try {
-      List<TableDataMap> datamaps = DataMapStoreManager.getInstance().getAllDataMap(carbonTable);
-      if (!datamaps.isEmpty()) {
-        for (TableDataMap dataMap : datamaps) {
-          DataMapFactory factoryClass = DataMapStoreManager.getInstance()
+      List<TableIndex> indexes = DataMapStoreManager.getInstance().getAllIndexes(carbonTable);
+      if (!indexes.isEmpty()) {
+        for (TableIndex dataMap : indexes) {
+          IndexFactory factoryClass = DataMapStoreManager.getInstance()
               .getDataMapFactoryClass(carbonTable, dataMap.getDataMapSchema());
           if (factoryClass.willBecomeStale(operation)) {
             return false;
           }
-          // check whether the operation is blocked for datamap
+          // check whether the operation is blocked for index
           if (factoryClass.isOperationBlocked(operation, targets)) {
             return false;
           }
@@ -967,18 +986,18 @@ public class CarbonTable implements Serializable, Writable {
    * Get all index columns specified by dataMapSchema
    */
   public List<CarbonColumn> getIndexedColumns(DataMapSchema dataMapSchema)
-      throws MalformedDataMapCommandException {
+      throws MalformedIndexCommandException {
     String[] columns = dataMapSchema.getIndexColumns();
     List<CarbonColumn> indexColumn = new ArrayList<>(columns.length);
     for (String column : columns) {
       CarbonColumn carbonColumn = getColumnByName(column.trim().toLowerCase());
       if (carbonColumn == null) {
-        throw new MalformedDataMapCommandException(String
-            .format("column '%s' does not exist in table. Please check create DataMap statement.",
+        throw new MalformedIndexCommandException(String
+            .format("column '%s' does not exist in table. Please check create index statement.",
                 column));
       }
       if (carbonColumn.getColName().isEmpty()) {
-        throw new MalformedDataMapCommandException(
+        throw new MalformedIndexCommandException(
             CarbonCommonConstants.INDEX_COLUMNS + " contains invalid column name");
       }
       indexColumn.add(carbonColumn);
@@ -987,6 +1006,17 @@ public class CarbonTable implements Serializable, Writable {
   }
 
   /**
+   * is index exist
+   * @return true if exist, else return false
+   */
+  public boolean isIndexExist(String indexName) throws IOException {
+    List<DataMapSchema> schemas = DataMapStoreManager.getInstance().getAllDataMapSchemas();
+    return schemas.stream().anyMatch(schema ->
+        schema.getDataMapName().equalsIgnoreCase(indexName) &&
+            schema.getRelationIdentifier().getTableId().equals(getTableId()));
+  }
+
+  /**
    * Whether this table supports flat folder structure, it means all data files directly written
    * under table path
    */
diff --git a/core/src/main/java/org/apache/carbondata/core/metadata/schema/table/DataMapSchema.java b/core/src/main/java/org/apache/carbondata/core/metadata/schema/table/DataMapSchema.java
index 965064b..acd90ac 100644
--- a/core/src/main/java/org/apache/carbondata/core/metadata/schema/table/DataMapSchema.java
+++ b/core/src/main/java/org/apache/carbondata/core/metadata/schema/table/DataMapSchema.java
@@ -28,13 +28,13 @@ import java.util.Objects;
 import java.util.Set;
 
 import org.apache.carbondata.common.Strings;
-import org.apache.carbondata.common.exceptions.sql.MalformedDataMapCommandException;
+import org.apache.carbondata.common.exceptions.sql.MalformedIndexCommandException;
 import org.apache.carbondata.core.constants.CarbonCommonConstants;
-import org.apache.carbondata.core.datamap.DataMapUtil;
-import org.apache.carbondata.core.datamap.status.DataMapSegmentStatusUtil;
+import org.apache.carbondata.core.datamap.IndexUtil;
 import org.apache.carbondata.core.datamap.status.DataMapStatus;
 import org.apache.carbondata.core.datamap.status.DataMapStatusDetail;
 import org.apache.carbondata.core.datamap.status.DataMapStatusManager;
+import org.apache.carbondata.core.datamap.status.MVSegmentStatusUtil;
 import org.apache.carbondata.core.metadata.schema.datamap.DataMapClassProvider;
 import org.apache.carbondata.core.metadata.schema.datamap.DataMapProperty;
 import org.apache.carbondata.core.statusmanager.LoadMetadataDetails;
@@ -59,8 +59,8 @@ public class DataMapSchema implements Serializable, Writable {
 
   /**
    * There are two kind of DataMaps:
-   * 1. Index DataMap: provider name is class name of implementation class of DataMapFactory
-   * 2. MV DataMap: provider name is class name of {@code MVDataMapProvider}
+   * 1. Index Index: provider name is class name of implementation class of IndexFactory
+   * 2. MV Index: provider name is class name of {@code MVDataMapProvider}
    */
   // the old version the field name for providerName was className, so to de-serialization
   // old schema provided the old field name in the alternate filed using annotation
@@ -99,7 +99,7 @@ public class DataMapSchema implements Serializable, Writable {
   private Map<String, Set<String>> mainTableColumnList;
 
   /**
-   * DataMap table column order map as per Select query
+   * Index table column order map as per Select query
    */
   private Map<Integer, String> columnsOrderMap;
 
@@ -179,10 +179,10 @@ public class DataMapSchema implements Serializable, Writable {
   }
 
   /**
-   * Return true if this datamap is an Index DataMap
+   * Return true if this is an Index
    * @return
    */
-  public boolean isIndexDataMap() {
+  public boolean isIndex() {
     if (providerName.equalsIgnoreCase(DataMapClassProvider.MV.getShortName()) ||
         ctasQuery != null) {
       return false;
@@ -252,15 +252,15 @@ public class DataMapSchema implements Serializable, Writable {
    * Return the list of column name
    */
   public String[] getIndexColumns()
-      throws MalformedDataMapCommandException {
+      throws MalformedIndexCommandException {
     String columns = getProperties().get(INDEX_COLUMNS);
     if (columns == null) {
       columns = getProperties().get(INDEX_COLUMNS.toLowerCase());
     }
     if (columns == null) {
-      throw new MalformedDataMapCommandException(INDEX_COLUMNS + " DMPROPERTY is required");
+      throw new MalformedIndexCommandException(INDEX_COLUMNS + " DMPROPERTY is required");
     } else if (StringUtils.isBlank(columns)) {
-      throw new MalformedDataMapCommandException(INDEX_COLUMNS + " DMPROPERTY is blank");
+      throw new MalformedIndexCommandException(INDEX_COLUMNS + " DMPROPERTY is blank");
     } else {
       return columns.split(",", -1);
     }
@@ -304,7 +304,7 @@ public class DataMapSchema implements Serializable, Writable {
   }
 
   /**
-   * Return true if this DataMap can support incremental build
+   * Return true if this Index can support incremental build
    */
   public boolean canBeIncrementalBuild() {
     String prop = getProperties().get(DataMapProperty.FULL_REFRESH);
@@ -342,12 +342,12 @@ public class DataMapSchema implements Serializable, Writable {
     LoadMetadataDetails[] loads =
         SegmentStatusManager.readLoadMetadata(
             CarbonTablePath.getMetadataPath(this.getRelationIdentifier().getTablePath()));
-    if (!isIndexDataMap() && loads.length > 0) {
+    if (!isIndex() && loads.length > 0) {
       for (int i = loads.length - 1; i >= 0; i--) {
         LoadMetadataDetails load = loads[i];
         if (load.getSegmentStatus().equals(SegmentStatus.SUCCESS)) {
           Map<String, List<String>> segmentMaps =
-              DataMapSegmentStatusUtil.getSegmentMap(load.getExtraInfo());
+              MVSegmentStatusUtil.getSegmentMap(load.getExtraInfo());
           Map<String, String> syncInfoMap = new HashMap<>();
           for (Map.Entry<String, List<String>> entry : segmentMaps.entrySet()) {
             // when in join scenario, one table is loaded and one more is not loaded,
@@ -355,7 +355,7 @@ public class DataMapSchema implements Serializable, Writable {
             if (entry.getValue().isEmpty()) {
               syncInfoMap.put(entry.getKey(), "NA");
             } else {
-              syncInfoMap.put(entry.getKey(), DataMapUtil.getMaxSegmentID(entry.getValue()));
+              syncInfoMap.put(entry.getKey(), IndexUtil.getMaxSegmentID(entry.getValue()));
             }
           }
           String loadEndTime;
diff --git a/core/src/main/java/org/apache/carbondata/core/metadata/schema/table/DatabaseDMSchemaStorageProvider.java b/core/src/main/java/org/apache/carbondata/core/metadata/schema/table/DatabaseDMSchemaStorageProvider.java
index 56af077..8fe79cd 100644
--- a/core/src/main/java/org/apache/carbondata/core/metadata/schema/table/DatabaseDMSchemaStorageProvider.java
+++ b/core/src/main/java/org/apache/carbondata/core/metadata/schema/table/DatabaseDMSchemaStorageProvider.java
@@ -32,7 +32,7 @@ public class DatabaseDMSchemaStorageProvider implements DataMapSchemaStorageProv
 
   @Override
   public void saveSchema(DataMapSchema dataMapSchema) {
-    throw new UnsupportedOperationException("not support saving DataMap schema into database");
+    throw new UnsupportedOperationException("not support saving Index schema into database");
   }
 
   @Override
@@ -53,6 +53,6 @@ public class DatabaseDMSchemaStorageProvider implements DataMapSchemaStorageProv
 
   @Override
   public void dropSchema(String dataMapName) {
-    throw new UnsupportedOperationException("not support dropping DataMap schema from database");
+    throw new UnsupportedOperationException("not support dropping Index schema from database");
   }
 }
diff --git a/core/src/main/java/org/apache/carbondata/core/metadata/schema/table/DiskBasedDMSchemaStorageProvider.java b/core/src/main/java/org/apache/carbondata/core/metadata/schema/table/DiskBasedDMSchemaStorageProvider.java
index c18298d..a862c78 100644
--- a/core/src/main/java/org/apache/carbondata/core/metadata/schema/table/DiskBasedDMSchemaStorageProvider.java
+++ b/core/src/main/java/org/apache/carbondata/core/metadata/schema/table/DiskBasedDMSchemaStorageProvider.java
@@ -72,7 +72,7 @@ public class DiskBasedDMSchemaStorageProvider implements DataMapSchemaStoragePro
     String schemaPath = getSchemaPath(storePath, dataMapSchema.getDataMapName());
     if (FileFactory.isFileExist(schemaPath)) {
       throw new IOException(
-          "DataMap with name " + dataMapSchema.getDataMapName() + " already exists in storage");
+              "Index with name " + dataMapSchema.getDataMapName() + " already exists in storage");
     }
     // write the datamap shema in json format.
     try {
@@ -172,17 +172,17 @@ public class DiskBasedDMSchemaStorageProvider implements DataMapSchemaStoragePro
       throws IOException {
     String schemaPath = getSchemaPath(storePath, dataMapName);
     if (!FileFactory.isFileExist(schemaPath)) {
-      throw new IOException("DataMap with name " + dataMapName + " does not exists in storage");
+      throw new IOException("Index with name " + dataMapName + " does not exists in storage");
     }
 
-    LOG.info(String.format("Trying to delete DataMap %s schema", dataMapName));
+    LOG.info(String.format("Trying to delete Index %s schema", dataMapName));
 
     dataMapSchemas.removeIf(schema -> schema.getDataMapName().equalsIgnoreCase(dataMapName));
     touchMDTFile();
     if (!FileFactory.deleteFile(schemaPath)) {
-      throw new IOException("DataMap with name " + dataMapName + " cannot be deleted");
+      throw new IOException("Index with name " + dataMapName + " cannot be deleted");
     }
-    LOG.info(String.format("DataMap %s schema is deleted", dataMapName));
+    LOG.info(String.format("Index %s schema is deleted", dataMapName));
   }
 
   private void checkAndReloadDataMapSchemas(boolean touchFile) throws IOException {
diff --git a/core/src/main/java/org/apache/carbondata/core/profiler/ExplainCollector.java b/core/src/main/java/org/apache/carbondata/core/profiler/ExplainCollector.java
index 75568e4..a4dc4df 100644
--- a/core/src/main/java/org/apache/carbondata/core/profiler/ExplainCollector.java
+++ b/core/src/main/java/org/apache/carbondata/core/profiler/ExplainCollector.java
@@ -27,7 +27,7 @@ import java.util.concurrent.ConcurrentHashMap;
 
 import org.apache.carbondata.common.annotations.InterfaceAudience;
 import org.apache.carbondata.core.constants.CarbonCommonConstants;
-import org.apache.carbondata.core.datamap.dev.expr.DataMapWrapperSimpleInfo;
+import org.apache.carbondata.core.datamap.dev.expr.IndexWrapperSimpleInfo;
 import org.apache.carbondata.core.util.CarbonProperties;
 
 /**
@@ -122,19 +122,19 @@ public class ExplainCollector {
     }
   }
 
-  public static void recordCGDataMapPruning(DataMapWrapperSimpleInfo dataMapWrapperSimpleInfo,
+  public static void recordCGDataMapPruning(IndexWrapperSimpleInfo indexWrapperSimpleInfo,
       int numBlocklets, int numBlocks) {
     if (enabled()) {
       TablePruningInfo scan = getCurrentTablePruningInfo();
-      scan.setNumBlockletsAfterCGPruning(dataMapWrapperSimpleInfo, numBlocklets, numBlocks);
+      scan.setNumBlockletsAfterCGPruning(indexWrapperSimpleInfo, numBlocklets, numBlocks);
     }
   }
 
-  public static void recordFGDataMapPruning(DataMapWrapperSimpleInfo dataMapWrapperSimpleInfo,
+  public static void recordFGDataMapPruning(IndexWrapperSimpleInfo indexWrapperSimpleInfo,
       int numBlocklets, int numBlocks) {
     if (enabled()) {
       TablePruningInfo scan = getCurrentTablePruningInfo();
-      scan.setNumBlockletsAfterFGPruning(dataMapWrapperSimpleInfo, numBlocklets, numBlocks);
+      scan.setNumBlockletsAfterFGPruning(indexWrapperSimpleInfo, numBlocklets, numBlocks);
     }
   }
 
@@ -183,7 +183,7 @@ public class ExplainCollector {
     StringBuilder builder = new StringBuilder();
     for (int i = 0; i < olapDataMapProviders.size(); i++) {
       if (i == 0) {
-        builder.append("Query rewrite based on DataMap:").append("\n");
+        builder.append("Query rewrite based on Index:").append("\n");
       }
       builder.append(" - ").append(olapDataMapNames.get(i)).append(" (")
           .append(olapDataMapProviders.get(i)).append(")").append("\n");
diff --git a/core/src/main/java/org/apache/carbondata/core/profiler/TablePruningInfo.java b/core/src/main/java/org/apache/carbondata/core/profiler/TablePruningInfo.java
index 424d7c7..f4cc720 100644
--- a/core/src/main/java/org/apache/carbondata/core/profiler/TablePruningInfo.java
+++ b/core/src/main/java/org/apache/carbondata/core/profiler/TablePruningInfo.java
@@ -18,7 +18,7 @@
 package org.apache.carbondata.core.profiler;
 
 import org.apache.carbondata.common.annotations.InterfaceAudience;
-import org.apache.carbondata.core.datamap.dev.expr.DataMapWrapperSimpleInfo;
+import org.apache.carbondata.core.datamap.dev.expr.IndexWrapperSimpleInfo;
 
 /**
  * Used for EXPLAIN command
@@ -34,11 +34,11 @@ public class TablePruningInfo {
   private int numBlocksAfterDefaultPruning;
   private int numBlockletsAfterDefaultPruning = 0;
 
-  private DataMapWrapperSimpleInfo cgDataMap;
+  private IndexWrapperSimpleInfo cgDataMap;
   private int numBlocksAfterCGPruning;
   private int numBlockletsAfterCGPruning;
 
-  private DataMapWrapperSimpleInfo fgDataMap;
+  private IndexWrapperSimpleInfo fgDataMap;
   private int numBlocksAfterFGPruning;
   private int numBlockletsAfterFGPruning;
 
@@ -71,16 +71,16 @@ public class TablePruningInfo {
     this.numBlockletsAfterDefaultPruning += numBlocklets;
   }
 
-  void setNumBlockletsAfterCGPruning(DataMapWrapperSimpleInfo dataMapWrapperSimpleInfo,
+  void setNumBlockletsAfterCGPruning(IndexWrapperSimpleInfo indexWrapperSimpleInfo,
       int numBlocklets, int numBlocks) {
-    this.cgDataMap = dataMapWrapperSimpleInfo;
+    this.cgDataMap = indexWrapperSimpleInfo;
     this.numBlocksAfterCGPruning = numBlocks;
     this.numBlockletsAfterCGPruning = numBlocklets;
   }
 
-  void setNumBlockletsAfterFGPruning(DataMapWrapperSimpleInfo dataMapWrapperSimpleInfo,
+  void setNumBlockletsAfterFGPruning(IndexWrapperSimpleInfo indexWrapperSimpleInfo,
       int numBlocklets, int numBlocks) {
-    this.fgDataMap = dataMapWrapperSimpleInfo;
+    this.fgDataMap = indexWrapperSimpleInfo;
     this.numBlocksAfterFGPruning = numBlocks;
     this.numBlockletsAfterFGPruning = numBlocklets;
   }
@@ -96,14 +96,14 @@ public class TablePruningInfo {
       int skipBlocks = totalBlocks - numBlocksAfterDefaultPruning;
       int skipBlocklets = totalBlocklets - numBlockletsAfterDefaultPruning;
       builder
-          .append(" - pruned by Main DataMap").append("\n")
+          .append(" - pruned by Main Index").append("\n")
           .append("    - skipped: ").append(skipBlocks).append(" blocks, ")
           .append(skipBlocklets).append(" blocklets").append("\n");
       if (cgDataMap != null) {
         skipBlocks = numBlocksAfterDefaultPruning - numBlocksAfterCGPruning;
         skipBlocklets = numBlockletsAfterDefaultPruning - numBlockletsAfterCGPruning;
         builder
-            .append(" - pruned by CG DataMap").append("\n")
+            .append(" - pruned by CG Index").append("\n")
             .append("    - name: ").append(cgDataMap.getDataMapWrapperName()).append("\n")
             .append("    - provider: ").append(cgDataMap.getDataMapWrapperProvider()).append("\n")
             .append("    - skipped: ").append(skipBlocks).append(" blocks, ")
@@ -118,7 +118,7 @@ public class TablePruningInfo {
           skipBlocklets = numBlockletsAfterDefaultPruning - numBlockletsAfterFGPruning;
         }
         builder
-            .append(" - pruned by FG DataMap").append("\n")
+            .append(" - pruned by FG Index").append("\n")
             .append("    - name: ").append(fgDataMap.getDataMapWrapperName()).append("\n")
             .append("    - provider: ").append(fgDataMap.getDataMapWrapperProvider()).append("\n")
             .append("    - skipped: ").append(skipBlocks).append(" blocks, ")
diff --git a/core/src/main/java/org/apache/carbondata/core/scan/executor/impl/AbstractQueryExecutor.java b/core/src/main/java/org/apache/carbondata/core/scan/executor/impl/AbstractQueryExecutor.java
index dbf405a..05d3876 100644
--- a/core/src/main/java/org/apache/carbondata/core/scan/executor/impl/AbstractQueryExecutor.java
+++ b/core/src/main/java/org/apache/carbondata/core/scan/executor/impl/AbstractQueryExecutor.java
@@ -34,7 +34,7 @@ import org.apache.carbondata.common.CarbonIterator;
 import org.apache.carbondata.common.logging.LogServiceFactory;
 import org.apache.carbondata.core.constants.CarbonCommonConstants;
 import org.apache.carbondata.core.constants.CarbonV3DataFormatConstants;
-import org.apache.carbondata.core.datamap.DataMapFilter;
+import org.apache.carbondata.core.datamap.IndexFilter;
 import org.apache.carbondata.core.datamap.Segment;
 import org.apache.carbondata.core.datastore.ReusableDataBuffer;
 import org.apache.carbondata.core.datastore.block.AbstractIndex;
@@ -62,7 +62,7 @@ import org.apache.carbondata.core.scan.model.ProjectionMeasure;
 import org.apache.carbondata.core.scan.model.QueryModel;
 import org.apache.carbondata.core.stats.QueryStatistic;
 import org.apache.carbondata.core.stats.QueryStatisticsConstants;
-import org.apache.carbondata.core.util.BlockletDataMapUtil;
+import org.apache.carbondata.core.util.BlockletIndexUtil;
 import org.apache.carbondata.core.util.CarbonProperties;
 import org.apache.carbondata.core.util.CarbonTimeStatisticsFactory;
 import org.apache.carbondata.core.util.CarbonUtil;
@@ -145,8 +145,8 @@ public abstract class AbstractQueryExecutor<E> implements QueryExecutor<E> {
     // and measure column start index
     queryProperties.filterMeasures = new HashSet<>();
     queryProperties.complexFilterDimension = new HashSet<>();
-    if (queryModel.getDataMapFilter() != null) {
-      QueryUtil.getAllFilterDimensionsAndMeasures(queryModel.getDataMapFilter().getResolver(),
+    if (queryModel.getIndexFilter() != null) {
+      QueryUtil.getAllFilterDimensionsAndMeasures(queryModel.getIndexFilter().getResolver(),
           queryProperties.complexFilterDimension, queryProperties.filterMeasures);
     }
 
@@ -248,7 +248,7 @@ public abstract class AbstractQueryExecutor<E> implements QueryExecutor<E> {
     }
     // First validate the schema of the carbondata file if the same column name have different
     // datatype
-    boolean sameColumnSchemaList = BlockletDataMapUtil
+    boolean sameColumnSchemaList = BlockletIndexUtil
         .isSameColumnAndDifferentDatatypeInSchema(columnsInTable,
             queryModel.getTable().getTableInfo().getFactTable().getListOfColumns());
     if (!sameColumnSchemaList) {
@@ -305,11 +305,11 @@ public abstract class AbstractQueryExecutor<E> implements QueryExecutor<E> {
   }
 
   private void createFilterExpression(QueryModel queryModel, SegmentProperties properties) {
-    if (queryModel.getDataMapFilter() != null) {
-      if (!queryModel.getDataMapFilter().isResolvedOnSegment(properties)) {
-        DataMapFilter expression = new DataMapFilter(properties, queryModel.getTable(),
-            queryModel.getDataMapFilter().getExpression());
-        queryModel.setDataMapFilter(expression);
+    if (queryModel.getIndexFilter() != null) {
+      if (!queryModel.getIndexFilter().isResolvedOnSegment(properties)) {
+        IndexFilter expression = new IndexFilter(properties, queryModel.getTable(),
+            queryModel.getIndexFilter().getExpression());
+        queryModel.setIndexFilter(expression);
       }
     }
   }
@@ -485,13 +485,13 @@ public abstract class AbstractQueryExecutor<E> implements QueryExecutor<E> {
             projectDimensions,
             segmentProperties.getDimensionOrdinalToChunkMapping(),
             queryProperties.complexFilterDimension));
-    if (null != queryModel.getDataMapFilter()) {
+    if (null != queryModel.getIndexFilter()) {
       FilterResolverIntf filterResolverIntf;
       if (!filePath.startsWith(queryModel.getTable().getTablePath())) {
-        filterResolverIntf = queryModel.getDataMapFilter().getExternalSegmentResolver();
+        filterResolverIntf = queryModel.getIndexFilter().getExternalSegmentResolver();
       } else {
         // loading the filter executor tree for filter evaluation
-        filterResolverIntf = queryModel.getDataMapFilter().getResolver();
+        filterResolverIntf = queryModel.getIndexFilter().getResolver();
       }
       blockExecutionInfo.setFilterExecuterTree(
           FilterUtil.getFilterExecuterTree(filterResolverIntf, segmentProperties,
diff --git a/core/src/main/java/org/apache/carbondata/core/scan/model/QueryModel.java b/core/src/main/java/org/apache/carbondata/core/scan/model/QueryModel.java
index 26301c6..54babaa 100644
--- a/core/src/main/java/org/apache/carbondata/core/scan/model/QueryModel.java
+++ b/core/src/main/java/org/apache/carbondata/core/scan/model/QueryModel.java
@@ -20,7 +20,7 @@ package org.apache.carbondata.core.scan.model;
 import java.util.ArrayList;
 import java.util.List;
 
-import org.apache.carbondata.core.datamap.DataMapFilter;
+import org.apache.carbondata.core.datamap.IndexFilter;
 import org.apache.carbondata.core.datastore.block.TableBlockInfo;
 import org.apache.carbondata.core.metadata.AbsoluteTableIdentifier;
 import org.apache.carbondata.core.metadata.schema.table.CarbonTable;
@@ -55,7 +55,7 @@ public class QueryModel {
   /**
    * filter expression tree
    */
-  private DataMapFilter dataMapFilter;
+  private IndexFilter indexFilter;
 
   /**
    * table block information in which query will be executed
@@ -271,12 +271,12 @@ public class QueryModel {
     this.tableBlockInfos = tableBlockInfos;
   }
 
-  public DataMapFilter getDataMapFilter() {
-    return dataMapFilter;
+  public IndexFilter getIndexFilter() {
+    return indexFilter;
   }
 
-  public void setDataMapFilter(DataMapFilter dataMapFilter) {
-    this.dataMapFilter = dataMapFilter;
+  public void setIndexFilter(IndexFilter indexFilter) {
+    this.indexFilter = indexFilter;
   }
 
   /**
@@ -394,7 +394,7 @@ public class QueryModel {
     return String.format("scan on table %s.%s, %d projection columns with filter (%s)",
         table.getDatabaseName(), table.getTableName(),
         projection.getDimensions().size() + projection.getMeasures().size(),
-        dataMapFilter.getExpression().toString());
+        indexFilter.getExpression().toString());
   }
 
   public boolean isFreeUnsafeMemory() {
diff --git a/core/src/main/java/org/apache/carbondata/core/scan/model/QueryModelBuilder.java b/core/src/main/java/org/apache/carbondata/core/scan/model/QueryModelBuilder.java
index 696993d..77b9bc8 100644
--- a/core/src/main/java/org/apache/carbondata/core/scan/model/QueryModelBuilder.java
+++ b/core/src/main/java/org/apache/carbondata/core/scan/model/QueryModelBuilder.java
@@ -25,7 +25,7 @@ import java.util.Map;
 import java.util.Objects;
 
 import org.apache.carbondata.common.logging.LogServiceFactory;
-import org.apache.carbondata.core.datamap.DataMapFilter;
+import org.apache.carbondata.core.datamap.IndexFilter;
 import org.apache.carbondata.core.metadata.datatype.DataTypes;
 import org.apache.carbondata.core.metadata.schema.table.CarbonTable;
 import org.apache.carbondata.core.metadata.schema.table.column.CarbonDimension;
@@ -38,7 +38,7 @@ public class QueryModelBuilder {
 
   private CarbonTable table;
   private QueryProjection projection;
-  private DataMapFilter dataMapFilter;
+  private IndexFilter indexFilter;
   private DataTypeConverter dataTypeConverter;
   private boolean forcedDetailRawQuery;
   private boolean readPageByPage;
@@ -286,8 +286,8 @@ public class QueryModelBuilder {
     return this;
   }
 
-  public QueryModelBuilder filterExpression(DataMapFilter filterExpression) {
-    this.dataMapFilter = filterExpression;
+  public QueryModelBuilder filterExpression(IndexFilter filterExpression) {
+    this.indexFilter = filterExpression;
     return this;
   }
 
@@ -328,15 +328,15 @@ public class QueryModelBuilder {
       queryModel.setIsFilterDimensions(isFilterDimensions);
       queryModel.setIsFilterMeasures(isFilterMeasures);
       // In case of Dictionary Include Range Column we donot optimize the range expression
-      if (dataMapFilter != null) {
+      if (indexFilter != null) {
         if (isConvertToRangeFilter()) {
-          dataMapFilter.processFilterExpression(isFilterDimensions, isFilterMeasures);
+          indexFilter.processFilterExpression(isFilterDimensions, isFilterMeasures);
         } else {
-          dataMapFilter.processFilterExpressionWithoutRange(isFilterDimensions, isFilterMeasures);
+          indexFilter.processFilterExpressionWithoutRange(isFilterDimensions, isFilterMeasures);
         }
       }
     }
-    queryModel.setDataMapFilter(dataMapFilter);
+    queryModel.setIndexFilter(indexFilter);
     return queryModel;
   }
 }
diff --git a/core/src/main/java/org/apache/carbondata/core/statusmanager/SegmentStatusManager.java b/core/src/main/java/org/apache/carbondata/core/statusmanager/SegmentStatusManager.java
index cb29b36..c2d6db0 100755
--- a/core/src/main/java/org/apache/carbondata/core/statusmanager/SegmentStatusManager.java
+++ b/core/src/main/java/org/apache/carbondata/core/statusmanager/SegmentStatusManager.java
@@ -49,6 +49,7 @@ import org.apache.carbondata.core.locks.LockUsage;
 import org.apache.carbondata.core.metadata.AbsoluteTableIdentifier;
 import org.apache.carbondata.core.metadata.CarbonTableIdentifier;
 import org.apache.carbondata.core.metadata.schema.table.CarbonTable;
+import org.apache.carbondata.core.metadata.schema.table.RelationIdentifier;
 import org.apache.carbondata.core.mutate.CarbonUpdateUtil;
 import org.apache.carbondata.core.readcommitter.ReadCommittedScope;
 import org.apache.carbondata.core.readcommitter.TableStatusReadCommittedScope;
@@ -225,6 +226,24 @@ public class SegmentStatusManager {
   }
 
   /**
+   * Returns valid segment list for a given RelationIdentifier
+   *
+   * @param relationIdentifier get list of segments for relation identifier
+   * @return list of valid segment id's
+   */
+  public static List<String> getValidSegmentList(RelationIdentifier relationIdentifier)
+      throws IOException {
+    List<String> segmentList = new ArrayList<>();
+    List<Segment> validSegments = new SegmentStatusManager(AbsoluteTableIdentifier
+        .from(relationIdentifier.getTablePath(), relationIdentifier.getDatabaseName(),
+            relationIdentifier.getTableName())).getValidAndInvalidSegments().getValidSegments();
+    for (Segment segment : validSegments) {
+      segmentList.add(segment.getSegmentNo());
+    }
+    return segmentList;
+  }
+
+  /**
    * Reads the table status file with the specified UUID if non empty.
    */
   public static LoadMetadataDetails[] readLoadMetadata(String metaDataFolderPath, String uuid)
diff --git a/core/src/main/java/org/apache/carbondata/core/util/BlockletDataMapUtil.java b/core/src/main/java/org/apache/carbondata/core/util/BlockletIndexUtil.java
similarity index 98%
rename from core/src/main/java/org/apache/carbondata/core/util/BlockletDataMapUtil.java
rename to core/src/main/java/org/apache/carbondata/core/util/BlockletIndexUtil.java
index 237d349..c5fb351 100644
--- a/core/src/main/java/org/apache/carbondata/core/util/BlockletDataMapUtil.java
+++ b/core/src/main/java/org/apache/carbondata/core/util/BlockletIndexUtil.java
@@ -46,8 +46,8 @@ import org.apache.carbondata.core.datastore.impl.FileFactory;
 import org.apache.carbondata.core.indexstore.BlockMetaInfo;
 import org.apache.carbondata.core.indexstore.TableBlockIndexUniqueIdentifier;
 import org.apache.carbondata.core.indexstore.TableBlockIndexUniqueIdentifierWrapper;
-import org.apache.carbondata.core.indexstore.blockletindex.BlockletDataMapDistributable;
-import org.apache.carbondata.core.indexstore.blockletindex.BlockletDataMapFactory;
+import org.apache.carbondata.core.indexstore.blockletindex.BlockletIndexFactory;
+import org.apache.carbondata.core.indexstore.blockletindex.BlockletIndexInputSplit;
 import org.apache.carbondata.core.indexstore.blockletindex.SegmentIndexFileStore;
 import org.apache.carbondata.core.metadata.blocklet.DataFileFooter;
 import org.apache.carbondata.core.metadata.blocklet.index.BlockletMinMaxIndex;
@@ -66,10 +66,10 @@ import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.fs.PathFilter;
 import org.apache.log4j.Logger;
 
-public class BlockletDataMapUtil {
+public class BlockletIndexUtil {
 
   private static final Logger LOG =
-      LogServiceFactory.getLogService(BlockletDataMapUtil.class.getName());
+      LogServiceFactory.getLogService(BlockletIndexUtil.class.getName());
 
   public static Set<TableBlockIndexUniqueIdentifier> getSegmentUniqueIdentifiers(Segment segment)
       throws IOException {
@@ -220,7 +220,7 @@ public class BlockletDataMapUtil {
    */
   public static TableBlockIndexUniqueIdentifier filterIdentifiersBasedOnDistributable(
       Set<TableBlockIndexUniqueIdentifier> tableBlockIndexUniqueIdentifiers,
-      BlockletDataMapDistributable distributable) {
+      BlockletIndexInputSplit distributable) {
     TableBlockIndexUniqueIdentifier validIdentifier = null;
     String fileName = CarbonTablePath.DataFileUtil.getFileName(distributable.getFilePath());
     for (TableBlockIndexUniqueIdentifier tableBlockIndexUniqueIdentifier :
@@ -265,7 +265,7 @@ public class BlockletDataMapUtil {
   public static boolean isCacheLevelBlock(CarbonTable carbonTable) {
     String cacheLevel = carbonTable.getTableInfo().getFactTable().getTableProperties()
         .get(CarbonCommonConstants.CACHE_LEVEL);
-    if (BlockletDataMapFactory.CACHE_LEVEL_BLOCKLET.equals(cacheLevel)) {
+    if (BlockletIndexFactory.CACHE_LEVEL_BLOCKLET.equals(cacheLevel)) {
       return false;
     }
     return true;
diff --git a/core/src/main/java/org/apache/carbondata/core/util/CarbonProperties.java b/core/src/main/java/org/apache/carbondata/core/util/CarbonProperties.java
index f4ca24e..eccbc57 100644
--- a/core/src/main/java/org/apache/carbondata/core/util/CarbonProperties.java
+++ b/core/src/main/java/org/apache/carbondata/core/util/CarbonProperties.java
@@ -1583,6 +1583,14 @@ public final class CarbonProperties {
    * @return
    */
   public String getSystemFolderLocation() {
+    return getSystemFolderLocation(null);
+  }
+
+  /**
+   * Get the configured system folder location.
+   * @return
+   */
+  public String getSystemFolderLocation(String databaseName) {
     String systemLocation = CarbonProperties.getInstance()
         .getProperty(CarbonCommonConstants.CARBON_SYSTEM_FOLDER_LOCATION);
     if (systemLocation == null) {
@@ -1592,7 +1600,12 @@ public final class CarbonProperties {
       systemLocation = CarbonUtil.checkAndAppendFileSystemURIScheme(systemLocation);
       systemLocation = FileFactory.getUpdatedFilePath(systemLocation);
     }
-    return systemLocation + CarbonCommonConstants.FILE_SEPARATOR + "_system";
+    if (databaseName == null) {
+      return systemLocation + CarbonCommonConstants.FILE_SEPARATOR + "_system";
+    } else {
+      return systemLocation + CarbonCommonConstants.FILE_SEPARATOR +
+          databaseName + CarbonCommonConstants.FILE_SEPARATOR + "_system";
+    }
   }
 
   /**
diff --git a/core/src/main/java/org/apache/carbondata/core/util/CarbonUtil.java b/core/src/main/java/org/apache/carbondata/core/util/CarbonUtil.java
index b3d9339..d00da15 100644
--- a/core/src/main/java/org/apache/carbondata/core/util/CarbonUtil.java
+++ b/core/src/main/java/org/apache/carbondata/core/util/CarbonUtil.java
@@ -3084,7 +3084,7 @@ public final class CarbonUtil {
       SegmentStatusManager segmentStatusManager =
           new SegmentStatusManager(carbonTable.getAbsoluteTableIdentifier());
       SegmentStatusManager.ValidAndInvalidSegmentsInfo validAndInvalidSegmentsInfo =
-          segmentStatusManager.getValidAndInvalidSegments(carbonTable.isChildTableForMV());
+          segmentStatusManager.getValidAndInvalidSegments(carbonTable.isMV());
       List<Segment> validSegments = validAndInvalidSegmentsInfo.getValidSegments();
       if (validSegments.isEmpty()) {
         return carbonProperties.getFormatVersion();
diff --git a/core/src/main/java/org/apache/carbondata/core/util/DeleteLoadFolders.java b/core/src/main/java/org/apache/carbondata/core/util/DeleteLoadFolders.java
index 4287c42..bb0b3bc 100644
--- a/core/src/main/java/org/apache/carbondata/core/util/DeleteLoadFolders.java
+++ b/core/src/main/java/org/apache/carbondata/core/util/DeleteLoadFolders.java
@@ -24,7 +24,7 @@ import java.util.List;
 import org.apache.carbondata.common.logging.LogServiceFactory;
 import org.apache.carbondata.core.datamap.DataMapStoreManager;
 import org.apache.carbondata.core.datamap.Segment;
-import org.apache.carbondata.core.datamap.TableDataMap;
+import org.apache.carbondata.core.datamap.TableIndex;
 import org.apache.carbondata.core.datastore.filesystem.CarbonFile;
 import org.apache.carbondata.core.datastore.filesystem.CarbonFileFilter;
 import org.apache.carbondata.core.datastore.impl.FileFactory;
@@ -97,10 +97,10 @@ public final class DeleteLoadFolders {
   private static void physicalFactAndMeasureMetadataDeletion(CarbonTable carbonTable,
       LoadMetadataDetails[] loadDetails, boolean isForceDelete, List<PartitionSpec> specs,
       LoadMetadataDetails[] currLoadDetails) {
-    List<TableDataMap> indexDataMaps = new ArrayList<>();
+    List<TableIndex> indexDataMaps = new ArrayList<>();
     try {
-      for (TableDataMap dataMap : DataMapStoreManager.getInstance().getAllDataMap(carbonTable)) {
-        if (dataMap.getDataMapSchema().isIndexDataMap()) {
+      for (TableIndex dataMap : DataMapStoreManager.getInstance().getAllIndexes(carbonTable)) {
+        if (dataMap.getDataMapSchema().isIndex()) {
           indexDataMaps.add(dataMap);
         }
       }
@@ -160,7 +160,7 @@ public final class DeleteLoadFolders {
             }
           }
           List<Segment> segments = new ArrayList<>(1);
-          for (TableDataMap dataMap : indexDataMaps) {
+          for (TableIndex dataMap : indexDataMaps) {
             segments.clear();
             segments.add(new Segment(oneLoad.getLoadName()));
             dataMap.deleteDatamapData(segments);
diff --git a/core/src/main/java/org/apache/carbondata/core/util/SessionParams.java b/core/src/main/java/org/apache/carbondata/core/util/SessionParams.java
index c9869dd..4b3a8fc 100644
--- a/core/src/main/java/org/apache/carbondata/core/util/SessionParams.java
+++ b/core/src/main/java/org/apache/carbondata/core/util/SessionParams.java
@@ -35,6 +35,7 @@ import static org.apache.carbondata.core.constants.CarbonCommonConstants.CARBON_
 import static org.apache.carbondata.core.constants.CarbonCommonConstants.CARBON_PUSH_ROW_FILTERS_FOR_VECTOR;
 import static org.apache.carbondata.core.constants.CarbonCommonConstants.CARBON_QUERY_STAGE_INPUT;
 import static org.apache.carbondata.core.constants.CarbonCommonConstants.COMPACTION_SEGMENT_LEVEL_THRESHOLD;
+import static org.apache.carbondata.core.constants.CarbonCommonConstants.DISABLE_SQL_REWRITE;
 import static org.apache.carbondata.core.constants.CarbonCommonConstants.ENABLE_AUTO_LOAD_MERGE;
 import static org.apache.carbondata.core.constants.CarbonCommonConstants.ENABLE_OFFHEAP_SORT;
 import static org.apache.carbondata.core.constants.CarbonCommonConstants.ENABLE_SI_LOOKUP_PARTIALSTRING;
@@ -154,6 +155,7 @@ public class SessionParams implements Serializable, Cloneable {
       case CARBON_PUSH_ROW_FILTERS_FOR_VECTOR:
       case CARBON_ENABLE_INDEX_SERVER:
       case CARBON_QUERY_STAGE_INPUT:
+      case DISABLE_SQL_REWRITE:
         isValid = CarbonUtil.validateBoolean(value);
         if (!isValid) {
           throw new InvalidConfigurationException("Invalid value " + value + " for key " + key);
diff --git a/core/src/main/java/org/apache/carbondata/core/datamap/DataMapCatalog.java b/core/src/main/java/org/apache/carbondata/core/view/MVCatalog.java
similarity index 61%
rename from core/src/main/java/org/apache/carbondata/core/datamap/DataMapCatalog.java
rename to core/src/main/java/org/apache/carbondata/core/view/MVCatalog.java
index d4eb0c3..0cecee9 100644
--- a/core/src/main/java/org/apache/carbondata/core/datamap/DataMapCatalog.java
+++ b/core/src/main/java/org/apache/carbondata/core/view/MVCatalog.java
@@ -15,42 +15,33 @@
  * limitations under the License.
  */
 
-package org.apache.carbondata.core.datamap;
+package org.apache.carbondata.core.view;
 
-import org.apache.carbondata.core.metadata.schema.table.DataMapSchema;
+import org.apache.carbondata.core.metadata.schema.table.RelationIdentifier;
 
 /**
- * This is the interface for inmemory catalog registry for datamap.
+ * This is the interface for inmemory catalog registry for mv.
+ * For query rewrite.
  * @since 1.4.0
  */
-public interface DataMapCatalog<T> {
-
-  /**
-   * Register schema to the catalog.
-   * @param dataMapSchema
-   */
-  void registerSchema(DataMapSchema dataMapSchema);
-
-  /**
-   * Unregister schema from catalog.
-   * @param dataMapName
-   */
-  void unregisterSchema(String dataMapName);
+public interface MVCatalog<T> {
 
   /**
    * List all registered valid schema catalogs
-   * @return
    */
-  T[] listAllValidSchema();
+  T[] getValidSchemas();
 
   /**
-   * It reloads/removes all registered schema catalogs
+   * Register schema to the catalog.
+   * @param schema mv schema
    */
-  void refresh();
+  void registerSchema(MVSchema schema);
 
   /**
-   * This checks whether the datamapSchema is already registered
+   * Unregister schema from catalog.
+   *
+   * @param mvIdentifier mv identifier
    */
-  Boolean isMVExists(String mvName);
+  void deregisterSchema(RelationIdentifier mvIdentifier);
 
 }
diff --git a/core/src/main/java/org/apache/carbondata/core/datamap/status/DataMapStatus.java b/core/src/main/java/org/apache/carbondata/core/view/MVCatalogFactory.java
similarity index 79%
copy from core/src/main/java/org/apache/carbondata/core/datamap/status/DataMapStatus.java
copy to core/src/main/java/org/apache/carbondata/core/view/MVCatalogFactory.java
index fce16c8..683284a 100644
--- a/core/src/main/java/org/apache/carbondata/core/datamap/status/DataMapStatus.java
+++ b/core/src/main/java/org/apache/carbondata/core/view/MVCatalogFactory.java
@@ -15,11 +15,16 @@
  * limitations under the License.
  */
 
-package org.apache.carbondata.core.datamap.status;
+package org.apache.carbondata.core.view;
 
 /**
- * DataMap status
+ * This is the interface for create mv catalog.
  */
-public enum DataMapStatus {
-  ENABLED, DISABLED, DROPPED
+public interface MVCatalogFactory<T> {
+
+  /**
+   * Create a mv catalog object.
+   */
+  MVCatalog<T> newCatalog();
+
 }
diff --git a/core/src/main/java/org/apache/carbondata/core/view/MVManager.java b/core/src/main/java/org/apache/carbondata/core/view/MVManager.java
new file mode 100644
index 0000000..dfea9ed
--- /dev/null
+++ b/core/src/main/java/org/apache/carbondata/core/view/MVManager.java
@@ -0,0 +1,330 @@
+/*
+ * 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.carbondata.core.view;
+
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+
+import org.apache.carbondata.common.annotations.InterfaceAudience;
+import org.apache.carbondata.common.logging.LogServiceFactory;
+import org.apache.carbondata.core.constants.CarbonCommonConstants;
+import org.apache.carbondata.core.locks.ICarbonLock;
+import org.apache.carbondata.core.metadata.AbsoluteTableIdentifier;
+import org.apache.carbondata.core.metadata.schema.table.CarbonTable;
+import org.apache.carbondata.core.metadata.schema.table.RelationIdentifier;
+import org.apache.carbondata.core.statusmanager.LoadMetadataDetails;
+import org.apache.carbondata.core.statusmanager.SegmentStatus;
+import org.apache.carbondata.core.statusmanager.SegmentStatusManager;
+import org.apache.carbondata.core.util.path.CarbonTablePath;
+
+import com.google.gson.Gson;
+import org.apache.log4j.Logger;
+
+/**
+ * It maintains all the mv schemas in it.
+ */
+@InterfaceAudience.Internal
+public abstract class MVManager {
+
+  private static final Logger LOGGER =
+      LogServiceFactory.getLogService(MVManager.class.getName());
+
+  private final MVProvider schemaProvider =
+      MVProvider.get();
+
+  private volatile MVCatalog<?> catalog;
+
+  private final Object lock = new Object();
+
+  public MVManager() {
+
+  }
+
+  public abstract List<String> getDatabases();
+
+  public boolean hasSchemaOnTable(CarbonTable table) throws IOException {
+    List<MVSchema> schemas = getSchemas();
+    for (MVSchema schema : schemas) {
+      for (RelationIdentifier relatedTable : schema.getRelatedTables()) {
+        if (relatedTable.getDatabaseName().equalsIgnoreCase(table.getDatabaseName()) &&
+            relatedTable.getTableName().equalsIgnoreCase(table.getTableName())) {
+          return true;
+        }
+      }
+    }
+    return false;
+  }
+
+  /**
+   * It gives all mv schemas of a given table.
+   * For show mv command.
+   */
+  public List<MVSchema> getSchemasOnTable(CarbonTable table)
+      throws IOException {
+    List<MVSchema> schemasOnTable = new ArrayList<>();
+    List<MVSchema> schemas = getSchemas();
+    for (MVSchema schema : schemas) {
+      boolean isSchemaOnTable = false;
+      for (RelationIdentifier relatedTable : schema.getRelatedTables()) {
+        if (relatedTable.getDatabaseName().equalsIgnoreCase(table.getDatabaseName()) &&
+            relatedTable.getTableName().equalsIgnoreCase(table.getTableName())) {
+          isSchemaOnTable = true;
+          break;
+        }
+      }
+      if (isSchemaOnTable) {
+        schemasOnTable.add(schema);
+      }
+    }
+    return schemasOnTable;
+  }
+
+  /**
+   * It gives all mv schemas of a given table.
+   * For show mv command.
+   */
+  public List<MVSchema> getSchemasOnTable(String databaseName,
+                                          CarbonTable carbonTable) throws IOException {
+    return schemaProvider.getSchemas(this, databaseName, carbonTable);
+  }
+
+  /**
+   * It gives all mv schemas from store.
+   */
+  public List<MVSchema> getSchemas() throws IOException {
+    List<MVSchema> schemas = new ArrayList<>();
+    for (String database : this.getDatabases()) {
+      schemas.addAll(this.getSchemas(database));
+    }
+    return schemas;
+  }
+
+  /**
+   * It gives all mv schemas from store.
+   */
+  public List<MVSchema> getSchemas(String databaseName) throws IOException {
+    return schemaProvider.getSchemas(this, databaseName);
+  }
+
+  public MVSchema getSchema(String databaseName, String viewName) throws IOException {
+    return schemaProvider.getSchema(this, databaseName, viewName);
+  }
+
+  /**
+   * Saves the mv schema to storage
+   *
+   * @param viewSchema mv schema
+   */
+  public void createSchema(String databaseName, MVSchema viewSchema)
+      throws IOException {
+    schemaProvider.saveSchema(this, databaseName, viewSchema);
+  }
+
+  /**
+   * Drops the mv schema from storage
+   *
+   * @param viewName data map name
+   */
+  public void deleteSchema(String databaseName, String viewName) throws IOException {
+    schemaProvider.dropSchema(databaseName, viewName);
+  }
+
+  /**
+   * Get the mv catalog.
+   */
+  public MVCatalog<?> getCatalog() {
+    return catalog;
+  }
+
+  /**
+   * Get the mv catalog.
+   */
+  public MVCatalog<?> getCatalog(
+      MVCatalogFactory<?> catalogFactory,
+      boolean reload) throws IOException {
+    MVCatalog<?> catalog = this.catalog;
+    if (reload || catalog == null) {
+      synchronized (lock) {
+        catalog = this.catalog;
+        if (reload || catalog == null) {
+          catalog = catalogFactory.newCatalog();
+          List<MVSchema> schemas = getSchemas();
+          if (null == catalog) {
+            throw new RuntimeException("Internal Error.");
+          }
+          for (MVSchema schema : schemas) {
+            try {
+              catalog.registerSchema(schema);
+            } catch (Exception e) {
+              // Ignore the schema
+              LOGGER.error("Error while registering schema", e);
+            }
+          }
+          this.catalog = catalog;
+        }
+      }
+    }
+    return catalog;
+  }
+
+  /**
+   * In case of compaction on mv table,this method will merge the segment list of main table
+   * and return updated segment mapping
+   *
+   * @param mergedLoadName      to find which all segments are merged to new compacted segment
+   * @param viewSchema       of mv table
+   * @param viewLoadMetadataDetails of mv table
+   * @return updated segment map after merging segment list
+   */
+  @SuppressWarnings("unchecked")
+  public static String getUpdatedSegmentMap(String mergedLoadName,
+      MVSchema viewSchema,
+      LoadMetadataDetails[] viewLoadMetadataDetails) {
+    Map<String, List<String>> segmentMapping = new HashMap<>();
+    List<RelationIdentifier> relationIdentifiers = viewSchema.getRelatedTables();
+    for (RelationIdentifier relationIdentifier : relationIdentifiers) {
+      for (LoadMetadataDetails loadMetadataDetail : viewLoadMetadataDetails) {
+        if (loadMetadataDetail.getSegmentStatus() == SegmentStatus.COMPACTED) {
+          if (mergedLoadName.equalsIgnoreCase(loadMetadataDetail.getMergedLoadName())) {
+            Map segmentMap = new Gson().fromJson(loadMetadataDetail.getExtraInfo(), Map.class);
+            if (segmentMapping.isEmpty()) {
+              segmentMapping.putAll(segmentMap);
+            } else {
+              segmentMapping.get(relationIdentifier.getDatabaseName() + CarbonCommonConstants.POINT
+                  + relationIdentifier.getTableName()).addAll(
+                  (List<String>) segmentMap.get(
+                      relationIdentifier.getDatabaseName() + CarbonCommonConstants.POINT
+                          + relationIdentifier.getTableName()));
+            }
+          }
+        }
+      }
+    }
+    Gson gson = new Gson();
+    return gson.toJson(segmentMapping);
+  }
+
+  /**
+   * Get enabled mv status details
+   */
+  public List<MVStatusDetail> getEnabledStatusDetails() throws IOException {
+    List<MVStatusDetail> statusDetails = new ArrayList<>();
+    for (String database : this.getDatabases()) {
+      statusDetails.addAll(this.getEnabledStatusDetails(database));
+    }
+    return statusDetails;
+  }
+
+  /**
+   * Get enabled mv status details
+   */
+  List<MVStatusDetail> getEnabledStatusDetails(String databaseName)
+      throws IOException {
+    List<MVStatusDetail> statusDetails =
+        schemaProvider.getStatusDetails(databaseName);
+    List<MVStatusDetail> enabledStatusDetails = new ArrayList<>(statusDetails.size());
+    for (MVStatusDetail statusDetail : statusDetails) {
+      if (statusDetail.getStatus() == MVStatus.ENABLED) {
+        enabledStatusDetails.add(statusDetail);
+      }
+    }
+    return enabledStatusDetails;
+  }
+
+  public void setStatus(RelationIdentifier viewIdentifier, MVStatus viewStatus)
+      throws IOException {
+    MVSchema schema = getSchema(
+        viewIdentifier.getDatabaseName(), viewIdentifier.getTableName());
+    if (schema != null) {
+      schemaProvider.updateStatus(Collections.singletonList(schema), viewStatus);
+    }
+  }
+
+  public void setStatus(List<MVSchema> viewSchemas, MVStatus viewStatus)
+      throws IOException {
+    if (viewSchemas != null && !viewSchemas.isEmpty()) {
+      schemaProvider.updateStatus(viewSchemas, viewStatus);
+    }
+  }
+
+  public void onDrop(String databaseName, String viewName)
+      throws IOException {
+    MVSchema viewSchema = getSchema(databaseName, viewName);
+    if (viewSchema != null) {
+      schemaProvider.updateStatus(
+          Collections.singletonList(viewSchema), MVStatus.DROPPED);
+    }
+  }
+
+  /**
+   * This method will remove all segments of dataMap table in case of Insert-Overwrite/Update/Delete
+   * operations on main table
+   *
+   * @param schemas mv schemas
+   */
+  public void onTruncate(List<MVSchema> schemas)
+      throws IOException {
+    for (MVSchema schema : schemas) {
+      if (!schema.isRefreshOnManual()) {
+        setStatus(schema.identifier, MVStatus.DISABLED);
+      }
+      RelationIdentifier relationIdentifier = schema.getIdentifier();
+      SegmentStatusManager segmentStatusManager = new SegmentStatusManager(AbsoluteTableIdentifier
+          .from(relationIdentifier.getTablePath(),
+              relationIdentifier.getDatabaseName(),
+              relationIdentifier.getTableName()));
+      ICarbonLock carbonLock = segmentStatusManager.getTableStatusLock();
+      try {
+        if (carbonLock.lockWithRetries()) {
+          LOGGER.info("Acquired lock for table" + relationIdentifier.getDatabaseName() + "."
+              + relationIdentifier.getTableName() + " for table status updation");
+          String metaDataPath =
+              CarbonTablePath.getMetadataPath(relationIdentifier.getTablePath());
+          LoadMetadataDetails[] loadMetadataDetails =
+              SegmentStatusManager.readLoadMetadata(metaDataPath);
+          for (LoadMetadataDetails entry : loadMetadataDetails) {
+            entry.setSegmentStatus(SegmentStatus.MARKED_FOR_DELETE);
+          }
+          SegmentStatusManager.writeLoadDetailsIntoFile(
+              CarbonTablePath.getTableStatusFilePath(relationIdentifier.getTablePath()),
+              loadMetadataDetails);
+        } else {
+          LOGGER.error("Not able to acquire the lock for Table status updation for table "
+              + relationIdentifier.getDatabaseName() + "." + relationIdentifier
+              .getTableName());
+        }
+      } finally {
+        if (carbonLock.unlock()) {
+          LOGGER.info(
+              "Table unlocked successfully after table status updation" + relationIdentifier
+                  .getDatabaseName() + "." + relationIdentifier.getTableName());
+        } else {
+          LOGGER.error(
+              "Unable to unlock Table lock for table" + relationIdentifier.getDatabaseName()
+                  + "." + relationIdentifier.getTableName()
+                  + " during table status updation");
+        }
+      }
+    }
+  }
+
+}
diff --git a/core/src/main/java/org/apache/carbondata/core/metadata/schema/datamap/DataMapProperty.java b/core/src/main/java/org/apache/carbondata/core/view/MVProperty.java
similarity index 55%
copy from core/src/main/java/org/apache/carbondata/core/metadata/schema/datamap/DataMapProperty.java
copy to core/src/main/java/org/apache/carbondata/core/view/MVProperty.java
index f706b96..ff3d5c8 100644
--- a/core/src/main/java/org/apache/carbondata/core/metadata/schema/datamap/DataMapProperty.java
+++ b/core/src/main/java/org/apache/carbondata/core/view/MVProperty.java
@@ -15,7 +15,7 @@
  * limitations under the License.
  */
 
-package org.apache.carbondata.core.metadata.schema.datamap;
+package org.apache.carbondata.core.view;
 
 import org.apache.carbondata.common.annotations.InterfaceAudience;
 
@@ -23,16 +23,36 @@ import org.apache.carbondata.common.annotations.InterfaceAudience;
  * Property that can be specified when creating DataMap
  */
 @InterfaceAudience.Internal
-public class DataMapProperty {
+public class MVProperty {
 
   /**
-   * For datamap created with 'WITH DEFERRED REBUILD' syntax, we will add this
-   * property internally
+   * internal property, choices: FULL, INCREMENTAL
    */
-  public static final String DEFERRED_REBUILD = "_internal.deferred.rebuild";
+  public static final String REFRESH_MODE = "refresh_mode";
 
   /**
-   * internal property, true if it needs full refresh, thus it does not support incremental load
+   * MV refresh model: FULL
    */
-  public static final String FULL_REFRESH = "full_refresh";
+  public static final String REFRESH_MODE_FULL = "full";
+
+  /**
+   * MV refresh model: INCREMENTAL
+   */
+  public static final String REFRESH_MODE_INCREMENTAL = "incremental";
+
+  /**
+   * internal property, choices: ON_COMMIT, ON_MANUAL
+   */
+  public static final String REFRESH_TRIGGER_MODE = "refresh_trigger_mode";
+
+  /**
+   * MV refresh trigger model: ON_COMMIT
+   */
+  public static final String REFRESH_TRIGGER_MODE_ON_COMMIT = "on_commit";
+
+  /**
+   * MV refresh trigger model: ON_MANUAL
+   */
+  public static final String REFRESH_TRIGGER_MODE_ON_MANUAL = "on_manual";
+
 }
diff --git a/core/src/main/java/org/apache/carbondata/core/view/MVProvider.java b/core/src/main/java/org/apache/carbondata/core/view/MVProvider.java
new file mode 100644
index 0000000..df7f379
--- /dev/null
+++ b/core/src/main/java/org/apache/carbondata/core/view/MVProvider.java
@@ -0,0 +1,573 @@
+/*
+ * 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.carbondata.core.view;
+
+import java.io.BufferedReader;
+import java.io.BufferedWriter;
+import java.io.DataInputStream;
+import java.io.DataOutputStream;
+import java.io.IOException;
+import java.io.InputStreamReader;
+import java.io.OutputStreamWriter;
+import java.nio.charset.Charset;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.concurrent.ConcurrentHashMap;
+
+import org.apache.carbondata.common.annotations.InterfaceAudience;
+import org.apache.carbondata.common.logging.LogServiceFactory;
+import org.apache.carbondata.core.constants.CarbonCommonConstants;
+import org.apache.carbondata.core.datastore.filesystem.CarbonFile;
+import org.apache.carbondata.core.datastore.filesystem.CarbonFileFilter;
+import org.apache.carbondata.core.datastore.impl.FileFactory;
+import org.apache.carbondata.core.fileoperations.AtomicFileOperationFactory;
+import org.apache.carbondata.core.fileoperations.AtomicFileOperations;
+import org.apache.carbondata.core.fileoperations.FileWriteOperation;
+import org.apache.carbondata.core.locks.CarbonLockFactory;
+import org.apache.carbondata.core.locks.CarbonLockUtil;
+import org.apache.carbondata.core.locks.ICarbonLock;
+import org.apache.carbondata.core.locks.LockUsage;
+import org.apache.carbondata.core.metadata.schema.table.CarbonTable;
+import org.apache.carbondata.core.metadata.schema.table.RelationIdentifier;
+import org.apache.carbondata.core.statusmanager.LoadMetadataDetails;
+import org.apache.carbondata.core.statusmanager.SegmentStatus;
+import org.apache.carbondata.core.statusmanager.SegmentStatusManager;
+import org.apache.carbondata.core.util.CarbonProperties;
+import org.apache.carbondata.core.util.CarbonUtil;
+import org.apache.carbondata.core.util.path.CarbonTablePath;
+
+import com.google.gson.Gson;
+import org.apache.commons.lang3.StringUtils;
+import org.apache.hadoop.fs.permission.FsAction;
+import org.apache.hadoop.fs.permission.FsPermission;
+import org.apache.log4j.Logger;
+
+/**
+ * Stores mv schema in disk as json format
+ */
+@InterfaceAudience.Internal
+public class MVProvider {
+
+  private static final Logger LOG = LogServiceFactory.getLogService(
+      MVProvider.class.getCanonicalName());
+
+  private static final String STATUS_FILE_NAME = "mv_status";
+
+  private final String storeLocation;
+
+  private final Map<String, SchemaProvider> schemaProviders = new ConcurrentHashMap<>();
+
+  private MVProvider(String storeLocation) {
+    this.storeLocation = storeLocation;
+  }
+
+  public static MVProvider get() {
+    String storeLocation =
+        CarbonProperties.getInstance().getProperty(CarbonCommonConstants.STORE_LOCATION);
+    if (storeLocation == null) {
+      throw new RuntimeException(
+          "Property [" + CarbonCommonConstants.STORE_LOCATION + "] is not set.");
+    }
+    return new MVProvider(storeLocation);
+  }
+
+  private static String getSchemaPath(String schemaRoot, String viewName) {
+    return schemaRoot + CarbonCommonConstants.FILE_SEPARATOR + "mv_schema." + viewName;
+  }
+
+  private SchemaProvider getSchemaProvider(String databaseName) {
+    String databaseNameUpper = databaseName.toUpperCase();
+    SchemaProvider schemaProvider = this.schemaProviders.get(databaseNameUpper);
+    if (schemaProvider == null) {
+      synchronized (this.schemaProviders) {
+        schemaProvider = this.schemaProviders.get(databaseNameUpper);
+        if (schemaProvider == null) {
+          String databaseLocation;
+          if (databaseNameUpper.equalsIgnoreCase(CarbonCommonConstants.DATABASE_DEFAULT_NAME)) {
+            databaseLocation = CarbonUtil.checkAndAppendHDFSUrl(this.storeLocation);
+          } else {
+            databaseLocation = CarbonUtil.checkAndAppendHDFSUrl(this.storeLocation +
+                CarbonCommonConstants.FILE_SEPARATOR + databaseName + ".db");
+          }
+          if (!FileFactory.getCarbonFile(databaseLocation).exists()) {
+            return null;
+          }
+          schemaProvider = new SchemaProvider(databaseLocation);
+          this.schemaProviders.put(databaseNameUpper, schemaProvider);
+        }
+      }
+    }
+    return schemaProvider;
+  }
+
+  public MVSchema getSchema(MVManager viewManager,
+                            String databaseName, String viewName) throws IOException {
+    SchemaProvider schemaProvider = this.getSchemaProvider(databaseName);
+    if (schemaProvider == null) {
+      return null;
+    }
+    return schemaProvider.retrieveSchema(viewManager, viewName);
+  }
+
+  List<MVSchema> getSchemas(MVManager viewManager,
+                            String databaseName, CarbonTable carbonTable) throws IOException {
+    SchemaProvider schemaProvider = this.getSchemaProvider(databaseName);
+    if (schemaProvider == null) {
+      return Collections.emptyList();
+    } else {
+      return schemaProvider.retrieveSchemas(viewManager, carbonTable);
+    }
+  }
+
+  List<MVSchema> getSchemas(MVManager viewManager,
+                            String databaseName) throws IOException {
+    SchemaProvider schemaProvider = this.getSchemaProvider(databaseName);
+    if (schemaProvider == null) {
+      return Collections.emptyList();
+    } else {
+      return schemaProvider.retrieveAllSchemas(viewManager);
+    }
+  }
+
+  void saveSchema(MVManager viewManager, String databaseName,
+                  MVSchema viewSchema) throws IOException {
+    SchemaProvider schemaProvider = this.getSchemaProvider(databaseName);
+    if (schemaProvider == null) {
+      throw new IOException("Database [" + databaseName + "] is not found.");
+    }
+    schemaProvider.saveSchema(viewManager, viewSchema);
+  }
+
+  public void dropSchema(String databaseName, String viewName)
+      throws IOException {
+    SchemaProvider schemaProvider = this.getSchemaProvider(databaseName);
+    if (schemaProvider == null) {
+      throw new IOException("Materialized view with name " + databaseName + "." + viewName +
+          " does not exists in storage");
+    }
+    schemaProvider.dropSchema(viewName);
+  }
+
+  private String getStatusFileName(String databaseName) {
+    if (databaseName.equalsIgnoreCase("default")) {
+      return this.storeLocation +
+          CarbonCommonConstants.FILE_SEPARATOR + "_system" +
+          CarbonCommonConstants.FILE_SEPARATOR + STATUS_FILE_NAME;
+    } else {
+      return this.storeLocation +
+          CarbonCommonConstants.FILE_SEPARATOR + databaseName + ".db" +
+          CarbonCommonConstants.FILE_SEPARATOR + "_system" +
+          CarbonCommonConstants.FILE_SEPARATOR + STATUS_FILE_NAME;
+    }
+  }
+
+  public List<MVStatusDetail> getStatusDetails(String databaseName)
+      throws IOException {
+    String statusPath = this.getStatusFileName(databaseName);
+    Gson gsonObjectToRead = new Gson();
+    DataInputStream dataInputStream = null;
+    BufferedReader buffReader = null;
+    InputStreamReader inStream = null;
+    MVStatusDetail[] statusDetails;
+    try {
+      if (!FileFactory.isFileExist(statusPath)) {
+        return Collections.emptyList();
+      }
+      dataInputStream = FileFactory.getDataInputStream(statusPath);
+      inStream = new InputStreamReader(dataInputStream,
+          Charset.forName(CarbonCommonConstants.DEFAULT_CHARSET));
+      buffReader = new BufferedReader(inStream);
+      statusDetails = gsonObjectToRead.fromJson(buffReader,
+          MVStatusDetail[].class);
+    } catch (IOException e) {
+      LOG.error("Failed to read datamap status", e);
+      throw e;
+    } finally {
+      CarbonUtil.closeStreams(buffReader, inStream, dataInputStream);
+    }
+    // if status details is null, return empty array
+    if (null == statusDetails) {
+      return Collections.emptyList();
+    }
+    return Arrays.asList(statusDetails);
+  }
+
+  private static ICarbonLock getStatusLock(String databaseName) {
+    return CarbonLockFactory
+        .getSystemLevelCarbonLockObj(
+            CarbonProperties.getInstance().getSystemFolderLocation(databaseName),
+            LockUsage.MATERIALIZED_VIEW_STATUS_LOCK);
+  }
+
+  /**
+   * Update or add the status of passed mvs with the given mv status. If the mv status
+   * given is enabled/disabled then updates/adds the mv, in case of drop it just removes it
+   * from the file.
+   * This method always overwrites the old file.
+   * @param schemaList schemas of which are need to be updated in mv status
+   * @param status  status to be updated for the mv schemas
+   */
+  public void updateStatus(List<MVSchema> schemaList,
+      MVStatus status) throws IOException {
+    if (schemaList == null || schemaList.size() == 0) {
+      // There is nothing to update
+      return;
+    }
+    final Map<String, List<MVSchema>> schemasMapByDatabase = new HashMap<>();
+    for (MVSchema schema : schemaList) {
+      String databaseName = schema.getIdentifier().getDatabaseName().toLowerCase();
+      List<MVSchema> schemas = schemasMapByDatabase.get(databaseName);
+      if (schemas == null) {
+        schemas = new ArrayList<>();
+        schemasMapByDatabase.put(databaseName, schemas);
+      }
+      schemas.add(schema);
+    }
+    for (Map.Entry<String, List<MVSchema>> entry : schemasMapByDatabase.entrySet()) {
+      this.updateStatus(entry.getKey(), entry.getValue(), status);
+    }
+  }
+
+  private void updateStatus(String databaseName, List<MVSchema> schemaList,
+      MVStatus status) throws IOException {
+    ICarbonLock carbonTableStatusLock = getStatusLock(databaseName);
+    boolean locked = false;
+    try {
+      locked = carbonTableStatusLock.lockWithRetries();
+      if (locked) {
+        LOG.info("Materialized view status lock has been successfully acquired.");
+        if (status == MVStatus.ENABLED) {
+          // Enable mv only if mv tables and main table are in sync
+          if (!isViewCanBeEnabled(schemaList.get(0))) {
+            return;
+          }
+        }
+        List<MVStatusDetail> statusDetailList =
+            new ArrayList<>(getStatusDetails(databaseName));
+        List<MVStatusDetail> changedStatusDetails = new ArrayList<>();
+        List<MVStatusDetail> newStatusDetails = new ArrayList<>();
+        for (MVSchema schema : schemaList) {
+          boolean exists = false;
+          for (MVStatusDetail statusDetail : statusDetailList) {
+            if (statusDetail.getIdentifier().equals(schema.getIdentifier())) {
+              statusDetail.setStatus(status);
+              changedStatusDetails.add(statusDetail);
+              exists = true;
+            }
+          }
+          if (!exists) {
+            newStatusDetails
+                .add(new MVStatusDetail(schema.getIdentifier(),
+                    status));
+          }
+        }
+        // Add the newly added datamaps to the list.
+        if (newStatusDetails.size() > 0 &&
+            status != MVStatus.DROPPED) {
+          statusDetailList.addAll(newStatusDetails);
+        }
+        // In case of dropped datamap, just remove from the list.
+        if (status == MVStatus.DROPPED) {
+          statusDetailList.removeAll(changedStatusDetails);
+        }
+        writeLoadDetailsIntoFile(
+            this.getStatusFileName(databaseName),
+            statusDetailList.toArray(
+                new MVStatusDetail[statusDetailList.size()]));
+      } else {
+        String errorMsg = "Upadating datamapstatus is failed due to another process taken the lock"
+            + " for updating it";
+        LOG.error(errorMsg);
+        throw new IOException(errorMsg + " Please try after some time.");
+      }
+    } finally {
+      if (locked) {
+        CarbonLockUtil.fileUnlock(carbonTableStatusLock, LockUsage.DATAMAP_STATUS_LOCK);
+      }
+    }
+  }
+
+  /**
+   * writes mv status details
+   */
+  private static void writeLoadDetailsIntoFile(String location,
+      MVStatusDetail[] statusDetails) throws IOException {
+    FileFactory.touchFile(FileFactory.getCarbonFile(location),
+        new FsPermission(FsAction.ALL, FsAction.ALL, FsAction.ALL));
+    AtomicFileOperations fileWrite = AtomicFileOperationFactory.getAtomicFileOperations(location);
+    BufferedWriter brWriter = null;
+    DataOutputStream dataOutputStream = null;
+    Gson gsonObjectToWrite = new Gson();
+    // write the updated data into the mv status file.
+    try {
+      dataOutputStream = fileWrite.openForWrite(FileWriteOperation.OVERWRITE);
+      brWriter = new BufferedWriter(new OutputStreamWriter(dataOutputStream,
+          Charset.forName(CarbonCommonConstants.DEFAULT_CHARSET)));
+
+      String metadataInstance = gsonObjectToWrite.toJson(statusDetails);
+      brWriter.write(metadataInstance);
+    } catch (IOException ioe) {
+      LOG.error("Error message: " + ioe.getLocalizedMessage());
+      fileWrite.setFailed();
+      throw ioe;
+    } finally {
+      if (null != brWriter) {
+        brWriter.flush();
+      }
+      CarbonUtil.closeStreams(brWriter);
+      fileWrite.close();
+    }
+
+  }
+
+  /**
+   * This method checks if main table and mv table are synchronised or not. If synchronised
+   * return true to enable the mv
+   *
+   * @param schema of mv to be disabled or enabled
+   * @return flag to enable or disable mv
+   * @throws IOException
+   */
+  private static boolean isViewCanBeEnabled(MVSchema schema)
+      throws IOException {
+    if (!schema.isRefreshIncremental()) {
+      return true;
+    }
+    boolean isViewCanBeEnabled = true;
+    String viewMetadataPath =
+        CarbonTablePath.getMetadataPath(schema.getIdentifier().getTablePath());
+    LoadMetadataDetails[] viewLoadMetadataDetails =
+        SegmentStatusManager.readLoadMetadata(viewMetadataPath);
+    Map<String, List<String>> viewSegmentMap = new HashMap<>();
+    for (LoadMetadataDetails loadMetadataDetail : viewLoadMetadataDetails) {
+      if (loadMetadataDetail.getSegmentStatus() == SegmentStatus.SUCCESS) {
+        Map<String, List<String>> segmentMap =
+            new Gson().fromJson(loadMetadataDetail.getExtraInfo(), Map.class);
+        if (viewSegmentMap.isEmpty()) {
+          viewSegmentMap.putAll(segmentMap);
+        } else {
+          for (Map.Entry<String, List<String>> entry : segmentMap.entrySet()) {
+            if (null != viewSegmentMap.get(entry.getKey())) {
+              viewSegmentMap.get(entry.getKey()).addAll(entry.getValue());
+            }
+          }
+        }
+      }
+    }
+    List<RelationIdentifier> relatedTables = schema.getRelatedTables();
+    for (RelationIdentifier relatedTable : relatedTables) {
+      List<String> relatedTableSegmentList =
+          SegmentStatusManager.getValidSegmentList(relatedTable);
+      if (!relatedTableSegmentList.isEmpty()) {
+        if (viewSegmentMap.isEmpty()) {
+          isViewCanBeEnabled = false;
+        } else {
+          isViewCanBeEnabled = viewSegmentMap.get(
+              relatedTable.getDatabaseName() + CarbonCommonConstants.POINT +
+                  relatedTable.getTableName()).containsAll(relatedTableSegmentList);
+        }
+      }
+    }
+    return isViewCanBeEnabled;
+  }
+
+  /**
+   * Data map schema provider of a database.
+   */
+  private static final class SchemaProvider {
+
+    private String systemDirectory;
+
+    private String schemaIndexFilePath;
+
+    private long lastModifiedTime;
+
+    private Set<MVSchema> schemas = new HashSet<>();
+
+    SchemaProvider(String databaseLocation) {
+      final String systemDirectory =
+          databaseLocation + CarbonCommonConstants.FILE_SEPARATOR + "_system";
+      this.systemDirectory = systemDirectory;
+      this.schemaIndexFilePath = systemDirectory + CarbonCommonConstants.FILE_SEPARATOR +
+          "mv_schema_index";
+    }
+
+    void saveSchema(MVManager viewManager, MVSchema viewSchema)
+        throws IOException {
+      BufferedWriter brWriter = null;
+      DataOutputStream dataOutputStream = null;
+      Gson gsonObjectToWrite = new Gson();
+      String schemaPath =
+          getSchemaPath(this.systemDirectory, viewSchema.getIdentifier().getTableName());
+      if (FileFactory.isFileExist(schemaPath)) {
+        throw new IOException(
+            "Materialized view with name " + viewSchema.getIdentifier().getTableName() +
+                " already exists in storage");
+      }
+      // write the mv schema in json format.
+      try {
+        FileFactory.mkdirs(this.systemDirectory);
+        FileFactory.createNewFile(schemaPath);
+        dataOutputStream =
+            FileFactory.getDataOutputStream(schemaPath);
+        brWriter = new BufferedWriter(new OutputStreamWriter(dataOutputStream,
+            Charset.forName(CarbonCommonConstants.DEFAULT_CHARSET)));
+
+        String metadataInstance = gsonObjectToWrite.toJson(viewSchema);
+        brWriter.write(metadataInstance);
+      } finally {
+        if (null != brWriter) {
+          brWriter.flush();
+        }
+        this.schemas.add(viewSchema);
+        CarbonUtil.closeStreams(dataOutputStream, brWriter);
+        checkAndReloadSchemas(viewManager, true);
+        touchMDTFile();
+      }
+    }
+
+    MVSchema retrieveSchema(MVManager viewManager, String viewName)
+        throws IOException {
+      checkAndReloadSchemas(viewManager, true);
+      for (MVSchema schema : this.schemas) {
+        if (schema.getIdentifier().getTableName().equalsIgnoreCase(viewName)) {
+          return schema;
+        }
+      }
+      return null;
+    }
+
+    List<MVSchema> retrieveSchemas(MVManager viewManager,
+                                   CarbonTable carbonTable) throws IOException {
+      checkAndReloadSchemas(viewManager, false);
+      List<MVSchema> schemas = new ArrayList<>();
+      for (MVSchema schema : this.schemas) {
+        List<RelationIdentifier> parentTables = schema.getRelatedTables();
+        for (RelationIdentifier identifier : parentTables) {
+          if (StringUtils.isNotEmpty(identifier.getTableId())) {
+            if (identifier.getTableId().equalsIgnoreCase(carbonTable.getTableId())) {
+              schemas.add(schema);
+              break;
+            }
+          } else if (identifier.getTableName().equalsIgnoreCase(carbonTable.getTableName()) &&
+              identifier.getDatabaseName().equalsIgnoreCase(carbonTable.getDatabaseName())) {
+            schemas.add(schema);
+            break;
+          }
+        }
+      }
+      return schemas;
+    }
+
+    List<MVSchema> retrieveAllSchemas(MVManager viewManager)
+        throws IOException {
+      checkAndReloadSchemas(viewManager, true);
+      return new ArrayList<>(this.schemas);
+    }
+
+    @SuppressWarnings("Convert2Lambda")
+    private Set<MVSchema> retrieveAllSchemasInternal(
+        MVManager viewManager) throws IOException {
+      Set<MVSchema> schemas = new HashSet<>();
+      CarbonFile carbonFile = FileFactory.getCarbonFile(this.systemDirectory);
+      CarbonFile[] carbonFiles = carbonFile.listFiles(new CarbonFileFilter() {
+        @Override
+        public boolean accept(CarbonFile file) {
+          return file.getName().startsWith("mv_schema.");
+        }
+      });
+      Gson gson = new Gson();
+      for (CarbonFile file :carbonFiles) {
+        DataInputStream dataInputStream = null;
+        BufferedReader buffReader = null;
+        InputStreamReader inStream = null;
+        try {
+          String absolutePath = file.getAbsolutePath();
+          dataInputStream =
+              FileFactory.getDataInputStream(
+                  absolutePath);
+          inStream = new InputStreamReader(dataInputStream,
+              Charset.forName(CarbonCommonConstants.DEFAULT_CHARSET));
+          buffReader = new BufferedReader(inStream);
+          MVSchema schema = gson.fromJson(buffReader, MVSchema.class);
+          schema.setManager(viewManager);
+          schemas.add(schema);
+        } finally {
+          CarbonUtil.closeStreams(buffReader, inStream, dataInputStream);
+        }
+      }
+      return schemas;
+    }
+
+    void dropSchema(String viewName) throws IOException {
+      String schemaPath = getSchemaPath(this.systemDirectory, viewName);
+      if (!FileFactory.isFileExist(schemaPath)) {
+        throw new IOException("Materialized with name " + viewName + " does not exists in storage");
+      }
+      LOG.info(String.format("Trying to delete materialized view %s schema", viewName));
+      this.schemas.removeIf(
+          schema -> schema.getIdentifier().getTableName().equalsIgnoreCase(viewName)
+      );
+      touchMDTFile();
+      if (!FileFactory.deleteFile(schemaPath)) {
+        throw new IOException("Materialized view with name " + viewName + " cannot be deleted");
+      }
+      LOG.info(String.format("Materialized view %s schema is deleted", viewName));
+    }
+
+    private void checkAndReloadSchemas(MVManager viewManager, boolean touchFile)
+        throws IOException {
+      if (FileFactory.isFileExist(this.schemaIndexFilePath)) {
+        long lastModifiedTime =
+            FileFactory.getCarbonFile(this.schemaIndexFilePath).getLastModifiedTime();
+        if (this.lastModifiedTime != lastModifiedTime) {
+          this.schemas = this.retrieveAllSchemasInternal(viewManager);
+          touchMDTFile();
+        }
+      } else {
+        this.schemas = this.retrieveAllSchemasInternal(viewManager);
+        if (touchFile) {
+          touchMDTFile();
+        }
+      }
+    }
+
+    private void touchMDTFile() throws IOException {
+      if (!FileFactory.isFileExist(this.systemDirectory)) {
+        FileFactory.createDirectoryAndSetPermission(this.systemDirectory,
+            new FsPermission(FsAction.ALL, FsAction.ALL, FsAction.ALL));
+      }
+      if (!FileFactory.isFileExist(this.schemaIndexFilePath)) {
+        FileFactory.createNewFile(
+            this.schemaIndexFilePath,
+            new FsPermission(FsAction.ALL, FsAction.ALL, FsAction.ALL));
+      }
+      long lastModifiedTime = System.currentTimeMillis();
+      FileFactory.getCarbonFile(this.schemaIndexFilePath).setLastModifiedTime(lastModifiedTime);
+      this.lastModifiedTime = lastModifiedTime;
+    }
+
+  }
+
+}
diff --git a/core/src/main/java/org/apache/carbondata/core/view/MVSchema.java b/core/src/main/java/org/apache/carbondata/core/view/MVSchema.java
new file mode 100644
index 0000000..738db62
--- /dev/null
+++ b/core/src/main/java/org/apache/carbondata/core/view/MVSchema.java
@@ -0,0 +1,285 @@
+/*
+ * 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.carbondata.core.view;
+
+import java.io.DataInput;
+import java.io.DataOutput;
+import java.io.IOException;
+import java.io.Serializable;
+import java.util.Arrays;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.Objects;
+import java.util.Set;
+
+import org.apache.carbondata.common.Strings;
+import org.apache.carbondata.core.constants.CarbonCommonConstants;
+import org.apache.carbondata.core.metadata.schema.table.RelationIdentifier;
+import org.apache.carbondata.core.metadata.schema.table.Writable;
+import org.apache.carbondata.core.statusmanager.LoadMetadataDetails;
+import org.apache.carbondata.core.statusmanager.SegmentStatus;
+import org.apache.carbondata.core.statusmanager.SegmentStatusManager;
+import org.apache.carbondata.core.util.path.CarbonTablePath;
+
+import com.google.gson.Gson;
+
+/**
+ * It is the new schama of mv and it has less fields compare to {{@link MVSchema}}
+ */
+public class MVSchema implements Serializable, Writable {
+
+  private static final long serialVersionUID = 1030007239497486385L;
+
+  /**
+   * For MV, this is the identifier of the MV table.
+   */
+  protected RelationIdentifier identifier;
+
+  /**
+   * SQL query string used to create MV
+   */
+  private String query;
+
+  /**
+   * Properties provided by user
+   */
+  protected Map<String, String> properties;
+
+  /**
+   * Identifiers of parent tables of the MV
+   */
+  private List<RelationIdentifier> relatedTables;
+
+  /**
+   * main table column list mapped to mv table
+   */
+  private Map<String, Set<String>> relatedTableColumnList;
+
+  /**
+   * DataMap table column order map as per Select query
+   */
+  private Map<Integer, String> columnsOrderMap;
+
+  /**
+   * timeseries query
+   */
+  private boolean timeSeries;
+
+  private transient volatile MVManager manager;
+
+  public MVSchema(MVManager manager) {
+    this.manager = manager;
+  }
+
+  public RelationIdentifier getIdentifier() {
+    return identifier;
+  }
+
+  public void setIdentifier(RelationIdentifier identifier) {
+    this.identifier = identifier;
+  }
+
+  public String getQuery() {
+    return query;
+  }
+
+  public void setQuery(String query) {
+    this.query = query;
+  }
+
+  public Map<String, String> getProperties() {
+    return properties;
+  }
+
+  public void setProperties(Map<String, String> properties) {
+    this.properties = properties;
+  }
+
+  public void setRelatedTables(List<RelationIdentifier> relatedTables) {
+    this.relatedTables = relatedTables;
+  }
+
+  public List<RelationIdentifier> getRelatedTables() {
+    return relatedTables;
+  }
+
+  /**
+   * Return true if this datamap is lazy (created with DEFERRED REBUILD syntax)
+   */
+  public boolean isRefreshOnManual() {
+    String refreshTriggerMode = getProperties().get(MVProperty.REFRESH_TRIGGER_MODE);
+    return refreshTriggerMode != null &&
+        refreshTriggerMode.equalsIgnoreCase(
+            MVProperty.REFRESH_TRIGGER_MODE_ON_MANUAL);
+  }
+
+  /**
+   * Return true if this DataMap can support incremental build
+   */
+  public boolean isRefreshIncremental() {
+    String refreshMode = getProperties().get(MVProperty.REFRESH_MODE);
+    return refreshMode != null && refreshMode.equalsIgnoreCase(
+        MVProperty.REFRESH_MODE_INCREMENTAL);
+  }
+
+  @Override
+  public void write(DataOutput out) throws IOException {
+    boolean isRelationIdentifierExists = null != identifier;
+    out.writeBoolean(isRelationIdentifierExists);
+    if (isRelationIdentifierExists) {
+      this.identifier.write(out);
+    }
+    if (properties == null) {
+      out.writeShort(0);
+    } else {
+      out.writeShort(properties.size());
+      for (Map.Entry<String, String> entry : properties.entrySet()) {
+        out.writeUTF(entry.getKey());
+        out.writeUTF(entry.getValue());
+      }
+    }
+  }
+
+  @Override
+  public void readFields(DataInput in) throws IOException {
+    boolean isRelationIdentifierExists = in.readBoolean();
+    if (isRelationIdentifierExists) {
+      this.identifier = new RelationIdentifier(null, null, null);
+      this.identifier.readFields(in);
+    }
+
+    int mapSize = in.readShort();
+    this.properties = new HashMap<>(mapSize);
+    for (int i = 0; i < mapSize; i++) {
+      String key = in.readUTF();
+      String value = in.readUTF();
+      this.properties.put(key, value);
+    }
+  }
+
+  @Override
+  public boolean equals(Object o) {
+    if (this == o) return true;
+    if (o == null || getClass() != o.getClass()) return false;
+    MVSchema that = (MVSchema) o;
+    return Objects.equals(this.identifier, that.identifier);
+  }
+
+  @Override
+  public int hashCode() {
+    return Objects.hash(this.identifier);
+  }
+
+  public Map<String, Set<String>> getRelatedTableColumns() {
+    return relatedTableColumnList;
+  }
+
+  public void setRelatedTableColumnList(Map<String, Set<String>> relatedTableColumnList) {
+    this.relatedTableColumnList = relatedTableColumnList;
+  }
+
+  public Map<Integer, String> getColumnsOrderMap() {
+    return columnsOrderMap;
+  }
+
+  public void setColumnsOrderMap(Map<Integer, String> columnsOrderMap) {
+    this.columnsOrderMap = columnsOrderMap;
+  }
+
+  public boolean isTimeSeries() {
+    return timeSeries;
+  }
+
+  public void setTimeSeries(boolean timeSeries) {
+    this.timeSeries = timeSeries;
+  }
+
+  public String getPropertiesAsString() {
+    String[] properties = getProperties().entrySet().stream()
+        .map(p -> "'" + p.getKey() + "'='" + p.getValue() + "'")
+        .sorted()
+        .toArray(String[]::new);
+    return Strings.mkString(properties, ",");
+  }
+
+  public MVStatus getStatus() throws IOException {
+    List<MVStatusDetail> details = this.manager.getEnabledStatusDetails(
+          this.identifier.getDatabaseName());
+    for (MVStatusDetail detail : details) {
+      if (detail.getIdentifier().equals(this.getIdentifier())) {
+        return MVStatus.ENABLED;
+      }
+    }
+    return MVStatus.DISABLED;
+  }
+
+  public String getSyncStatus() {
+    LoadMetadataDetails[] loads =
+        SegmentStatusManager.readLoadMetadata(
+            CarbonTablePath.getMetadataPath(this.getIdentifier().getTablePath()));
+    if (loads.length > 0) {
+      for (int i = loads.length - 1; i >= 0; i--) {
+        LoadMetadataDetails load = loads[i];
+        if (load.getSegmentStatus().equals(SegmentStatus.SUCCESS)) {
+          Map<String, List<String>> segmentMaps =
+              new Gson().fromJson(load.getExtraInfo(), Map.class);
+          Map<String, String> syncInfoMap = new HashMap<>();
+          for (Map.Entry<String, List<String>> entry : segmentMaps.entrySet()) {
+            // when in join scenario, one table is loaded and one more is not loaded,
+            // then put value as NA
+            if (entry.getValue().isEmpty()) {
+              syncInfoMap.put(entry.getKey(), "NA");
+            } else {
+              syncInfoMap.put(entry.getKey(), getMaxSegmentID(entry.getValue()));
+            }
+          }
+          String loadEndTime;
+          if (load.getLoadEndTime() == CarbonCommonConstants.SEGMENT_LOAD_TIME_DEFAULT) {
+            loadEndTime = "NA";
+          } else {
+            loadEndTime = new java.sql.Timestamp(load.getLoadEndTime()).toString();
+          }
+          syncInfoMap.put(CarbonCommonConstants.LOAD_SYNC_TIME, loadEndTime);
+          return new Gson().toJson(syncInfoMap);
+        }
+      }
+    }
+    return "NA";
+  }
+
+  private static String getMaxSegmentID(List<String> segmentList) {
+    double[] segment = new double[segmentList.size()];
+    int i = 0;
+    for (String id : segmentList) {
+      segment[i] = Double.parseDouble(id);
+      i++;
+    }
+    Arrays.sort(segment);
+    String maxId = Double.toString(segment[segmentList.size() - 1]);
+    if (maxId.endsWith(".0")) {
+      maxId = maxId.substring(0, maxId.indexOf("."));
+    }
+    return maxId;
+  }
+
+  public void setManager(MVManager manager) {
+    this.manager = manager;
+  }
+
+}
diff --git a/core/src/main/java/org/apache/carbondata/core/datamap/status/DataMapStatus.java b/core/src/main/java/org/apache/carbondata/core/view/MVStatus.java
similarity index 89%
copy from core/src/main/java/org/apache/carbondata/core/datamap/status/DataMapStatus.java
copy to core/src/main/java/org/apache/carbondata/core/view/MVStatus.java
index fce16c8..f3e8cc5 100644
--- a/core/src/main/java/org/apache/carbondata/core/datamap/status/DataMapStatus.java
+++ b/core/src/main/java/org/apache/carbondata/core/view/MVStatus.java
@@ -15,11 +15,11 @@
  * limitations under the License.
  */
 
-package org.apache.carbondata.core.datamap.status;
+package org.apache.carbondata.core.view;
 
 /**
- * DataMap status
+ * MV status
  */
-public enum DataMapStatus {
+public enum MVStatus {
   ENABLED, DISABLED, DROPPED
 }
diff --git a/index/lucene/src/main/java/org/apache/carbondata/datamap/lucene/LuceneDataMapDistributable.java b/core/src/main/java/org/apache/carbondata/core/view/MVStatusDetail.java
similarity index 52%
copy from index/lucene/src/main/java/org/apache/carbondata/datamap/lucene/LuceneDataMapDistributable.java
copy to core/src/main/java/org/apache/carbondata/core/view/MVStatusDetail.java
index 1d47ee8..4e97b5c 100644
--- a/index/lucene/src/main/java/org/apache/carbondata/datamap/lucene/LuceneDataMapDistributable.java
+++ b/core/src/main/java/org/apache/carbondata/core/view/MVStatusDetail.java
@@ -15,33 +15,47 @@
  * limitations under the License.
  */
 
-package org.apache.carbondata.datamap.lucene;
+package org.apache.carbondata.core.view;
+
+import java.io.Serializable;
 
 import org.apache.carbondata.common.annotations.InterfaceAudience;
-import org.apache.carbondata.core.datamap.DataMapDistributable;
+import org.apache.carbondata.core.metadata.schema.table.RelationIdentifier;
 
+/**
+ * Status of mv
+ */
 @InterfaceAudience.Internal
-class LuceneDataMapDistributable extends DataMapDistributable {
+public class MVStatusDetail implements Serializable {
+
+  private static final long serialVersionUID = 1570997199499681821L;
+
+  private RelationIdentifier identifier;
 
-  // TODO: seems no one use this?
-  private String dataPath;
+  private MVStatus status;
 
-  private String indexPath;
+  MVStatusDetail(RelationIdentifier identifier, MVStatus status) {
+    this.identifier = identifier;
+    this.status = status;
+  }
+
+  public RelationIdentifier getIdentifier() {
+    return identifier;
+  }
 
-  LuceneDataMapDistributable(String dataPath, String indexPath) {
-    this.dataPath = dataPath;
-    this.indexPath = indexPath;
+  public void setIdentifier(RelationIdentifier name) {
+    this.identifier = name;
   }
 
-  public String getDataPath() {
-    return dataPath;
+  public MVStatus getStatus() {
+    return status;
   }
 
-  public String getIndexPath() {
-    return indexPath;
+  public boolean isEnabled() {
+    return status == MVStatus.ENABLED;
   }
 
-  public void setIndexPath(String indexPath) {
-    this.indexPath = indexPath;
+  public void setStatus(MVStatus status) {
+    this.status = status;
   }
 }
diff --git a/core/src/main/java/org/apache/carbondata/hadoop/CarbonInputSplit.java b/core/src/main/java/org/apache/carbondata/hadoop/CarbonInputSplit.java
index 9584257..80eb3a9 100644
--- a/core/src/main/java/org/apache/carbondata/hadoop/CarbonInputSplit.java
+++ b/core/src/main/java/org/apache/carbondata/hadoop/CarbonInputSplit.java
@@ -36,14 +36,14 @@ import org.apache.carbondata.core.datamap.Segment;
 import org.apache.carbondata.core.datastore.block.Distributable;
 import org.apache.carbondata.core.datastore.block.TableBlockInfo;
 import org.apache.carbondata.core.indexstore.BlockletDetailInfo;
-import org.apache.carbondata.core.indexstore.blockletindex.BlockletDataMapRowIndexes;
-import org.apache.carbondata.core.indexstore.row.DataMapRow;
+import org.apache.carbondata.core.indexstore.blockletindex.BlockletIndexRowIndexes;
+import org.apache.carbondata.core.indexstore.row.IndexRow;
 import org.apache.carbondata.core.metadata.ColumnarFormatVersion;
 import org.apache.carbondata.core.metadata.schema.table.column.ColumnSchema;
 import org.apache.carbondata.core.statusmanager.FileFormat;
 import org.apache.carbondata.core.stream.ExtendedByteArrayInputStream;
 import org.apache.carbondata.core.stream.ExtendedDataInputStream;
-import org.apache.carbondata.core.util.BlockletDataMapUtil;
+import org.apache.carbondata.core.util.BlockletIndexUtil;
 import org.apache.carbondata.core.util.ByteUtil;
 import org.apache.carbondata.core.util.CarbonProperties;
 import org.apache.carbondata.core.util.CarbonUtil;
@@ -93,7 +93,7 @@ public class CarbonInputSplit extends FileSplit
    */
   private Set<Integer> validBlockletIds;
 
-  private transient DataMapRow dataMapRow;
+  private transient IndexRow indexRow;
 
   private transient List<ColumnSchema> columnSchema;
 
@@ -404,8 +404,8 @@ public class CarbonInputSplit extends FileSplit
     out.writeLong(length);
     out.writeShort(version.number());
     //TODO remove this code once count(*) optmization is added in case of index server
-    if (null != dataMapRow) {
-      out.writeInt(this.dataMapRow.getInt(BlockletDataMapRowIndexes.ROW_COUNT_INDEX));
+    if (null != indexRow) {
+      out.writeInt(this.indexRow.getInt(BlockletIndexRowIndexes.ROW_COUNT_INDEX));
     } else if (null != detailInfo) {
       out.writeInt(this.detailInfo.getRowCount());
     } else {
@@ -418,11 +418,11 @@ public class CarbonInputSplit extends FileSplit
     out.writeUTF(blockletId);
     out.writeUTF(segment.toString());
     // please refer writeDetailInfo doc
-    out.writeBoolean(writeDetailInfo && (detailInfo != null || dataMapRow != null));
+    out.writeBoolean(writeDetailInfo && (detailInfo != null || indexRow != null));
     if (writeDetailInfo && detailInfo != null) {
       detailInfo.write(out);
       // please refer writeDetailInfo doc
-    } else if (writeDetailInfo && dataMapRow != null) {
+    } else if (writeDetailInfo && indexRow != null) {
       writeBlockletDetailsInfo(out);
     }
     out.writeBoolean(dataMapWritePath != null);
@@ -623,8 +623,8 @@ public class CarbonInputSplit extends FileSplit
     return dataMapWritePath;
   }
 
-  public void setDataMapRow(DataMapRow dataMapRow) {
-    this.dataMapRow = dataMapRow;
+  public void setIndexRow(IndexRow indexRow) {
+    this.indexRow = indexRow;
   }
 
   public void setColumnSchema(List<ColumnSchema> columnSchema) {
@@ -644,21 +644,21 @@ public class CarbonInputSplit extends FileSplit
   }
 
   private void writeBlockletDetailsInfo(DataOutput out) throws IOException {
-    out.writeInt(this.dataMapRow.getInt(BlockletDataMapRowIndexes.ROW_COUNT_INDEX));
+    out.writeInt(this.indexRow.getInt(BlockletIndexRowIndexes.ROW_COUNT_INDEX));
     if (this.isBlockCache) {
       out.writeShort(0);
     } else {
-      out.writeShort(this.dataMapRow.getShort(BlockletDataMapRowIndexes.BLOCKLET_PAGE_COUNT_INDEX));
+      out.writeShort(this.indexRow.getShort(BlockletIndexRowIndexes.BLOCKLET_PAGE_COUNT_INDEX));
     }
-    out.writeShort(this.dataMapRow.getShort(BlockletDataMapRowIndexes.VERSION_INDEX));
+    out.writeShort(this.indexRow.getShort(BlockletIndexRowIndexes.VERSION_INDEX));
     out.writeShort(Short.parseShort(this.blockletId));
-    out.writeLong(this.dataMapRow.getLong(BlockletDataMapRowIndexes.SCHEMA_UPADATED_TIME_INDEX));
+    out.writeLong(this.indexRow.getLong(BlockletIndexRowIndexes.SCHEMA_UPADATED_TIME_INDEX));
     out.writeBoolean(false);
-    out.writeLong(this.dataMapRow.getLong(BlockletDataMapRowIndexes.BLOCK_FOOTER_OFFSET));
+    out.writeLong(this.indexRow.getLong(BlockletIndexRowIndexes.BLOCK_FOOTER_OFFSET));
     // write -1 if columnSchemaBinary is null so that at the time of reading it can distinguish
     // whether schema is written or not
     if (null != this.columnSchema) {
-      byte[] columnSchemaBinary = BlockletDataMapUtil.convertSchemaToBinary(this.columnSchema);
+      byte[] columnSchemaBinary = BlockletIndexUtil.convertSchemaToBinary(this.columnSchema);
       out.writeInt(columnSchemaBinary.length);
       out.write(columnSchemaBinary);
     } else {
@@ -671,7 +671,7 @@ public class CarbonInputSplit extends FileSplit
       out.write(new byte[0]);
     } else {
       byte[] blockletInfoBinary =
-          this.dataMapRow.getByteArray(BlockletDataMapRowIndexes.BLOCKLET_INFO_INDEX);
+          this.indexRow.getByteArray(BlockletIndexRowIndexes.BLOCKLET_INFO_INDEX);
       out.writeInt(blockletInfoBinary.length);
       out.write(blockletInfoBinary);
     }
@@ -680,18 +680,18 @@ public class CarbonInputSplit extends FileSplit
   }
 
   public BlockletDetailInfo getDetailInfo() {
-    if (null != dataMapRow && detailInfo == null) {
+    if (null != indexRow && detailInfo == null) {
       detailInfo = new BlockletDetailInfo();
       detailInfo
-          .setRowCount(this.dataMapRow.getInt(BlockletDataMapRowIndexes.ROW_COUNT_INDEX));
+          .setRowCount(this.indexRow.getInt(BlockletIndexRowIndexes.ROW_COUNT_INDEX));
       rowCount = detailInfo.getRowCount();
       detailInfo
-          .setVersionNumber(this.dataMapRow.getShort(BlockletDataMapRowIndexes.VERSION_INDEX));
+          .setVersionNumber(this.indexRow.getShort(BlockletIndexRowIndexes.VERSION_INDEX));
       detailInfo.setBlockletId(Short.parseShort(this.blockletId));
       detailInfo.setSchemaUpdatedTimeStamp(
-          this.dataMapRow.getLong(BlockletDataMapRowIndexes.SCHEMA_UPADATED_TIME_INDEX));
+          this.indexRow.getLong(BlockletIndexRowIndexes.SCHEMA_UPADATED_TIME_INDEX));
       detailInfo.setBlockFooterOffset(
-          this.dataMapRow.getLong(BlockletDataMapRowIndexes.BLOCK_FOOTER_OFFSET));
+          this.indexRow.getLong(BlockletIndexRowIndexes.BLOCK_FOOTER_OFFSET));
       start = detailInfo.getBlockFooterOffset();
       detailInfo
           .setBlockSize(getLength());
@@ -700,21 +700,21 @@ public class CarbonInputSplit extends FileSplit
       if (!this.isBlockCache) {
         detailInfo.setColumnSchemas(this.columnSchema);
         detailInfo.setPagesCount(
-            this.dataMapRow.getShort(BlockletDataMapRowIndexes.BLOCKLET_PAGE_COUNT_INDEX));
+            this.indexRow.getShort(BlockletIndexRowIndexes.BLOCKLET_PAGE_COUNT_INDEX));
         detailInfo.setBlockletInfoBinary(
-            this.dataMapRow.getByteArray(BlockletDataMapRowIndexes.BLOCKLET_INFO_INDEX));
+            this.indexRow.getByteArray(BlockletIndexRowIndexes.BLOCKLET_INFO_INDEX));
       } else {
         detailInfo.setBlockletInfoBinary(new byte[0]);
       }
       if (location == null) {
         try {
-          location = new String(dataMapRow.getByteArray(BlockletDataMapRowIndexes.LOCATIONS),
+          location = new String(indexRow.getByteArray(BlockletIndexRowIndexes.LOCATIONS),
               CarbonCommonConstants.DEFAULT_CHARSET).split(",");
         } catch (IOException e) {
           throw new RuntimeException(e);
         }
       }
-      dataMapRow = null;
+      indexRow = null;
     }
     return detailInfo;
   }
@@ -751,8 +751,8 @@ public class CarbonInputSplit extends FileSplit
    */
   public void updateFooteroffset() {
     if (isBlockCache && start == 0) {
-      if (null != dataMapRow) {
-        start = this.dataMapRow.getLong(BlockletDataMapRowIndexes.BLOCK_FOOTER_OFFSET);
+      if (null != indexRow) {
+        start = this.indexRow.getLong(BlockletIndexRowIndexes.BLOCK_FOOTER_OFFSET);
       } else if (null != detailInfo) {
         start = detailInfo.getBlockFooterOffset();
       }
@@ -761,8 +761,8 @@ public class CarbonInputSplit extends FileSplit
 
   public void updateBlockLength() {
     if (length == -1) {
-      if (null != dataMapRow) {
-        length = this.dataMapRow.getLong(BlockletDataMapRowIndexes.BLOCK_LENGTH);
+      if (null != indexRow) {
+        length = this.indexRow.getLong(BlockletIndexRowIndexes.BLOCK_LENGTH);
       } else if (null != detailInfo) {
         length = detailInfo.getBlockSize();
       }
@@ -782,10 +782,10 @@ public class CarbonInputSplit extends FileSplit
 
   @Override
   public String[] getLocations() throws IOException {
-    if (this.location == null && dataMapRow == null) {
+    if (this.location == null && indexRow == null) {
       return new String[] {};
-    } else if (dataMapRow != null) {
-      location = new String(dataMapRow.getByteArray(BlockletDataMapRowIndexes.LOCATIONS),
+    } else if (indexRow != null) {
+      location = new String(indexRow.getByteArray(BlockletIndexRowIndexes.LOCATIONS),
           CarbonCommonConstants.DEFAULT_CHARSET).split(",");
     }
     return this.location;
diff --git a/core/src/test/java/org/apache/carbondata/core/indexstore/blockletindex/TestBlockletDataMap.java b/core/src/test/java/org/apache/carbondata/core/indexstore/blockletindex/TestBlockletIndex.java
similarity index 96%
rename from core/src/test/java/org/apache/carbondata/core/indexstore/blockletindex/TestBlockletDataMap.java
rename to core/src/test/java/org/apache/carbondata/core/indexstore/blockletindex/TestBlockletIndex.java
index d849194..8a998ef 100644
--- a/core/src/test/java/org/apache/carbondata/core/indexstore/blockletindex/TestBlockletDataMap.java
+++ b/core/src/test/java/org/apache/carbondata/core/indexstore/blockletindex/TestBlockletIndex.java
@@ -32,7 +32,7 @@ import mockit.MockUp;
 import org.junit.Before;
 import org.junit.Test;
 
-public class TestBlockletDataMap {
+public class TestBlockletIndex {
 
   ImplicitIncludeFilterExecutorImpl implicitIncludeFilterExecutor;
   @Before public void setUp() throws Exception {
@@ -58,8 +58,8 @@ public class TestBlockletDataMap {
       }
     };
 
-    BlockDataMap blockletDataMap = new BlockletDataMap();
-    Method method = BlockDataMap.class
+    BlockIndex blockletDataMap = new BlockletIndex();
+    Method method = BlockIndex.class
         .getDeclaredMethod("addBlockBasedOnMinMaxValue", FilterExecuter.class, byte[][].class,
             byte[][].class, boolean[].class, String.class, int.class);
     method.setAccessible(true);
diff --git a/core/src/test/java/org/apache/carbondata/core/indexstore/blockletindex/TestBlockletDataMapFactory.java b/core/src/test/java/org/apache/carbondata/core/indexstore/blockletindex/TestBlockletIndexFactory.java
similarity index 79%
rename from core/src/test/java/org/apache/carbondata/core/indexstore/blockletindex/TestBlockletDataMapFactory.java
rename to core/src/test/java/org/apache/carbondata/core/indexstore/blockletindex/TestBlockletIndexFactory.java
index c5812bc..ad7061c 100644
--- a/core/src/test/java/org/apache/carbondata/core/indexstore/blockletindex/TestBlockletDataMapFactory.java
+++ b/core/src/test/java/org/apache/carbondata/core/indexstore/blockletindex/TestBlockletIndexFactory.java
@@ -30,9 +30,9 @@ import java.util.UUID;
 import org.apache.carbondata.core.cache.Cache;
 import org.apache.carbondata.core.cache.CacheProvider;
 import org.apache.carbondata.core.cache.CacheType;
-import org.apache.carbondata.core.datamap.DataMapDistributable;
+import org.apache.carbondata.core.datamap.IndexInputSplit;
 import org.apache.carbondata.core.datamap.Segment;
-import org.apache.carbondata.core.indexstore.BlockletDataMapIndexWrapper;
+import org.apache.carbondata.core.indexstore.BlockletIndexWrapper;
 import org.apache.carbondata.core.indexstore.TableBlockIndexUniqueIdentifier;
 import org.apache.carbondata.core.indexstore.TableBlockIndexUniqueIdentifierWrapper;
 import org.apache.carbondata.core.memory.MemoryException;
@@ -50,7 +50,7 @@ import org.apache.hadoop.conf.Configuration;
 import org.junit.Before;
 import org.junit.Test;
 
-public class TestBlockletDataMapFactory {
+public class TestBlockletIndexFactory {
 
   private CarbonTable carbonTable;
 
@@ -58,13 +58,13 @@ public class TestBlockletDataMapFactory {
 
   private TableInfo tableInfo;
 
-  private BlockletDataMapFactory blockletDataMapFactory;
+  private BlockletIndexFactory blockletIndexFactory;
 
   private TableBlockIndexUniqueIdentifier tableBlockIndexUniqueIdentifier;
 
   private TableBlockIndexUniqueIdentifierWrapper tableBlockIndexUniqueIdentifierWrapper;
 
-  private Cache<TableBlockIndexUniqueIdentifierWrapper, BlockletDataMapIndexWrapper> cache;
+  private Cache<TableBlockIndexUniqueIdentifierWrapper, BlockletIndexWrapper> cache;
 
   private TableSchema factTable;
 
@@ -90,8 +90,8 @@ public class TestBlockletDataMapFactory {
         return absoluteTableIdentifier;
       }
     };
-    blockletDataMapFactory = new BlockletDataMapFactory(carbonTable, new DataMapSchema());
-    Deencapsulation.setField(blockletDataMapFactory, "cache",
+    blockletIndexFactory = new BlockletIndexFactory(carbonTable, new DataMapSchema());
+    Deencapsulation.setField(blockletIndexFactory, "cache",
         CacheProvider.getInstance().createCache(CacheType.DRIVER_BLOCKLET_DATAMAP));
     tableBlockIndexUniqueIdentifier =
         new TableBlockIndexUniqueIdentifier("/opt/store/default/carbon_table/Fact/Part0/Segment_0",
@@ -104,30 +104,30 @@ public class TestBlockletDataMapFactory {
   @Test public void addDataMapToCache()
       throws IOException, MemoryException, NoSuchMethodException, InvocationTargetException,
       IllegalAccessException {
-    List<BlockDataMap> dataMaps = new ArrayList<>();
-    Method method = BlockletDataMapFactory.class
+    List<BlockIndex> dataMaps = new ArrayList<>();
+    Method method = BlockletIndexFactory.class
         .getDeclaredMethod("cache", TableBlockIndexUniqueIdentifierWrapper.class,
-            BlockletDataMapIndexWrapper.class);
+            BlockletIndexWrapper.class);
     method.setAccessible(true);
-    method.invoke(blockletDataMapFactory, tableBlockIndexUniqueIdentifierWrapper,
-        new BlockletDataMapIndexWrapper(tableBlockIndexUniqueIdentifier.getSegmentId(), dataMaps));
-    BlockletDataMapIndexWrapper result = cache.getIfPresent(tableBlockIndexUniqueIdentifierWrapper);
+    method.invoke(blockletIndexFactory, tableBlockIndexUniqueIdentifierWrapper,
+        new BlockletIndexWrapper(tableBlockIndexUniqueIdentifier.getSegmentId(), dataMaps));
+    BlockletIndexWrapper result = cache.getIfPresent(tableBlockIndexUniqueIdentifierWrapper);
     assert null != result;
   }
 
   @Test public void getValidDistributables() throws IOException {
-    BlockletDataMapDistributable blockletDataMapDistributable = new BlockletDataMapDistributable(
+    BlockletIndexInputSplit blockletDataMapDistributable = new BlockletIndexInputSplit(
         "/opt/store/default/carbon_table/Fact/Part0/Segment_0/0_batchno0-0-1521012756709.carbonindex");
     Segment segment = new Segment("0", null, new TableStatusReadCommittedScope(carbonTable
         .getAbsoluteTableIdentifier(), new Configuration(false)));
     blockletDataMapDistributable.setSegment(segment);
-    BlockletDataMapDistributable blockletDataMapDistributable1 = new BlockletDataMapDistributable(
+    BlockletIndexInputSplit blockletDataMapDistributable1 = new BlockletIndexInputSplit(
         "/opt/store/default/carbon_table/Fact/Part0/Segment_0/0_batchno0-0-1521012756701.carbonindex");
     blockletDataMapDistributable1.setSegment(segment);
-    List<DataMapDistributable> dataMapDistributables = new ArrayList<>(2);
-    dataMapDistributables.add(blockletDataMapDistributable);
-    dataMapDistributables.add(blockletDataMapDistributable1);
-    new MockUp<BlockletDataMapFactory>() {
+    List<IndexInputSplit> indexInputSplits = new ArrayList<>(2);
+    indexInputSplits.add(blockletDataMapDistributable);
+    indexInputSplits.add(blockletDataMapDistributable1);
+    new MockUp<BlockletIndexFactory>() {
       @Mock Set<TableBlockIndexUniqueIdentifier> getTableBlockIndexUniqueIdentifiers(
           Segment segment) {
         TableBlockIndexUniqueIdentifier tableBlockIndexUniqueIdentifier1 =
@@ -140,8 +140,8 @@ public class TestBlockletDataMapFactory {
         return tableBlockIndexUniqueIdentifiers;
       }
     };
-    List<DataMapDistributable> validDistributables =
-        blockletDataMapFactory.getAllUncachedDistributables(dataMapDistributables);
+    List<IndexInputSplit> validDistributables =
+        blockletIndexFactory.getAllUncachedDistributables(indexInputSplits);
     assert 1 == validDistributables.size();
   }
 }
\ No newline at end of file
diff --git a/dev/findbugs-exclude.xml b/dev/findbugs-exclude.xml
index 7c4a412..ac458d9 100644
--- a/dev/findbugs-exclude.xml
+++ b/dev/findbugs-exclude.xml
@@ -59,7 +59,7 @@
     <Bug pattern="STCAL_INVOKE_ON_STATIC_DATE_FORMAT_INSTANCE"/>
   </Match>
   <Match>
-    <Class name="org.apache.carbondata.core.datamap.DistributableDataMapFormat"/>
+    <Class name="org.apache.carbondata.core.datamap.IndexInputFormat"/>
     <Bug pattern="SE_TRANSIENT_FIELD_NOT_RESTORED"/>
   </Match>
   <Match>
@@ -115,17 +115,17 @@
     <Bug pattern="SE_TRANSIENT_FIELD_NOT_RESTORED"/>
   </Match>
   <Match>
-    <Class name="org.apache.carbondata.core.datamap.DataMapFilter"/>
+    <Class name="org.apache.carbondata.core.datamap.IndexFilter"/>
     <Bug pattern="SE_TRANSIENT_FIELD_NOT_RESTORED"/>
   </Match>
   <Match>
-    <Class name="org.apache.carbondata.core.datamap.DataMapFilter"/>
+    <Class name="org.apache.carbondata.core.datamap.IndexFilter"/>
     <Bug pattern="SE_BAD_FIELD"/>
   </Match>
   <Match>
     <Class name="org.apache.carbondata.events.OperationContext"/>
     <Bug pattern="SE_TRANSIENT_FIELD_NOT_RESTORED"/>
   </Match>
-  <Match> <Class name="~org.apache.spark.sql.secondaryindex.Jobs.DistributableBlockletDataMapLoader"/> <Field name="dataMapExprWrapper"/> <Bug pattern="SE_TRANSIENT_FIELD_NOT_RESTORED"/> </Match>
-  <Match> <Class name="~org.apache.spark.sql.secondaryindex.Jobs.DistributableBlockletDataMapLoader"/> <Field name="validSegments"/> <Bug pattern="SE_TRANSIENT_FIELD_NOT_RESTORED"/> </Match>
+  <Match> <Class name="~org.apache.spark.sql.secondaryindex.Jobs.BlockletIndexInputFormat"/> <Field name="indexExprWrapper"/> <Bug pattern="SE_TRANSIENT_FIELD_NOT_RESTORED"/> </Match>
+  <Match> <Class name="~org.apache.spark.sql.secondaryindex.Jobs.BlockletIndexInputFormat"/> <Field name="validSegments"/> <Bug pattern="SE_TRANSIENT_FIELD_NOT_RESTORED"/> </Match>
 </FindBugsFilter>
\ No newline at end of file
diff --git a/docs/faq.md b/docs/faq.md
index 88ca186..45607f4 100644
--- a/docs/faq.md
+++ b/docs/faq.md
@@ -44,7 +44,7 @@
 - [Failed to insert data on the cluster](#failed-to-insert-data-on-the-cluster)
 - [Failed to execute Concurrent Operations(Load,Insert,Update) on table by multiple workers](#failed-to-execute-concurrent-operations-on-table-by-multiple-workers)
 - [Failed to create a table with a single numeric column](#failed-to-create-a-table-with-a-single-numeric-column)
-- [Failed to create datamap and drop datamap is also not working](#failed-to-create-datamap-and-drop-datamap-is-also-not-working)
+- [Failed to create_index and drop index is also not working](#failed-to-create-index-and-drop-index-is-also-not-working)
 
 ## 
 
@@ -499,7 +499,7 @@ Note : Refrain from using "mvn clean package" without specifying the profile.
 
   A single column that can be considered as dimension is mandatory for table creation.
 
-## Failed to create datamap and drop datamap is also not working
+## Failed to create index and drop index is also not working
   
   **Symptom**
 
@@ -515,5 +515,4 @@ Note : Refrain from using "mvn clean package" without specifying the profile.
 
   **Procedure**
 
-  Drop that particular datamap using Drop Table command using table name as
-  parentTableName_datamapName so as to clear the stale folders.
+  Drop that particular index using Drop Index command so as to clear the stale folders.
diff --git a/docs/index-developer-guide.md b/docs/index-developer-guide.md
index d48c890..106bf39 100644
--- a/docs/index-developer-guide.md
+++ b/docs/index-developer-guide.md
@@ -24,11 +24,11 @@ Currently, there are two types of DataMap supported:
 2. MVDataMap: DataMap that leverages Materialized View to accelerate olap style query, like SPJG query (select, predicate, join, groupby). Preaggregate, timeseries and mv DataMap belong to this type of DataMaps.
 
 ### Index Provider
-When user issues `CREATE DATAMAP dm ON TABLE main USING 'provider'`, the corresponding DataMapProvider implementation will be created and initialized. 
+When user issues `CREATE INDEX index_name ON TABLE main AS 'provider'`, the corresponding DataMapProvider implementation will be created and initialized. 
 Currently, the provider string can be:
 1. class name IndexDataMapFactory implementation: Developer can implement new type of IndexDataMap by extending IndexDataMapFactory
 
-When user issues `DROP DATAMAP dm ON TABLE main`, the corresponding DataMapProvider interface will be called.
+When user issues `DROP INDEX index_name ON TABLE main`, the corresponding DataMapProvider interface will be called.
 
 Click for more details about [DataMap Management](./index/index-management.md#index-management) and supported [DSL](./index/index-management.md#overview).
 
diff --git a/docs/index/lucene-index-guide.md b/docs/index/lucene-index-guide.md
index aa9c8d4..d12aa47 100644
--- a/docs/index/lucene-index-guide.md
+++ b/docs/index/lucene-index-guide.md
@@ -160,7 +160,7 @@ is not supported:
 `ALTER TABLE RENAME`.
 
 **Note**: Adding a new column is supported, and for dropping columns and change datatype 
-command, CarbonData will check whether it will impact the lucene datamap, if not, the operation 
+command, CarbonData will check whether it will impact the lucene index, if not, the operation 
 is allowed, otherwise operation will be rejected by throwing exception.
 
 
@@ -168,9 +168,9 @@ is allowed, otherwise operation will be rejected by throwing exception.
 
 However, there is still way to support these operations on main table, in current CarbonData 
 release, user can do as following:
-1. Remove the lucene datamap by `DROP DATAMAP` command.
+1. Remove the lucene index by `DROP INDEX` command.
 2. Carry out the data management operation on main table.
-3. Create the lucene datamap again by `CREATE DATAMAP` command.
-Basically, user can manually trigger the operation by re-building the datamap.
+3. Create the lucene index again by `CREATE INDEX` command.
+Basically, user can manually trigger the operation by refreshing the index.
 
 
diff --git a/examples/flink/src/main/scala/org/apache/carbondata/examples/FlinkExample.scala b/examples/flink/src/main/scala/org/apache/carbondata/examples/FlinkExample.scala
index 239a038..6b9c4fb 100644
--- a/examples/flink/src/main/scala/org/apache/carbondata/examples/FlinkExample.scala
+++ b/examples/flink/src/main/scala/org/apache/carbondata/examples/FlinkExample.scala
@@ -21,8 +21,9 @@ import org.apache.flink.api.java.ExecutionEnvironment
 import org.apache.hadoop.conf.Configuration
 import org.apache.hadoop.mapreduce.Job
 
+import org.apache.carbondata.examples.util.ExampleUtils
 import org.apache.carbondata.hadoop.CarbonProjection
-import org.apache.carbondata.hadoop.api.CarbonTableInputFormat
+import org.apache.carbondata.hadoop.api.{CarbonInputFormat, CarbonTableInputFormat}
 
 // Write carbondata file by spark and read it by flink
 // scalastyle:off println
@@ -38,7 +39,7 @@ object FlinkExample {
     projection.addColumn("c1")  // column c1
     projection.addColumn("c3")  // column c3
     val conf = new Configuration()
-    CarbonTableInputFormat.setColumnProjection(conf, projection)
+    CarbonInputFormat.setColumnProjection(conf, projection)
 
     val env = ExecutionEnvironment.getExecutionEnvironment
     val ds = env.readHadoopFile(
diff --git a/examples/spark/src/main/scala/org/apache/carbondata/examples/LuceneDataMapExample.scala b/examples/spark/src/main/scala/org/apache/carbondata/examples/LuceneIndexExample.scala
similarity index 91%
rename from examples/spark/src/main/scala/org/apache/carbondata/examples/LuceneDataMapExample.scala
rename to examples/spark/src/main/scala/org/apache/carbondata/examples/LuceneIndexExample.scala
index bcc16ca..be83b2c 100644
--- a/examples/spark/src/main/scala/org/apache/carbondata/examples/LuceneDataMapExample.scala
+++ b/examples/spark/src/main/scala/org/apache/carbondata/examples/LuceneIndexExample.scala
@@ -23,13 +23,13 @@ import org.apache.carbondata.examples.util.ExampleUtils
 
 
 /**
- * This example is for lucene datamap.
+ * This example is for lucene index.
  */
 
-object LuceneDataMapExample {
+object LuceneIndexExample {
 
   def main(args: Array[String]) {
-    val spark = ExampleUtils.createSparkSession("LuceneDataMapExample")
+    val spark = ExampleUtils.createSparkSession("LuceneIndexExample")
     exampleBody(spark)
     spark.close()
   }
@@ -56,9 +56,8 @@ object LuceneDataMapExample {
     // create lucene datamap on personTable
     spark.sql(
       s"""
-         | CREATE DATAMAP IF NOT EXISTS dm ON TABLE personTable
-         | USING 'lucene'
-         | DMProperties('INDEX_COLUMNS'='id , name')
+         | CREATE INDEX IF NOT EXISTS dm ON TABLE personTable (id, name)
+         | AS 'lucene'
       """.stripMargin)
 
     // 1. Compare the performance:
diff --git a/examples/spark/src/main/scala/org/apache/carbondata/examples/MVExample.scala b/examples/spark/src/main/scala/org/apache/carbondata/examples/MVExample.scala
index c89d868..4c0f8bd 100644
--- a/examples/spark/src/main/scala/org/apache/carbondata/examples/MVExample.scala
+++ b/examples/spark/src/main/scala/org/apache/carbondata/examples/MVExample.scala
@@ -145,9 +145,9 @@ object MVExample {
          |employee_salary group by name, address""".stripMargin)
 
     spark.sql(
-      s"""create datamap simple_agg_employee using 'mv' as
+      s"""create materialized view simple_agg_employee as
          | select id,sum(salary) from employee_salary group by id""".stripMargin)
-    spark.sql(s"""rebuild datamap simple_agg_employee""")
+    spark.sql(s"""refresh materialized view simple_agg_employee""")
 
     // Test performance of aggregate queries with mv datamap
     val timeWithOutMv = time(spark
@@ -173,10 +173,10 @@ object MVExample {
 
     // Tests performance of aggregate with join queries.
     spark.sql(
-      s"""create datamap simple_join_agg_employee using 'mv' as
+      s"""create materialized view simple_join_agg_employee as
          | select id,address, sum(salary) from employee_salary f join emp_address d
          | on f.name=d.name group by id,address""".stripMargin)
-    spark.sql(s"""rebuild datamap simple_join_agg_employee""")
+    spark.sql(s"""refresh materialized view simple_join_agg_employee""")
 
     val timeWithMVJoin =
       time(spark.sql(
diff --git a/examples/spark/src/test/scala/org/apache/carbondata/examplesCI/RunExamples.scala b/examples/spark/src/test/scala/org/apache/carbondata/examplesCI/RunExamples.scala
index 7155eaf..5c2b767 100644
--- a/examples/spark/src/test/scala/org/apache/carbondata/examplesCI/RunExamples.scala
+++ b/examples/spark/src/test/scala/org/apache/carbondata/examplesCI/RunExamples.scala
@@ -105,7 +105,7 @@ class RunExamples extends QueryTest with BeforeAndAfterAll {
   }
 
   test("LuceneDataMapExample") {
-    LuceneDataMapExample.exampleBody(spark)
+    LuceneIndexExample.exampleBody(spark)
   }
 
   test("ExternalTableExample") {
diff --git a/hadoop/src/main/java/org/apache/carbondata/hadoop/CarbonRecordReader.java b/hadoop/src/main/java/org/apache/carbondata/hadoop/CarbonRecordReader.java
index c8c1834..d4eda22 100644
--- a/hadoop/src/main/java/org/apache/carbondata/hadoop/CarbonRecordReader.java
+++ b/hadoop/src/main/java/org/apache/carbondata/hadoop/CarbonRecordReader.java
@@ -161,7 +161,7 @@ public class CarbonRecordReader<T> extends AbstractRecordReader<T> {
     logStatistics(rowCount, queryModel.getStatisticsRecorder());
     if (!skipClearDataMapAtClose) {
       // Clear the datamap cache
-      DataMapStoreManager.getInstance().clearDataMapCache(
+      DataMapStoreManager.getInstance().clearIndexCache(
           queryModel.getTable().getAbsoluteTableIdentifier(), false);
     }
     // close read support
diff --git a/hadoop/src/main/java/org/apache/carbondata/hadoop/api/CarbonFileInputFormat.java b/hadoop/src/main/java/org/apache/carbondata/hadoop/api/CarbonFileInputFormat.java
index 002ab7b..14f9240 100644
--- a/hadoop/src/main/java/org/apache/carbondata/hadoop/api/CarbonFileInputFormat.java
+++ b/hadoop/src/main/java/org/apache/carbondata/hadoop/api/CarbonFileInputFormat.java
@@ -27,7 +27,7 @@ import java.util.List;
 
 import org.apache.carbondata.common.annotations.InterfaceAudience;
 import org.apache.carbondata.common.annotations.InterfaceStability;
-import org.apache.carbondata.core.datamap.DataMapFilter;
+import org.apache.carbondata.core.datamap.IndexFilter;
 import org.apache.carbondata.core.datamap.Segment;
 import org.apache.carbondata.core.datastore.filesystem.CarbonFile;
 import org.apache.carbondata.core.datastore.filesystem.CarbonFileFilter;
@@ -128,7 +128,7 @@ public class CarbonFileInputFormat<T> extends CarbonInputFormat<T> implements Se
       }
     }
     // this will be null in case of corrupt schema file.
-    DataMapFilter filter = getFilterPredicates(job.getConfiguration());
+    IndexFilter filter = getFilterPredicates(job.getConfiguration());
 
     // if external table Segments are found, add it to the List
     List<Segment> externalTableSegments = new ArrayList<Segment>();
@@ -249,7 +249,7 @@ public class CarbonFileInputFormat<T> extends CarbonInputFormat<T> implements Se
    */
   private List<InputSplit> getSplits(
       JobContext job,
-      DataMapFilter dataMapFilter,
+      IndexFilter indexFilter,
       List<Segment> validSegments) throws IOException {
 
     numSegments = validSegments.size();
@@ -257,7 +257,7 @@ public class CarbonFileInputFormat<T> extends CarbonInputFormat<T> implements Se
 
     // for each segment fetch blocks matching filter in Driver BTree
     List<CarbonInputSplit> dataBlocksOfSegment =
-        getDataBlocksOfSegment(job, carbonTable, dataMapFilter, validSegments,
+        getDataBlocksOfSegment(job, carbonTable, indexFilter, validSegments,
             new ArrayList<Segment>(), new ArrayList<String>());
     numBlocks = dataBlocksOfSegment.size();
     result.addAll(dataBlocksOfSegment);
diff --git a/hadoop/src/main/java/org/apache/carbondata/hadoop/api/CarbonInputFormat.java b/hadoop/src/main/java/org/apache/carbondata/hadoop/api/CarbonInputFormat.java
index 4f477d7..48fc382 100644
--- a/hadoop/src/main/java/org/apache/carbondata/hadoop/api/CarbonInputFormat.java
+++ b/hadoop/src/main/java/org/apache/carbondata/hadoop/api/CarbonInputFormat.java
@@ -31,17 +31,17 @@ import java.util.Set;
 
 import org.apache.carbondata.common.logging.LogServiceFactory;
 import org.apache.carbondata.core.constants.CarbonCommonConstants;
-import org.apache.carbondata.core.datamap.DataMapChooser;
-import org.apache.carbondata.core.datamap.DataMapFilter;
-import org.apache.carbondata.core.datamap.DataMapJob;
-import org.apache.carbondata.core.datamap.DataMapLevel;
 import org.apache.carbondata.core.datamap.DataMapStoreManager;
-import org.apache.carbondata.core.datamap.DataMapUtil;
-import org.apache.carbondata.core.datamap.DistributableDataMapFormat;
+import org.apache.carbondata.core.datamap.IndexChooser;
+import org.apache.carbondata.core.datamap.IndexFilter;
+import org.apache.carbondata.core.datamap.IndexInputFormat;
+import org.apache.carbondata.core.datamap.IndexJob;
+import org.apache.carbondata.core.datamap.IndexLevel;
+import org.apache.carbondata.core.datamap.IndexUtil;
 import org.apache.carbondata.core.datamap.Segment;
-import org.apache.carbondata.core.datamap.TableDataMap;
-import org.apache.carbondata.core.datamap.dev.expr.DataMapExprWrapper;
-import org.apache.carbondata.core.datamap.dev.expr.DataMapWrapperSimpleInfo;
+import org.apache.carbondata.core.datamap.TableIndex;
+import org.apache.carbondata.core.datamap.dev.expr.IndexExprWrapper;
+import org.apache.carbondata.core.datamap.dev.expr.IndexWrapperSimpleInfo;
 import org.apache.carbondata.core.exception.InvalidConfigurationException;
 import org.apache.carbondata.core.indexstore.ExtendedBlocklet;
 import org.apache.carbondata.core.indexstore.PartitionSpec;
@@ -60,7 +60,7 @@ import org.apache.carbondata.core.scan.model.QueryModelBuilder;
 import org.apache.carbondata.core.stats.QueryStatistic;
 import org.apache.carbondata.core.stats.QueryStatisticsConstants;
 import org.apache.carbondata.core.stats.QueryStatisticsRecorder;
-import org.apache.carbondata.core.util.BlockletDataMapUtil;
+import org.apache.carbondata.core.util.BlockletIndexUtil;
 import org.apache.carbondata.core.util.CarbonProperties;
 import org.apache.carbondata.core.util.CarbonTimeStatisticsFactory;
 import org.apache.carbondata.core.util.CarbonUtil;
@@ -217,11 +217,11 @@ public abstract class CarbonInputFormat<T> extends FileInputFormat<Void, T> {
    * It sets unresolved filter expression.
    *
    * @param configuration
-   * @para    DataMapJob dataMapJob = getDataMapJob(job.getConfiguration());
+   * @para    IndexJob dataMapJob = getDataMapJob(job.getConfiguration());
 m filterExpression
    */
   public static void setFilterPredicates(Configuration configuration,
-      DataMapFilter filterExpression) {
+      IndexFilter filterExpression) {
     if (filterExpression == null || filterExpression.getExpression() == null) {
       return;
     }
@@ -412,21 +412,21 @@ m filterExpression
    */
   Long getDistributedCount(CarbonTable table,
       List<PartitionSpec> partitionNames, List<Segment> validSegments) {
-    DistributableDataMapFormat dataMapFormat =
-        new DistributableDataMapFormat(table, null, validSegments, new ArrayList<String>(),
+    IndexInputFormat dataMapFormat =
+        new IndexInputFormat(table, null, validSegments, new ArrayList<String>(),
             partitionNames, false, null, false, false);
     dataMapFormat.setIsWriteToFile(false);
     try {
-      DataMapJob dataMapJob =
-          (DataMapJob) DataMapUtil.createDataMapJob(DataMapUtil.DISTRIBUTED_JOB_NAME);
-      if (dataMapJob == null) {
+      IndexJob indexJob =
+          (IndexJob) IndexUtil.createDataMapJob(IndexUtil.DISTRIBUTED_JOB_NAME);
+      if (indexJob == null) {
         throw new ExceptionInInitializerError("Unable to create DistributedDataMapJob");
       }
-      return dataMapJob.executeCountJob(dataMapFormat);
+      return indexJob.executeCountJob(dataMapFormat);
     } catch (Exception e) {
       LOG.error("Failed to get count from index server. Initializing fallback", e);
-      DataMapJob dataMapJob = DataMapUtil.getEmbeddedJob();
-      return dataMapJob.executeCountJob(dataMapFormat);
+      IndexJob indexJob = IndexUtil.getEmbeddedJob();
+      return indexJob.executeCountJob(dataMapFormat);
     }
   }
 
@@ -442,13 +442,13 @@ m filterExpression
       List<Segment> validSegments, List<Segment> invalidSegments,
       List<String> segmentsToBeRefreshed, boolean isCountJob) {
     try {
-      DataMapJob dataMapJob =
-          (DataMapJob) DataMapUtil.createDataMapJob(DataMapUtil.DISTRIBUTED_JOB_NAME);
-      if (dataMapJob == null) {
+      IndexJob indexJob =
+          (IndexJob) IndexUtil.createDataMapJob(IndexUtil.DISTRIBUTED_JOB_NAME);
+      if (indexJob == null) {
         throw new ExceptionInInitializerError("Unable to create DistributedDataMapJob");
       }
-      return DataMapUtil
-          .executeDataMapJob(table, filterResolverIntf, dataMapJob, partitionNames, validSegments,
+      return IndexUtil
+          .executeDataMapJob(table, filterResolverIntf, indexJob, partitionNames, validSegments,
               invalidSegments, null, false, segmentsToBeRefreshed, isCountJob);
     } catch (Exception e) {
       // Check if fallback is disabled for testing purposes then directly throw exception.
@@ -457,20 +457,20 @@ m filterExpression
       }
       LOG.error("Exception occurred while getting splits using index server. Initiating Fall "
           + "back to embedded mode", e);
-      return DataMapUtil.executeDataMapJob(table, filterResolverIntf,
-          DataMapUtil.getEmbeddedJob(), partitionNames, validSegments,
+      return IndexUtil.executeDataMapJob(table, filterResolverIntf,
+          IndexUtil.getEmbeddedJob(), partitionNames, validSegments,
           invalidSegments, null, true, segmentsToBeRefreshed, isCountJob);
     }
   }
 
-  public DataMapFilter getFilterPredicates(Configuration configuration) {
+  public IndexFilter getFilterPredicates(Configuration configuration) {
     try {
       String filterExprString = configuration.get(FILTER_PREDICATE);
       if (filterExprString == null) {
         return null;
       }
-      DataMapFilter filter =
-          (DataMapFilter) ObjectSerializationUtil.convertStringToObject(filterExprString);
+      IndexFilter filter =
+          (IndexFilter) ObjectSerializationUtil.convertStringToObject(filterExprString);
       if (filter != null) {
         CarbonTable carbonTable = getOrCreateCarbonTable(configuration);
         filter.setTable(carbonTable);
@@ -485,7 +485,7 @@ m filterExpression
    * get data blocks of given segment
    */
   protected List<CarbonInputSplit> getDataBlocksOfSegment(JobContext job, CarbonTable carbonTable,
-      DataMapFilter expression, List<Segment> segmentIds,
+      IndexFilter expression, List<Segment> segmentIds,
       List<Segment> invalidSegments, List<String> segmentsToBeRefreshed)
       throws IOException {
 
@@ -530,19 +530,19 @@ m filterExpression
    * First pruned with default blocklet datamap, then pruned with CG and FG datamaps
    */
   private List<ExtendedBlocklet> getPrunedBlocklets(JobContext job, CarbonTable carbonTable,
-      DataMapFilter filter, List<Segment> segmentIds, List<Segment> invalidSegments,
+      IndexFilter filter, List<Segment> segmentIds, List<Segment> invalidSegments,
       List<String> segmentsToBeRefreshed) throws IOException {
     ExplainCollector.addPruningInfo(carbonTable.getTableName());
-    filter = filter == null ? new DataMapFilter(carbonTable, null) : filter;
+    filter = filter == null ? new IndexFilter(carbonTable, null) : filter;
     ExplainCollector.setFilterStatement(
         filter.getExpression() == null ? "none" : filter.getExpression().getStatement());
     boolean distributedCG = Boolean.parseBoolean(CarbonProperties.getInstance()
         .getProperty(CarbonCommonConstants.USE_DISTRIBUTED_DATAMAP,
             CarbonCommonConstants.USE_DISTRIBUTED_DATAMAP_DEFAULT));
-    DataMapJob dataMapJob = DataMapUtil.getDataMapJob(job.getConfiguration());
+    IndexJob indexJob = IndexUtil.getDataMapJob(job.getConfiguration());
     List<PartitionSpec> partitionsToPrune = getPartitionsToPrune(job.getConfiguration());
     // First prune using default datamap on driver side.
-    TableDataMap defaultDataMap = DataMapStoreManager.getInstance().getDefaultDataMap(carbonTable);
+    TableIndex defaultDataMap = DataMapStoreManager.getInstance().getDefaultIndex(carbonTable);
     List<ExtendedBlocklet> prunedBlocklets = null;
     // This is to log the event, so user will know what is happening by seeing logs.
     LOG.info("Started block pruning ...");
@@ -563,9 +563,9 @@ m filterExpression
       }
     } else {
       if (carbonTable.isTransactionalTable()) {
-        DataMapExprWrapper dataMapExprWrapper =
-            DataMapChooser.getDefaultDataMap(getOrCreateCarbonTable(job.getConfiguration()), null);
-        DataMapUtil.loadDataMaps(carbonTable, dataMapExprWrapper, segmentIds, partitionsToPrune);
+        IndexExprWrapper indexExprWrapper =
+            IndexChooser.getDefaultDataMap(getOrCreateCarbonTable(job.getConfiguration()), null);
+        IndexUtil.loadDataMaps(carbonTable, indexExprWrapper, segmentIds, partitionsToPrune);
       }
       prunedBlocklets = defaultDataMap.prune(segmentIds, filter, partitionsToPrune);
 
@@ -577,24 +577,24 @@ m filterExpression
         return prunedBlocklets;
       }
 
-      DataMapChooser chooser = new DataMapChooser(getOrCreateCarbonTable(job.getConfiguration()));
+      IndexChooser chooser = new IndexChooser(getOrCreateCarbonTable(job.getConfiguration()));
 
       // Get the available CG datamaps and prune further.
-      DataMapExprWrapper cgDataMapExprWrapper = chooser.chooseCGDataMap(filter.getResolver());
+      IndexExprWrapper cgIndexExprWrapper = chooser.chooseCGDataMap(filter.getResolver());
 
-      if (cgDataMapExprWrapper != null) {
+      if (cgIndexExprWrapper != null) {
         // Prune segments from already pruned blocklets
-        DataMapUtil.pruneSegments(segmentIds, prunedBlocklets);
+        IndexUtil.pruneSegments(segmentIds, prunedBlocklets);
         List<ExtendedBlocklet> cgPrunedBlocklets = new ArrayList<>();
         boolean isCGPruneFallback = false;
         // Again prune with CG datamap.
         try {
-          if (distributedCG && dataMapJob != null) {
-            cgPrunedBlocklets = DataMapUtil
-                .executeDataMapJob(carbonTable, filter.getResolver(), dataMapJob, partitionsToPrune,
-                    segmentIds, invalidSegments, DataMapLevel.CG, new ArrayList<String>());
+          if (distributedCG && indexJob != null) {
+            cgPrunedBlocklets = IndexUtil
+                .executeDataMapJob(carbonTable, filter.getResolver(), indexJob, partitionsToPrune,
+                    segmentIds, invalidSegments, IndexLevel.CG, new ArrayList<String>());
           } else {
-            cgPrunedBlocklets = cgDataMapExprWrapper.prune(segmentIds, partitionsToPrune);
+            cgPrunedBlocklets = cgIndexExprWrapper.prune(segmentIds, partitionsToPrune);
           }
         } catch (Exception e) {
           isCGPruneFallback = true;
@@ -610,7 +610,7 @@ m filterExpression
         }
         if (ExplainCollector.enabled()) {
           ExplainCollector.recordCGDataMapPruning(
-              DataMapWrapperSimpleInfo.fromDataMapWrapper(cgDataMapExprWrapper),
+              IndexWrapperSimpleInfo.fromDataMapWrapper(cgIndexExprWrapper),
               prunedBlocklets.size(), getBlockCount(prunedBlocklets));
         }
       }
@@ -618,24 +618,24 @@ m filterExpression
       if (prunedBlocklets.size() == 0) {
         return prunedBlocklets;
       }
-      // Now try to prune with FG DataMap.
-      if (isFgDataMapPruningEnable(job.getConfiguration()) && dataMapJob != null) {
-        DataMapExprWrapper fgDataMapExprWrapper = chooser.chooseFGDataMap(filter.getResolver());
+      // Now try to prune with FG Index.
+      if (isFgDataMapPruningEnable(job.getConfiguration()) && indexJob != null) {
+        IndexExprWrapper fgIndexExprWrapper = chooser.chooseFGDataMap(filter.getResolver());
         List<ExtendedBlocklet> fgPrunedBlocklets;
-        if (fgDataMapExprWrapper != null) {
+        if (fgIndexExprWrapper != null) {
           // Prune segments from already pruned blocklets
-          DataMapUtil.pruneSegments(segmentIds, prunedBlocklets);
+          IndexUtil.pruneSegments(segmentIds, prunedBlocklets);
           // Prune segments from already pruned blocklets
-          fgPrunedBlocklets = DataMapUtil
-              .executeDataMapJob(carbonTable, filter.getResolver(), dataMapJob, partitionsToPrune,
-                  segmentIds, invalidSegments, fgDataMapExprWrapper.getDataMapLevel(),
+          fgPrunedBlocklets = IndexUtil
+              .executeDataMapJob(carbonTable, filter.getResolver(), indexJob, partitionsToPrune,
+                  segmentIds, invalidSegments, fgIndexExprWrapper.getDataMapLevel(),
                   new ArrayList<String>());
           // note that the 'fgPrunedBlocklets' has extra datamap related info compared with
           // 'prunedBlocklets', so the intersection should keep the elements in 'fgPrunedBlocklets'
           prunedBlocklets =
               intersectFilteredBlocklets(carbonTable, prunedBlocklets, fgPrunedBlocklets);
           ExplainCollector.recordFGDataMapPruning(
-              DataMapWrapperSimpleInfo.fromDataMapWrapper(fgDataMapExprWrapper),
+              IndexWrapperSimpleInfo.fromDataMapWrapper(fgIndexExprWrapper),
               prunedBlocklets.size(), getBlockCount(prunedBlocklets));
         }
       }
@@ -648,7 +648,7 @@ m filterExpression
       List<ExtendedBlocklet> previousDataMapPrunedBlocklets,
       List<ExtendedBlocklet> otherDataMapPrunedBlocklets) {
     List<ExtendedBlocklet> prunedBlocklets = null;
-    if (BlockletDataMapUtil.isCacheLevelBlock(carbonTable)) {
+    if (BlockletIndexUtil.isCacheLevelBlock(carbonTable)) {
       prunedBlocklets = new ArrayList<>();
       for (ExtendedBlocklet otherBlocklet : otherDataMapPrunedBlocklets) {
         if (previousDataMapPrunedBlocklets.contains(otherBlocklet)) {
@@ -690,7 +690,7 @@ m filterExpression
   }
 
   public QueryModel createQueryModel(InputSplit inputSplit, TaskAttemptContext taskAttemptContext,
-      DataMapFilter dataMapFilter) throws IOException {
+      IndexFilter indexFilter) throws IOException {
     Configuration configuration = taskAttemptContext.getConfiguration();
     CarbonTable carbonTable = getOrCreateCarbonTable(configuration);
 
@@ -702,12 +702,12 @@ m filterExpression
     } else {
       projectColumns = new String[]{};
     }
-    if (dataMapFilter != null) {
-      checkAndAddImplicitExpression(dataMapFilter.getExpression(), inputSplit);
+    if (indexFilter != null) {
+      checkAndAddImplicitExpression(indexFilter.getExpression(), inputSplit);
     }
     QueryModel queryModel = new QueryModelBuilder(carbonTable)
         .projectColumns(projectColumns)
-        .filterExpression(dataMapFilter)
+        .filterExpression(indexFilter)
         .dataConverter(getDataTypeConverter(configuration))
         .build();
     String readDeltaOnly = configuration.get(READ_ONLY_DELTA);
diff --git a/hadoop/src/main/java/org/apache/carbondata/hadoop/api/CarbonOutputCommitter.java b/hadoop/src/main/java/org/apache/carbondata/hadoop/api/CarbonOutputCommitter.java
index b6c935b..98dbb51 100644
--- a/hadoop/src/main/java/org/apache/carbondata/hadoop/api/CarbonOutputCommitter.java
+++ b/hadoop/src/main/java/org/apache/carbondata/hadoop/api/CarbonOutputCommitter.java
@@ -236,7 +236,7 @@ public class CarbonOutputCommitter extends FileOutputCommitter {
     List<Segment> segmentDeleteList = Segment.toSegmentList(segmentsToBeDeleted.split(","), null);
     Set<Segment> segmentSet = new HashSet<>(
         new SegmentStatusManager(carbonTable.getAbsoluteTableIdentifier(),
-            context.getConfiguration()).getValidAndInvalidSegments(carbonTable.isChildTableForMV())
+            context.getConfiguration()).getValidAndInvalidSegments(carbonTable.isMV())
             .getValidSegments());
     if (updateTime != null) {
       CarbonUpdateUtil.updateTableMetadataStatus(segmentSet, carbonTable, updateTime, true,
@@ -264,7 +264,7 @@ public class CarbonOutputCommitter extends FileOutputCommitter {
     OperationContext operationContext = (OperationContext) getOperationContext();
     CarbonTable carbonTable = loadModel.getCarbonDataLoadSchema().getCarbonTable();
     String uuid = "";
-    if (loadModel.getCarbonDataLoadSchema().getCarbonTable().isChildTableForMV()
+    if (loadModel.getCarbonDataLoadSchema().getCarbonTable().isMV()
         && operationContext != null) {
       uuid = operationContext.getProperty("uuid").toString();
     }
@@ -322,7 +322,7 @@ public class CarbonOutputCommitter extends FileOutputCommitter {
     if (partitionSpecs != null && partitionSpecs.size() > 0) {
       List<Segment> validSegments =
           new SegmentStatusManager(table.getAbsoluteTableIdentifier())
-              .getValidAndInvalidSegments(table.isChildTableForMV()).getValidSegments();
+              .getValidAndInvalidSegments(table.isMV()).getValidSegments();
       String uniqueId = String.valueOf(System.currentTimeMillis());
       List<String> tobeUpdatedSegs = new ArrayList<>();
       List<String> tobeDeletedSegs = new ArrayList<>();
diff --git a/hadoop/src/main/java/org/apache/carbondata/hadoop/api/CarbonTableInputFormat.java b/hadoop/src/main/java/org/apache/carbondata/hadoop/api/CarbonTableInputFormat.java
index 845eca3..bdda845 100644
--- a/hadoop/src/main/java/org/apache/carbondata/hadoop/api/CarbonTableInputFormat.java
+++ b/hadoop/src/main/java/org/apache/carbondata/hadoop/api/CarbonTableInputFormat.java
@@ -31,13 +31,13 @@ import org.apache.carbondata.common.exceptions.DeprecatedFeatureException;
 import org.apache.carbondata.common.logging.LogServiceFactory;
 import org.apache.carbondata.core.constants.CarbonCommonConstants;
 import org.apache.carbondata.core.constants.CarbonCommonConstantsInternal;
-import org.apache.carbondata.core.datamap.DataMapChooser;
-import org.apache.carbondata.core.datamap.DataMapFilter;
 import org.apache.carbondata.core.datamap.DataMapStoreManager;
-import org.apache.carbondata.core.datamap.DataMapUtil;
+import org.apache.carbondata.core.datamap.IndexChooser;
+import org.apache.carbondata.core.datamap.IndexFilter;
+import org.apache.carbondata.core.datamap.IndexUtil;
 import org.apache.carbondata.core.datamap.Segment;
-import org.apache.carbondata.core.datamap.TableDataMap;
-import org.apache.carbondata.core.datamap.dev.expr.DataMapExprWrapper;
+import org.apache.carbondata.core.datamap.TableIndex;
+import org.apache.carbondata.core.datamap.dev.expr.IndexExprWrapper;
 import org.apache.carbondata.core.datastore.impl.FileFactory;
 import org.apache.carbondata.core.indexstore.ExtendedBlocklet;
 import org.apache.carbondata.core.indexstore.PartitionSpec;
@@ -150,7 +150,7 @@ public class CarbonTableInputFormat<T> extends CarbonInputFormat<T> {
         new SegmentStatusManager(carbonTable.getAbsoluteTableIdentifier(),
             readCommittedScope.getConfiguration());
     SegmentStatusManager.ValidAndInvalidSegmentsInfo segments = segmentStatusManager
-        .getValidAndInvalidSegments(carbonTable.isChildTableForMV(), loadMetadataDetails,
+        .getValidAndInvalidSegments(carbonTable.isMV(), loadMetadataDetails,
             this.readCommittedScope);
 
     if (getValidateSegmentsToAccess(job.getConfiguration())) {
@@ -190,15 +190,15 @@ public class CarbonTableInputFormat<T> extends CarbonInputFormat<T> {
         getFilteredSegment(job, validAndInProgressSegments, false, readCommittedScope);
 
     // process and resolve the expression
-    DataMapFilter dataMapFilter = getFilterPredicates(job.getConfiguration());
+    IndexFilter indexFilter = getFilterPredicates(job.getConfiguration());
 
-    if (dataMapFilter != null) {
-      dataMapFilter.resolve(false);
+    if (indexFilter != null) {
+      indexFilter.resolve(false);
     }
 
     // do block filtering and get split
     List<InputSplit> batchSplits = getSplits(
-        job, dataMapFilter, segmentToAccess,
+        job, indexFilter, segmentToAccess,
         updateStatusManager, segments.getInvalidSegments());
     splits.addAll(batchSplits);
 
@@ -287,7 +287,7 @@ public class CarbonTableInputFormat<T> extends CarbonInputFormat<T> {
       long maxSize = getMaxSplitSize(job);
       if (filterResolverIntf == null) {
         if (carbonTable != null) {
-          DataMapFilter filter = getFilterPredicates(job.getConfiguration());
+          IndexFilter filter = getFilterPredicates(job.getConfiguration());
           if (filter != null) {
             filter.processFilterExpression();
             filterResolverIntf = filter.getResolver();
@@ -346,7 +346,7 @@ public class CarbonTableInputFormat<T> extends CarbonInputFormat<T> {
    * @return
    * @throws IOException
    */
-  private List<InputSplit> getSplits(JobContext job, DataMapFilter expression,
+  private List<InputSplit> getSplits(JobContext job, IndexFilter expression,
       List<Segment> validSegments, SegmentUpdateStatusManager updateStatusManager,
       List<Segment> invalidSegments) throws IOException {
 
@@ -438,7 +438,7 @@ public class CarbonTableInputFormat<T> extends CarbonInputFormat<T> {
         table, loadMetadataDetails);
     SegmentStatusManager.ValidAndInvalidSegmentsInfo allSegments =
         new SegmentStatusManager(identifier, readCommittedScope.getConfiguration())
-            .getValidAndInvalidSegments(table.isChildTableForMV(), loadMetadataDetails,
+            .getValidAndInvalidSegments(table.isMV(), loadMetadataDetails,
                 readCommittedScope);
     Map<String, Long> blockRowCountMapping = new HashMap<>();
     Map<String, Long> segmentAndBlockCountMapping = new HashMap<>();
@@ -471,9 +471,9 @@ public class CarbonTableInputFormat<T> extends CarbonInputFormat<T> {
           .clearInvalidSegments(getOrCreateCarbonTable(job.getConfiguration()),
               toBeCleanedSegments);
     }
-    DataMapExprWrapper dataMapExprWrapper =
-        DataMapChooser.getDefaultDataMap(getOrCreateCarbonTable(job.getConfiguration()), null);
-    DataMapUtil.loadDataMaps(table, dataMapExprWrapper, filteredSegment, partitions);
+    IndexExprWrapper indexExprWrapper =
+        IndexChooser.getDefaultDataMap(getOrCreateCarbonTable(job.getConfiguration()), null);
+    IndexUtil.loadDataMaps(table, indexExprWrapper, filteredSegment, partitions);
     if (isIUDTable || isUpdateFlow) {
       Map<String, Long> blockletToRowCountMap = new HashMap<>();
       if (CarbonProperties.getInstance()
@@ -494,12 +494,12 @@ public class CarbonTableInputFormat<T> extends CarbonInputFormat<T> {
           if (CarbonProperties.getInstance().isFallBackDisabled()) {
             throw e;
           }
-          TableDataMap defaultDataMap = DataMapStoreManager.getInstance().getDefaultDataMap(table);
+          TableIndex defaultDataMap = DataMapStoreManager.getInstance().getDefaultIndex(table);
           blockletToRowCountMap
               .putAll(defaultDataMap.getBlockRowCount(filteredSegment, partitions, defaultDataMap));
         }
       } else {
-        TableDataMap defaultDataMap = DataMapStoreManager.getInstance().getDefaultDataMap(table);
+        TableIndex defaultDataMap = DataMapStoreManager.getInstance().getDefaultIndex(table);
         blockletToRowCountMap
             .putAll(defaultDataMap.getBlockRowCount(filteredSegment, partitions, defaultDataMap));
       }
@@ -537,7 +537,7 @@ public class CarbonTableInputFormat<T> extends CarbonInputFormat<T> {
         totalRowCount =
             getDistributedCount(table, partitions, filteredSegment);
       } else {
-        TableDataMap defaultDataMap = DataMapStoreManager.getInstance().getDefaultDataMap(table);
+        TableIndex defaultDataMap = DataMapStoreManager.getInstance().getDefaultIndex(table);
         totalRowCount = defaultDataMap.getRowCount(filteredSegment, partitions, defaultDataMap);
       }
       blockRowCountMapping.put(CarbonCommonConstantsInternal.ROW_COUNT, totalRowCount);
diff --git a/hadoop/src/main/java/org/apache/carbondata/hadoop/stream/StreamRecordReader.java b/hadoop/src/main/java/org/apache/carbondata/hadoop/stream/StreamRecordReader.java
index dcdee41..52b10dc 100644
--- a/hadoop/src/main/java/org/apache/carbondata/hadoop/stream/StreamRecordReader.java
+++ b/hadoop/src/main/java/org/apache/carbondata/hadoop/stream/StreamRecordReader.java
@@ -203,7 +203,7 @@ public class StreamRecordReader extends RecordReader<Void, Object> {
     }
 
     // initialize filter
-    if (null != model.getDataMapFilter()) {
+    if (null != model.getIndexFilter()) {
       initializeFilter();
     } else if (projection.length == 0) {
       skipScanData = true;
@@ -218,7 +218,7 @@ public class StreamRecordReader extends RecordReader<Void, Object> {
     SegmentProperties segmentProperties = new SegmentProperties(wrapperColumnSchemaList);
     Map<Integer, GenericQueryType> complexDimensionInfoMap = new HashMap<>();
 
-    FilterResolverIntf resolverIntf = model.getDataMapFilter().getResolver();
+    FilterResolverIntf resolverIntf = model.getIndexFilter().getResolver();
     filter = FilterUtil.getFilterExecuterTree(
         resolverIntf, segmentProperties, complexDimensionInfoMap, true);
     // for row filter, we need update column index
diff --git a/hadoop/src/main/java/org/apache/carbondata/hadoop/testutil/StoreCreator.java b/hadoop/src/main/java/org/apache/carbondata/hadoop/testutil/StoreCreator.java
index c6ba4ec..3337104 100644
--- a/hadoop/src/main/java/org/apache/carbondata/hadoop/testutil/StoreCreator.java
+++ b/hadoop/src/main/java/org/apache/carbondata/hadoop/testutil/StoreCreator.java
@@ -175,7 +175,7 @@ public class StoreCreator {
    * Method to clear the data maps
    */
   public void clearDataMaps() {
-    DataMapStoreManager.getInstance().clearDataMaps(absoluteTableIdentifier);
+    DataMapStoreManager.getInstance().clearIndex(absoluteTableIdentifier);
   }
 
   public CarbonLoadModel createTableAndLoadModel(boolean deleteOldStore) throws Exception {
diff --git a/hadoop/src/main/java/org/apache/carbondata/hadoop/util/CarbonInputFormatUtil.java b/hadoop/src/main/java/org/apache/carbondata/hadoop/util/CarbonInputFormatUtil.java
index c20647d..de01509 100644
--- a/hadoop/src/main/java/org/apache/carbondata/hadoop/util/CarbonInputFormatUtil.java
+++ b/hadoop/src/main/java/org/apache/carbondata/hadoop/util/CarbonInputFormatUtil.java
@@ -22,7 +22,7 @@ import java.text.SimpleDateFormat;
 import java.util.Locale;
 
 import org.apache.carbondata.common.logging.LogServiceFactory;
-import org.apache.carbondata.core.datamap.DataMapUtil;
+import org.apache.carbondata.core.datamap.IndexUtil;
 import org.apache.carbondata.core.metadata.AbsoluteTableIdentifier;
 import org.apache.carbondata.core.util.CarbonProperties;
 import org.apache.carbondata.hadoop.api.CarbonTableInputFormat;
@@ -59,14 +59,14 @@ public class CarbonInputFormatUtil {
   }
 
   /**
-   * This method set DataMapJob if configured
+   * This method set IndexJob if configured
    *
    * @param conf
    * @throws IOException
    */
   public static void setDataMapJobIfConfigured(Configuration conf) throws IOException {
-    String className = "org.apache.carbondata.indexserver.EmbeddedDataMapJob";
-    DataMapUtil.setDataMapJob(conf, DataMapUtil.createDataMapJob(className));
+    String className = "org.apache.carbondata.indexserver.EmbeddedIndexJob";
+    IndexUtil.setDataMapJob(conf, IndexUtil.createDataMapJob(className));
   }
 
   public static String createJobTrackerID(java.util.Date date) {
diff --git a/hadoop/src/test/java/org/apache/carbondata/hadoop/ft/CarbonTableInputFormatTest.java b/hadoop/src/test/java/org/apache/carbondata/hadoop/ft/CarbonTableInputFormatTest.java
index 79d447a..6fac83a 100644
--- a/hadoop/src/test/java/org/apache/carbondata/hadoop/ft/CarbonTableInputFormatTest.java
+++ b/hadoop/src/test/java/org/apache/carbondata/hadoop/ft/CarbonTableInputFormatTest.java
@@ -28,7 +28,7 @@ import java.util.List;
 import java.util.UUID;
 
 import org.apache.carbondata.core.constants.CarbonCommonConstants;
-import org.apache.carbondata.core.datamap.DataMapFilter;
+import org.apache.carbondata.core.datamap.IndexFilter;
 import org.apache.carbondata.core.datastore.impl.FileFactory;
 import org.apache.carbondata.core.metadata.AbsoluteTableIdentifier;
 import org.apache.carbondata.core.metadata.datatype.DataTypes;
@@ -91,7 +91,7 @@ public class CarbonTableInputFormatTest {
     Expression expression = new EqualToExpression(new ColumnExpression("country", DataTypes.STRING),
         new LiteralExpression("china", DataTypes.STRING));
     CarbonTableInputFormat.setFilterPredicates(job.getConfiguration(),
-        new DataMapFilter(loadModel.getCarbonDataLoadSchema().getCarbonTable(), expression));
+        new IndexFilter(loadModel.getCarbonDataLoadSchema().getCarbonTable(), expression));
     List splits = carbonInputFormat.getSplits(job);
 
     Assert.assertTrue(splits != null);
@@ -265,7 +265,7 @@ public class CarbonTableInputFormatTest {
     }
     if (filter != null) {
       CarbonTableInputFormat.setFilterPredicates(job.getConfiguration(),
-          new DataMapFilter(loadModel.getCarbonDataLoadSchema().getCarbonTable(), filter));
+          new IndexFilter(loadModel.getCarbonDataLoadSchema().getCarbonTable(), filter));
     }
     CarbonTableInputFormat.setDatabaseName(job.getConfiguration(),
         abs.getCarbonTableIdentifier().getDatabaseName());
diff --git a/index/bloom/src/main/java/org/apache/carbondata/datamap/bloom/AbstractBloomDataMapWriter.java b/index/bloom/src/main/java/org/apache/carbondata/datamap/bloom/AbstractBloomIndexWriter.java
similarity index 95%
rename from index/bloom/src/main/java/org/apache/carbondata/datamap/bloom/AbstractBloomDataMapWriter.java
rename to index/bloom/src/main/java/org/apache/carbondata/datamap/bloom/AbstractBloomIndexWriter.java
index 289f43d..ea61c7a 100644
--- a/index/bloom/src/main/java/org/apache/carbondata/datamap/bloom/AbstractBloomDataMapWriter.java
+++ b/index/bloom/src/main/java/org/apache/carbondata/datamap/bloom/AbstractBloomIndexWriter.java
@@ -25,7 +25,7 @@ import java.util.List;
 import org.apache.carbondata.common.annotations.InterfaceAudience;
 import org.apache.carbondata.core.constants.CarbonCommonConstants;
 import org.apache.carbondata.core.datamap.Segment;
-import org.apache.carbondata.core.datamap.dev.DataMapWriter;
+import org.apache.carbondata.core.datamap.dev.IndexWriter;
 import org.apache.carbondata.core.datastore.block.SegmentProperties;
 import org.apache.carbondata.core.datastore.impl.FileFactory;
 import org.apache.carbondata.core.datastore.page.ColumnPage;
@@ -39,7 +39,7 @@ import org.apache.hadoop.util.bloom.Key;
 import org.apache.hadoop.util.hash.Hash;
 
 @InterfaceAudience.Internal
-public abstract class AbstractBloomDataMapWriter extends DataMapWriter {
+public abstract class AbstractBloomIndexWriter extends IndexWriter {
   private int bloomFilterSize;
   private double bloomFilterFpp;
   private boolean compressBloom;
@@ -48,7 +48,7 @@ public abstract class AbstractBloomDataMapWriter extends DataMapWriter {
   private List<DataOutputStream> currentDataOutStreams;
   protected List<CarbonBloomFilter> indexBloomFilters;
 
-  AbstractBloomDataMapWriter(String tablePath, String dataMapName, List<CarbonColumn> indexColumns,
+  AbstractBloomIndexWriter(String tablePath, String dataMapName, List<CarbonColumn> indexColumns,
       Segment segment, String shardName, SegmentProperties segmentProperties,
       int bloomFilterSize, double bloomFilterFpp, boolean compressBloom)
       throws IOException {
@@ -152,7 +152,8 @@ public abstract class AbstractBloomDataMapWriter extends DataMapWriter {
 
   private void initDataMapFile() throws IOException {
     if (!FileFactory.isFileExist(dataMapPath)) {
-      if (!FileFactory.mkdirs(dataMapPath)) {
+      FileFactory.touchDirectory(FileFactory.getCarbonFile(dataMapPath));
+      if (!FileFactory.isFileExist(dataMapPath)) {
         throw new IOException("Failed to create directory " + dataMapPath);
       }
     }
diff --git a/index/bloom/src/main/java/org/apache/carbondata/datamap/bloom/BloomCoarseGrainDataMap.java b/index/bloom/src/main/java/org/apache/carbondata/datamap/bloom/BloomCoarseGrainIndex.java
similarity index 96%
rename from index/bloom/src/main/java/org/apache/carbondata/datamap/bloom/BloomCoarseGrainDataMap.java
rename to index/bloom/src/main/java/org/apache/carbondata/datamap/bloom/BloomCoarseGrainIndex.java
index 9dbc718..ec99bbe 100644
--- a/index/bloom/src/main/java/org/apache/carbondata/datamap/bloom/BloomCoarseGrainDataMap.java
+++ b/index/bloom/src/main/java/org/apache/carbondata/datamap/bloom/BloomCoarseGrainIndex.java
@@ -29,8 +29,8 @@ import org.apache.carbondata.common.annotations.InterfaceAudience;
 import org.apache.carbondata.common.logging.LogServiceFactory;
 import org.apache.carbondata.core.cache.Cache;
 import org.apache.carbondata.core.constants.CarbonCommonConstants;
-import org.apache.carbondata.core.datamap.dev.DataMapModel;
-import org.apache.carbondata.core.datamap.dev.cgdatamap.CoarseGrainDataMap;
+import org.apache.carbondata.core.datamap.dev.IndexModel;
+import org.apache.carbondata.core.datamap.dev.cgdatamap.CoarseGrainIndex;
 import org.apache.carbondata.core.datastore.block.SegmentProperties;
 import org.apache.carbondata.core.datastore.impl.FileFactory;
 import org.apache.carbondata.core.datastore.page.encoding.bool.BooleanConvert;
@@ -68,9 +68,9 @@ import org.apache.log4j.Logger;
  * More information of the index file can be found in the corresponding datamap writer.
  */
 @InterfaceAudience.Internal
-public class BloomCoarseGrainDataMap extends CoarseGrainDataMap {
+public class BloomCoarseGrainIndex extends CoarseGrainIndex {
   private static final Logger LOGGER =
-      LogServiceFactory.getLogService(BloomCoarseGrainDataMap.class.getName());
+      LogServiceFactory.getLogService(BloomCoarseGrainIndex.class.getName());
   private Map<String, CarbonColumn> name2Col;
   private Cache<BloomCacheKeyValue.CacheKey, BloomCacheKeyValue.CacheValue> cache;
   private String shardName;
@@ -84,11 +84,11 @@ public class BloomCoarseGrainDataMap extends CoarseGrainDataMap {
   private BadRecordLogHolder badRecordLogHolder;
 
   @Override
-  public void init(DataMapModel dataMapModel) {
-    this.indexPath = FileFactory.getPath(dataMapModel.getFilePath());
+  public void init(IndexModel indexModel) {
+    this.indexPath = FileFactory.getPath(indexModel.getFilePath());
     this.shardName = indexPath.getName();
-    if (dataMapModel instanceof BloomDataMapModel) {
-      BloomDataMapModel model = (BloomDataMapModel) dataMapModel;
+    if (indexModel instanceof BloomIndexModel) {
+      BloomIndexModel model = (BloomIndexModel) indexModel;
       this.cache = model.getCache();
     }
   }
@@ -173,14 +173,14 @@ public class BloomCoarseGrainDataMap extends CoarseGrainDataMap {
         }
         if (scanRequired) {
           if (LOGGER.isDebugEnabled()) {
-            LOGGER.debug(String.format("BloomCoarseGrainDataMap: Need to scan -> blocklet#%s",
+            LOGGER.debug(String.format("BloomCoarseGrainIndex: Need to scan -> blocklet#%s",
                 String.valueOf(bloomFilter.getBlockletNo())));
           }
           Blocklet blocklet = new Blocklet(bloomFilter.getShardName(),
               String.valueOf(bloomFilter.getBlockletNo()));
           tempHitBlockletsResult.add(blocklet);
         } else if (LOGGER.isDebugEnabled()) {
-          LOGGER.debug(String.format("BloomCoarseGrainDataMap: Skip scan -> blocklet#%s",
+          LOGGER.debug(String.format("BloomCoarseGrainIndex: Skip scan -> blocklet#%s",
               String.valueOf(bloomFilter.getBlockletNo())));
         }
         // get intersect result between query models
diff --git a/index/bloom/src/main/java/org/apache/carbondata/datamap/bloom/BloomCoarseGrainDataMapFactory.java b/index/bloom/src/main/java/org/apache/carbondata/datamap/bloom/BloomCoarseGrainIndexFactory.java
similarity index 76%
rename from index/bloom/src/main/java/org/apache/carbondata/datamap/bloom/BloomCoarseGrainDataMapFactory.java
rename to index/bloom/src/main/java/org/apache/carbondata/datamap/bloom/BloomCoarseGrainIndexFactory.java
index 750624b..5fe7f41 100644
--- a/index/bloom/src/main/java/org/apache/carbondata/datamap/bloom/BloomCoarseGrainDataMapFactory.java
+++ b/index/bloom/src/main/java/org/apache/carbondata/datamap/bloom/BloomCoarseGrainIndexFactory.java
@@ -28,20 +28,20 @@ import java.util.Set;
 import java.util.concurrent.ConcurrentHashMap;
 
 import org.apache.carbondata.common.annotations.InterfaceAudience;
-import org.apache.carbondata.common.exceptions.sql.MalformedDataMapCommandException;
+import org.apache.carbondata.common.exceptions.sql.MalformedIndexCommandException;
 import org.apache.carbondata.common.logging.LogServiceFactory;
 import org.apache.carbondata.core.cache.Cache;
 import org.apache.carbondata.core.cache.CacheProvider;
 import org.apache.carbondata.core.cache.CacheType;
 import org.apache.carbondata.core.constants.CarbonV3DataFormatConstants;
-import org.apache.carbondata.core.datamap.DataMapDistributable;
-import org.apache.carbondata.core.datamap.DataMapLevel;
-import org.apache.carbondata.core.datamap.DataMapMeta;
+import org.apache.carbondata.core.datamap.IndexInputSplit;
+import org.apache.carbondata.core.datamap.IndexLevel;
+import org.apache.carbondata.core.datamap.IndexMeta;
 import org.apache.carbondata.core.datamap.Segment;
-import org.apache.carbondata.core.datamap.dev.DataMapBuilder;
-import org.apache.carbondata.core.datamap.dev.DataMapFactory;
-import org.apache.carbondata.core.datamap.dev.DataMapWriter;
-import org.apache.carbondata.core.datamap.dev.cgdatamap.CoarseGrainDataMap;
+import org.apache.carbondata.core.datamap.dev.IndexBuilder;
+import org.apache.carbondata.core.datamap.dev.IndexFactory;
+import org.apache.carbondata.core.datamap.dev.IndexWriter;
+import org.apache.carbondata.core.datamap.dev.cgdatamap.CoarseGrainIndex;
 import org.apache.carbondata.core.datastore.block.SegmentProperties;
 import org.apache.carbondata.core.datastore.filesystem.CarbonFile;
 import org.apache.carbondata.core.datastore.impl.FileFactory;
@@ -63,9 +63,9 @@ import org.apache.log4j.Logger;
  * This class is for Bloom Filter for blocklet level
  */
 @InterfaceAudience.Internal
-public class BloomCoarseGrainDataMapFactory extends DataMapFactory<CoarseGrainDataMap> {
+public class BloomCoarseGrainIndexFactory extends IndexFactory<CoarseGrainIndex> {
   private static final Logger LOGGER = LogServiceFactory.getLogService(
-      BloomCoarseGrainDataMapFactory.class.getName());
+      BloomCoarseGrainIndexFactory.class.getName());
   /**
    * property for size of bloom filter
    */
@@ -93,7 +93,7 @@ public class BloomCoarseGrainDataMapFactory extends DataMapFactory<CoarseGrainDa
    */
   private static final boolean DEFAULT_BLOOM_COMPRESS = true;
 
-  private DataMapMeta dataMapMeta;
+  private IndexMeta indexMeta;
   private String dataMapName;
   private int bloomFilterSize;
   private double bloomFilterFpp;
@@ -102,8 +102,8 @@ public class BloomCoarseGrainDataMapFactory extends DataMapFactory<CoarseGrainDa
   // segmentId -> list of index file
   private Map<String, Set<String>> segmentMap = new ConcurrentHashMap<>();
 
-  public BloomCoarseGrainDataMapFactory(CarbonTable carbonTable, DataMapSchema dataMapSchema)
-      throws MalformedDataMapCommandException {
+  public BloomCoarseGrainIndexFactory(CarbonTable carbonTable, DataMapSchema dataMapSchema)
+      throws MalformedIndexCommandException {
     super(carbonTable, dataMapSchema);
     Objects.requireNonNull(carbonTable);
     Objects.requireNonNull(dataMapSchema);
@@ -118,25 +118,25 @@ public class BloomCoarseGrainDataMapFactory extends DataMapFactory<CoarseGrainDa
     // todo: support more optimize operations
     optimizedOperations.add(ExpressionType.EQUALS);
     optimizedOperations.add(ExpressionType.IN);
-    this.dataMapMeta = new DataMapMeta(this.dataMapName, indexedColumns, optimizedOperations);
-    LOGGER.info(String.format("DataMap %s works for %s with bloom size %d",
-        this.dataMapName, this.dataMapMeta, this.bloomFilterSize));
+    this.indexMeta = new IndexMeta(this.dataMapName, indexedColumns, optimizedOperations);
+    LOGGER.info(String.format("Index %s works for %s with bloom size %d",
+        this.dataMapName, this.indexMeta, this.bloomFilterSize));
     try {
       this.cache = CacheProvider.getInstance()
-          .createCache(new CacheType("bloom_cache"), BloomDataMapCache.class.getName());
+          .createCache(new CacheType("bloom_cache"), BloomIndexCache.class.getName());
     } catch (Exception e) {
       LOGGER.error(e.getMessage(), e);
-      throw new MalformedDataMapCommandException(e.getMessage());
+      throw new MalformedIndexCommandException(e.getMessage());
     }
   }
 
   /**
-   * validate Lucene DataMap BLOOM_SIZE
+   * validate Lucene Index BLOOM_SIZE
    * 1. BLOOM_SIZE property is optional, 32000 * 20 will be the default size.
    * 2. BLOOM_SIZE should be an integer that greater than 0
    */
   private int validateAndGetBloomFilterSize(DataMapSchema dmSchema)
-      throws MalformedDataMapCommandException {
+      throws MalformedIndexCommandException {
     String bloomFilterSizeStr = dmSchema.getProperties().get(BLOOM_SIZE);
     if (StringUtils.isBlank(bloomFilterSizeStr)) {
       LOGGER.warn(
@@ -148,13 +148,13 @@ public class BloomCoarseGrainDataMapFactory extends DataMapFactory<CoarseGrainDa
     try {
       bloomFilterSize = Integer.parseInt(bloomFilterSizeStr);
     } catch (NumberFormatException e) {
-      throw new MalformedDataMapCommandException(
+      throw new MalformedIndexCommandException(
           String.format("Invalid value of bloom filter size '%s', it should be an integer",
               bloomFilterSizeStr));
     }
     // todo: reconsider the boundaries of bloom filter size
     if (bloomFilterSize <= 0) {
-      throw new MalformedDataMapCommandException(
+      throw new MalformedIndexCommandException(
           String.format("Invalid value of bloom filter size '%s', it should be greater than 0",
               bloomFilterSizeStr));
     }
@@ -162,12 +162,12 @@ public class BloomCoarseGrainDataMapFactory extends DataMapFactory<CoarseGrainDa
   }
 
   /**
-   * validate bloom DataMap BLOOM_FPP
+   * validate bloom Index BLOOM_FPP
    * 1. BLOOM_FPP property is optional, 0.00001 will be the default value.
    * 2. BLOOM_FPP should be (0, 1)
    */
   private double validateAndGetBloomFilterFpp(DataMapSchema dmSchema)
-      throws MalformedDataMapCommandException {
+      throws MalformedIndexCommandException {
     String bloomFilterFppStr = dmSchema.getProperties().get(BLOOM_FPP);
     if (StringUtils.isBlank(bloomFilterFppStr)) {
       LOGGER.warn(
@@ -179,12 +179,12 @@ public class BloomCoarseGrainDataMapFactory extends DataMapFactory<CoarseGrainDa
     try {
       bloomFilterFpp = Double.parseDouble(bloomFilterFppStr);
     } catch (NumberFormatException e) {
-      throw new MalformedDataMapCommandException(
+      throw new MalformedIndexCommandException(
           String.format("Invalid value of bloom filter fpp '%s', it should be an numeric",
               bloomFilterFppStr));
     }
     if (bloomFilterFpp < 0 || bloomFilterFpp - 1 >= 0) {
-      throw new MalformedDataMapCommandException(
+      throw new MalformedIndexCommandException(
           String.format("Invalid value of bloom filter fpp '%s', it should be in range 0~1",
               bloomFilterFppStr));
     }
@@ -192,7 +192,7 @@ public class BloomCoarseGrainDataMapFactory extends DataMapFactory<CoarseGrainDa
   }
 
   /**
-   * validate bloom DataMap COMPRESS_BLOOM
+   * validate bloom Index COMPRESS_BLOOM
    * Default value is true
    */
   private boolean validateAndGetBloomCompress(DataMapSchema dmSchema) {
@@ -207,21 +207,21 @@ public class BloomCoarseGrainDataMapFactory extends DataMapFactory<CoarseGrainDa
   }
 
   @Override
-  public DataMapWriter createWriter(Segment segment, String shardName,
+  public IndexWriter createWriter(Segment segment, String shardName,
       SegmentProperties segmentProperties) throws IOException {
     LOGGER.info(
         String.format("Data of BloomCoarseGranDataMap %s for table %s will be written to %s",
             this.dataMapName, getCarbonTable().getTableName() , shardName));
-    return new BloomDataMapWriter(getCarbonTable().getTablePath(), this.dataMapName,
-        this.dataMapMeta.getIndexedColumns(), segment, shardName, segmentProperties,
+    return new BloomIndexWriter(getCarbonTable().getTablePath(), this.dataMapName,
+        this.indexMeta.getIndexedColumns(), segment, shardName, segmentProperties,
         this.bloomFilterSize, this.bloomFilterFpp, bloomCompress);
   }
 
   @Override
-  public DataMapBuilder createBuilder(Segment segment, String shardName,
+  public IndexBuilder createBuilder(Segment segment, String shardName,
       SegmentProperties segmentProperties) throws IOException {
-    return new BloomDataMapBuilder(getCarbonTable().getTablePath(), this.dataMapName,
-        this.dataMapMeta.getIndexedColumns(), segment, shardName, segmentProperties,
+    return new BloomIndexBuilder(getCarbonTable().getTablePath(), this.dataMapName,
+        this.indexMeta.getIndexedColumns(), segment, shardName, segmentProperties,
         this.bloomFilterSize, this.bloomFilterFpp, bloomCompress);
   }
 
@@ -255,8 +255,8 @@ public class BloomCoarseGrainDataMapFactory extends DataMapFactory<CoarseGrainDa
   }
 
   @Override
-  public List<CoarseGrainDataMap> getDataMaps(Segment segment) throws IOException {
-    List<CoarseGrainDataMap> dataMaps = new ArrayList<>();
+  public List<CoarseGrainIndex> getIndexes(Segment segment) throws IOException {
+    List<CoarseGrainIndex> dataMaps = new ArrayList<>();
     try {
       Set<String> shardPaths = segmentMap.get(segment.getSegmentNo());
       if (shardPaths == null) {
@@ -270,41 +270,41 @@ public class BloomCoarseGrainDataMapFactory extends DataMapFactory<CoarseGrainDa
             filteredShards.contains(new File(shard).getName())) {
           // Filter out the tasks which are filtered through Main datamap.
           // for merge shard, shard pruning delay to be done before pruning blocklet
-          BloomCoarseGrainDataMap bloomDM = new BloomCoarseGrainDataMap();
-          bloomDM.init(new BloomDataMapModel(shard, cache, segment.getConfiguration()));
-          bloomDM.initIndexColumnConverters(getCarbonTable(), dataMapMeta.getIndexedColumns());
+          BloomCoarseGrainIndex bloomDM = new BloomCoarseGrainIndex();
+          bloomDM.init(new BloomIndexModel(shard, cache, segment.getConfiguration()));
+          bloomDM.initIndexColumnConverters(getCarbonTable(), indexMeta.getIndexedColumns());
           bloomDM.setFilteredShard(filteredShards);
           dataMaps.add(bloomDM);
         }
       }
     } catch (Exception e) {
-      throw new IOException("Error occurs while init Bloom DataMap", e);
+      throw new IOException("Error occurs while init Bloom Index", e);
     }
     return dataMaps;
   }
 
   @Override
-  public List<CoarseGrainDataMap> getDataMaps(Segment segment, List<PartitionSpec> partitionSpecs)
+  public List<CoarseGrainIndex> getIndexes(Segment segment, List<PartitionSpec> partitionSpecs)
       throws IOException {
-    return getDataMaps(segment);
+    return getIndexes(segment);
   }
 
   @Override
-  public List<CoarseGrainDataMap> getDataMaps(DataMapDistributable distributable) {
-    List<CoarseGrainDataMap> dataMaps = new ArrayList<>();
-    String indexPath = ((BloomDataMapDistributable) distributable).getIndexPath();
-    Set<String> filteredShards = ((BloomDataMapDistributable) distributable).getFilteredShards();
-    BloomCoarseGrainDataMap bloomDM = new BloomCoarseGrainDataMap();
-    bloomDM.init(new BloomDataMapModel(indexPath, cache, FileFactory.getConfiguration()));
-    bloomDM.initIndexColumnConverters(getCarbonTable(), dataMapMeta.getIndexedColumns());
+  public List<CoarseGrainIndex> getIndexes(IndexInputSplit distributable) {
+    List<CoarseGrainIndex> dataMaps = new ArrayList<>();
+    String indexPath = ((BloomIndexInputSplit) distributable).getIndexPath();
+    Set<String> filteredShards = ((BloomIndexInputSplit) distributable).getFilteredShards();
+    BloomCoarseGrainIndex bloomDM = new BloomCoarseGrainIndex();
+    bloomDM.init(new BloomIndexModel(indexPath, cache, FileFactory.getConfiguration()));
+    bloomDM.initIndexColumnConverters(getCarbonTable(), indexMeta.getIndexedColumns());
     bloomDM.setFilteredShard(filteredShards);
     dataMaps.add(bloomDM);
     return dataMaps;
   }
 
   @Override
-  public List<DataMapDistributable> toDistributable(Segment segment) {
-    List<DataMapDistributable> dataMapDistributableList = new ArrayList<>();
+  public List<IndexInputSplit> toDistributable(Segment segment) {
+    List<IndexInputSplit> indexInputSplitList = new ArrayList<>();
     Set<String> shardPaths = segmentMap.get(segment.getSegmentNo());
     if (shardPaths == null) {
       shardPaths =
@@ -317,14 +317,14 @@ public class BloomCoarseGrainDataMapFactory extends DataMapFactory<CoarseGrainDa
       // for merge shard, shard pruning delay to be done before pruning blocklet
... 22530 lines suppressed ...