You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@carbondata.apache.org by ra...@apache.org on 2018/03/08 16:55:43 UTC

[42/54] [abbrv] carbondata git commit: [CARBONDATA-2189] Add DataMapProvider developer interface

[CARBONDATA-2189] Add DataMapProvider developer interface

Add developer interface for 2 types of DataMap:

1.IndexDataMap: DataMap that leveraging index to accelerate filter query
2.MVDataMap: DataMap that leveraging Materialized View to accelerate olap style query, like SPJG query (select, predicate, join, groupby)
This PR adds support for following logic when creating and dropping the DataMap

This closes #1987


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

Branch: refs/heads/master
Commit: 89a12af5aba17f12c4e695971982abfeff256fc1
Parents: 56330ae
Author: Jacky Li <ja...@qq.com>
Authored: Thu Feb 22 20:59:59 2018 +0800
Committer: ravipesala <ra...@gmail.com>
Committed: Thu Mar 8 22:21:11 2018 +0530

----------------------------------------------------------------------
 .../sql/MalformedDataMapCommandException.java   |   4 +
 .../carbondata/core/datamap/DataMapChooser.java |   5 +-
 .../core/datamap/DataMapRegistry.java           |  37 +
 .../core/datamap/DataMapStoreManager.java       |  82 +-
 .../carbondata/core/datamap/TableDataMap.java   |  42 +-
 .../core/datamap/dev/AbstractDataMapWriter.java |   2 +-
 .../carbondata/core/datamap/dev/DataMap.java    |  56 --
 .../core/datamap/dev/DataMapFactory.java        |  86 --
 .../core/datamap/dev/IndexDataMap.java          |  56 ++
 .../core/datamap/dev/IndexDataMapFactory.java   |  86 ++
 .../cgdatamap/AbstractCoarseGrainDataMap.java   |  24 -
 .../AbstractCoarseGrainDataMapFactory.java      |  34 -
 .../AbstractCoarseGrainIndexDataMap.java        |  24 +
 .../AbstractCoarseGrainIndexDataMapFactory.java |  34 +
 .../dev/expr/DataMapExprWrapperImpl.java        |   2 +-
 .../dev/fgdatamap/AbstractFineGrainDataMap.java |  24 -
 .../AbstractFineGrainDataMapFactory.java        |  38 -
 .../AbstractFineGrainIndexDataMap.java          |  24 +
 .../AbstractFineGrainIndexDataMapFactory.java   |  38 +
 .../indexstore/BlockletDataMapIndexStore.java   |  33 +-
 .../blockletindex/BlockletDataMap.java          | 971 -------------------
 .../blockletindex/BlockletDataMapFactory.java   | 285 ------
 .../blockletindex/BlockletDataMapModel.java     |   2 +-
 .../blockletindex/BlockletIndexDataMap.java     | 971 +++++++++++++++++++
 .../BlockletIndexDataMapFactory.java            | 285 ++++++
 .../core/metadata/schema/table/CarbonTable.java |  10 +-
 .../metadata/schema/table/DataMapSchema.java    |   1 +
 .../schema/table/DataMapSchemaFactory.java      |  13 +-
 .../core/metadata/schema/table/TableSchema.java |   3 +-
 .../blockletindex/TestBlockletDataMap.java      |  66 --
 .../blockletindex/TestBlockletIndexDataMap.java |  59 ++
 .../datamap/examples/MinMaxDataMap.java         | 152 ---
 .../datamap/examples/MinMaxDataMapFactory.java  | 117 ---
 .../datamap/examples/MinMaxIndexDataMap.java    | 150 +++
 .../examples/MinMaxIndexDataMapFactory.java     | 117 +++
 .../MinMaxDataMapExample.scala                  |   4 +-
 docs/datamap-developer-guide.md                 |  16 +
 .../hadoop/api/CarbonTableInputFormat.java      |   5 +-
 .../preaggregate/TestPreAggCreateCommand.scala  |   2 +-
 .../TestPreAggregateTableSelection.scala        |   4 +-
 .../timeseries/TestTimeSeriesCreateTable.scala  |   4 +-
 .../testsuite/datamap/CGDataMapTestCase.scala   | 381 --------
 .../datamap/CGIndexDataMapTestCase.scala        | 383 ++++++++
 .../testsuite/datamap/DataMapWriterSuite.scala  | 216 -----
 .../testsuite/datamap/FGDataMapTestCase.scala   | 473 ---------
 .../datamap/FGIndexDataMapTestCase.scala        | 472 +++++++++
 .../datamap/IndexDataMapWriterSuite.scala       | 217 +++++
 .../testsuite/datamap/TestDataMapCommand.scala  | 288 ------
 .../datamap/TestIndexDataMapCommand.scala       | 285 ++++++
 .../carbondata/datamap/DataMapManager.java      |  53 +
 .../carbondata/datamap/DataMapProperty.java     |  32 +
 .../carbondata/datamap/DataMapProvider.java     | 105 ++
 .../datamap/IndexDataMapProvider.java           | 116 +++
 .../datamap/PreAggregateDataMapProvider.java    |  92 ++
 .../datamap/TimeseriesDataMapProvider.java      |  50 +
 .../scala/org/apache/spark/sql/CarbonEnv.scala  |   2 +-
 .../datamap/CarbonCreateDataMapCommand.scala    |  92 +-
 .../datamap/CarbonDropDataMapCommand.scala      |  73 +-
 .../CarbonAlterTableDropPartitionCommand.scala  |   2 +-
 .../CarbonAlterTableSplitPartitionCommand.scala |   2 +-
 .../CreatePreAggregateTableCommand.scala        | 203 ----
 .../preaaggregate/PreAggregateTableHelper.scala | 195 ++++
 .../preaaggregate/PreAggregateUtil.scala        |  24 +-
 .../CarbonAlterTableAddColumnCommand.scala      |   2 +-
 .../CarbonAlterTableDataTypeChangeCommand.scala |   2 +-
 .../CarbonAlterTableDropColumnCommand.scala     |   2 +-
 .../schema/CarbonAlterTableRenameCommand.scala  |   2 +-
 .../command/timeseries/TimeSeriesUtil.scala     |  16 +-
 .../spark/sql/hive/CarbonFileMetastore.scala    |   3 +-
 .../spark/sql/hive/CarbonHiveMetaStore.scala    |   3 +-
 .../apache/spark/sql/hive/CarbonMetaStore.scala |   5 +-
 .../org/apache/spark/util/AlterTableUtil.scala  |  10 +-
 .../datamap/DataMapWriterListener.java          |   8 +-
 73 files changed, 4094 insertions(+), 3683 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/carbondata/blob/89a12af5/common/src/main/java/org/apache/carbondata/common/exceptions/sql/MalformedDataMapCommandException.java
----------------------------------------------------------------------
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/MalformedDataMapCommandException.java
index 7c25b2c..83cae7c 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/MalformedDataMapCommandException.java
@@ -34,4 +34,8 @@ public class MalformedDataMapCommandException extends MalformedCarbonCommandExce
   public MalformedDataMapCommandException(String msg) {
     super(msg);
   }
+
+  public MalformedDataMapCommandException(String msg, Throwable e) {
+    super(msg, e);
+  }
 }

http://git-wip-us.apache.org/repos/asf/carbondata/blob/89a12af5/core/src/main/java/org/apache/carbondata/core/datamap/DataMapChooser.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/carbondata/core/datamap/DataMapChooser.java b/core/src/main/java/org/apache/carbondata/core/datamap/DataMapChooser.java
index 5155009..41e9b56 100644
--- a/core/src/main/java/org/apache/carbondata/core/datamap/DataMapChooser.java
+++ b/core/src/main/java/org/apache/carbondata/core/datamap/DataMapChooser.java
@@ -212,9 +212,10 @@ public class DataMapChooser {
       List<ColumnExpression> columnExpressions, Set<ExpressionType> expressionTypes) {
     List<DataMapTuple> tuples = new ArrayList<>();
     for (TableDataMap dataMap : allDataMap) {
-      if (contains(dataMap.getDataMapFactory().getMeta(), columnExpressions, expressionTypes)) {
+      if (contains(dataMap.getIndexDataMapFactory().getMeta(), columnExpressions, expressionTypes))
+      {
         tuples.add(
-            new DataMapTuple(dataMap.getDataMapFactory().getMeta().getIndexedColumns().size(),
+            new DataMapTuple(dataMap.getIndexDataMapFactory().getMeta().getIndexedColumns().size(),
                 dataMap));
       }
     }

http://git-wip-us.apache.org/repos/asf/carbondata/blob/89a12af5/core/src/main/java/org/apache/carbondata/core/datamap/DataMapRegistry.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/carbondata/core/datamap/DataMapRegistry.java b/core/src/main/java/org/apache/carbondata/core/datamap/DataMapRegistry.java
new file mode 100644
index 0000000..03c0c3e
--- /dev/null
+++ b/core/src/main/java/org/apache/carbondata/core/datamap/DataMapRegistry.java
@@ -0,0 +1,37 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.carbondata.core.datamap;
+
+import java.util.Map;
+import java.util.Objects;
+import java.util.concurrent.ConcurrentHashMap;
+
+public class DataMapRegistry {
+  private static Map<String, String> shortNameToClassName = new ConcurrentHashMap<>();
+
+  public static void registerDataMap(String datamapClassName, String shortName) {
+    Objects.requireNonNull(datamapClassName);
+    Objects.requireNonNull(shortName);
+    shortNameToClassName.put(shortName, datamapClassName);
+  }
+
+  public static String getDataMapClassName(String shortName) {
+    Objects.requireNonNull(shortName);
+    return shortNameToClassName.get(shortName);
+  }
+}

http://git-wip-us.apache.org/repos/asf/carbondata/blob/89a12af5/core/src/main/java/org/apache/carbondata/core/datamap/DataMapStoreManager.java
----------------------------------------------------------------------
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 cd89d74..ed4e18c 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
@@ -26,11 +26,10 @@ import org.apache.carbondata.common.exceptions.MetadataProcessException;
 import org.apache.carbondata.common.exceptions.sql.MalformedDataMapCommandException;
 import org.apache.carbondata.common.logging.LogService;
 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.IndexDataMapFactory;
 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.BlockletIndexDataMapFactory;
 import org.apache.carbondata.core.metadata.AbsoluteTableIdentifier;
 import org.apache.carbondata.core.metadata.schema.datamap.DataMapProvider;
 import org.apache.carbondata.core.metadata.schema.table.CarbonTable;
@@ -69,7 +68,7 @@ public final class DataMapStoreManager {
     List<TableDataMap> tableDataMaps = getAllDataMap(carbonTable);
     if (tableDataMaps != null) {
       for (TableDataMap dataMap : tableDataMaps) {
-        if (mapType == dataMap.getDataMapFactory().getDataMapType()) {
+        if (mapType == dataMap.getIndexDataMapFactory().getDataMapType()) {
           dataMaps.add(dataMap);
         }
       }
@@ -87,8 +86,8 @@ public final class DataMapStoreManager {
     List<TableDataMap> dataMaps = new ArrayList<>();
     if (dataMapSchemaList != null) {
       for (DataMapSchema dataMapSchema : dataMapSchemaList) {
-        if (!dataMapSchema.getClassName()
-            .equalsIgnoreCase(DataMapProvider.PREAGGREGATE.getClassName())) {
+        if (!dataMapSchema.getClassName().equalsIgnoreCase(
+            DataMapProvider.PREAGGREGATE.toString())) {
           dataMaps.add(getDataMap(carbonTable.getAbsoluteTableIdentifier(), dataMapSchema));
         }
       }
@@ -97,13 +96,13 @@ 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 BlockletIndexDataMap
    *
    * @param identifier
    * @return
    */
   public TableDataMap getDefaultDataMap(AbsoluteTableIdentifier identifier) {
-    return getDataMap(identifier, BlockletDataMapFactory.DATA_MAP_SCHEMA);
+    return getDataMap(identifier, BlockletIndexDataMapFactory.DATA_MAP_SCHEMA);
   }
 
   /**
@@ -142,9 +141,26 @@ 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 TableDataMap createAndRegisterDataMap(AbsoluteTableIdentifier identifier,
-      DataMapSchema dataMapSchema)
-      throws MalformedDataMapCommandException {
+  private TableDataMap createAndRegisterDataMap(AbsoluteTableIdentifier identifier,
+      DataMapSchema dataMapSchema) throws MalformedDataMapCommandException {
+    IndexDataMapFactory indexDataMapFactory;
+    try {
+      // try to create datamap by reflection to test whether it is a valid IndexDataMapFactory class
+      Class<? extends IndexDataMapFactory> factoryClass =
+          (Class<? extends IndexDataMapFactory>) Class.forName(dataMapSchema.getClassName());
+      indexDataMapFactory = factoryClass.newInstance();
+    } catch (ClassNotFoundException e) {
+      throw new MalformedDataMapCommandException(
+          "DataMap '" + dataMapSchema.getClassName() + "' not found");
+    } catch (Throwable e) {
+      throw new MetadataProcessException(
+          "failed to create DataMap '" + dataMapSchema.getClassName() + "'", e);
+    }
+    return registerDataMap(identifier, dataMapSchema, indexDataMapFactory);
+  }
+
+  public TableDataMap registerDataMap(AbsoluteTableIdentifier identifier,
+      DataMapSchema dataMapSchema,  IndexDataMapFactory indexDataMapFactory) {
     String table = identifier.getCarbonTableIdentifier().getTableUniqueName();
     // Just update the segmentRefreshMap with the table if not added.
     getTableSegmentRefresher(identifier);
@@ -152,37 +168,19 @@ public final class DataMapStoreManager {
     if (tableDataMaps == null) {
       tableDataMaps = new ArrayList<>();
     }
-    String dataMapName = dataMapSchema.getDataMapName();
-    TableDataMap dataMap = getTableDataMap(dataMapName, tableDataMaps);
-    if (dataMap != null && dataMap.getDataMapSchema().getDataMapName()
-        .equalsIgnoreCase(dataMapName)) {
-      throw new MalformedDataMapCommandException("Already datamap exists in that path with type " +
-          dataMapName);
-    }
 
-    try {
-      // try to create datamap by reflection to test whether it is a valid DataMapFactory class
-      Class<? extends DataMapFactory> factoryClass =
-          (Class<? extends DataMapFactory>) Class.forName(dataMapSchema.getClassName());
-      DataMapFactory dataMapFactory = factoryClass.newInstance();
-      dataMapFactory.init(identifier, dataMapSchema);
-      BlockletDetailsFetcher blockletDetailsFetcher;
-      SegmentPropertiesFetcher segmentPropertiesFetcher = null;
-      if (dataMapFactory instanceof BlockletDetailsFetcher) {
-        blockletDetailsFetcher = (BlockletDetailsFetcher) dataMapFactory;
-      } else {
-        blockletDetailsFetcher = getBlockletDetailsFetcher(identifier);
-      }
-      segmentPropertiesFetcher = (SegmentPropertiesFetcher) blockletDetailsFetcher;
-      dataMap = new TableDataMap(identifier, dataMapSchema, dataMapFactory, blockletDetailsFetcher,
-          segmentPropertiesFetcher);
-    } catch (ClassNotFoundException e) {
-      throw new MalformedDataMapCommandException("DataMap class '" +
-          dataMapSchema.getClassName() + "' not found");
-    } catch (Throwable e) {
-      throw new MetadataProcessException(
-          "failed to create DataMap instance for '" + dataMapSchema.getClassName() + "'", e);
+    indexDataMapFactory.init(identifier, dataMapSchema);
+    BlockletDetailsFetcher blockletDetailsFetcher;
+    SegmentPropertiesFetcher segmentPropertiesFetcher = null;
+    if (indexDataMapFactory instanceof BlockletDetailsFetcher) {
+      blockletDetailsFetcher = (BlockletDetailsFetcher) indexDataMapFactory;
+    } else {
+      blockletDetailsFetcher = getBlockletDetailsFetcher(identifier);
     }
+    segmentPropertiesFetcher = (SegmentPropertiesFetcher) blockletDetailsFetcher;
+    TableDataMap dataMap = new TableDataMap(identifier, dataMapSchema, indexDataMapFactory,
+        blockletDetailsFetcher, segmentPropertiesFetcher);
+
     tableDataMaps.add(dataMap);
     allDataMaps.put(table, tableDataMaps);
     return dataMap;
@@ -262,8 +260,8 @@ public final class DataMapStoreManager {
    * @return
    */
   private BlockletDetailsFetcher getBlockletDetailsFetcher(AbsoluteTableIdentifier identifier) {
-    TableDataMap blockletMap = getDataMap(identifier, BlockletDataMapFactory.DATA_MAP_SCHEMA);
-    return (BlockletDetailsFetcher) blockletMap.getDataMapFactory();
+    TableDataMap blockletMap = getDataMap(identifier, BlockletIndexDataMapFactory.DATA_MAP_SCHEMA);
+    return (BlockletDetailsFetcher) blockletMap.getIndexDataMapFactory();
   }
 
   /**

http://git-wip-us.apache.org/repos/asf/carbondata/blob/89a12af5/core/src/main/java/org/apache/carbondata/core/datamap/TableDataMap.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/carbondata/core/datamap/TableDataMap.java b/core/src/main/java/org/apache/carbondata/core/datamap/TableDataMap.java
index 3c66c89..b9a50d2 100644
--- a/core/src/main/java/org/apache/carbondata/core/datamap/TableDataMap.java
+++ b/core/src/main/java/org/apache/carbondata/core/datamap/TableDataMap.java
@@ -22,8 +22,8 @@ import java.util.List;
 
 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.IndexDataMap;
+import org.apache.carbondata.core.datamap.dev.IndexDataMapFactory;
 import org.apache.carbondata.core.datastore.block.SegmentProperties;
 import org.apache.carbondata.core.datastore.impl.FileFactory;
 import org.apache.carbondata.core.indexstore.Blocklet;
@@ -40,7 +40,7 @@ import org.apache.carbondata.events.OperationContext;
 import org.apache.carbondata.events.OperationEventListener;
 
 /**
- * DataMap at the table level, user can add any number of datamaps for one table. Depends
+ * IndexDataMap at the table level, user can add any number of datamaps for one table. Depends
  * on the filter condition it can prune the blocklets.
  */
 public final class TableDataMap extends OperationEventListener {
@@ -49,7 +49,7 @@ public final class TableDataMap extends OperationEventListener {
 
   private DataMapSchema dataMapSchema;
 
-  private DataMapFactory dataMapFactory;
+  private IndexDataMapFactory indexDataMapFactory;
 
   private BlockletDetailsFetcher blockletDetailsFetcher;
 
@@ -59,11 +59,11 @@ public final class TableDataMap extends OperationEventListener {
    * It is called to initialize and load the required table datamap metadata.
    */
   public TableDataMap(AbsoluteTableIdentifier identifier, DataMapSchema dataMapSchema,
-      DataMapFactory dataMapFactory, BlockletDetailsFetcher blockletDetailsFetcher,
+      IndexDataMapFactory indexDataMapFactory, BlockletDetailsFetcher blockletDetailsFetcher,
       SegmentPropertiesFetcher segmentPropertiesFetcher) {
     this.identifier = identifier;
     this.dataMapSchema = dataMapSchema;
-    this.dataMapFactory = dataMapFactory;
+    this.indexDataMapFactory = indexDataMapFactory;
     this.blockletDetailsFetcher = blockletDetailsFetcher;
     this.segmentPropertiesFetcher = segmentPropertiesFetcher;
   }
@@ -85,9 +85,9 @@ public final class TableDataMap extends OperationEventListener {
       if (filterExp == null) {
         pruneBlocklets = blockletDetailsFetcher.getAllBlocklets(segment, partitions);
       } else {
-        List<DataMap> dataMaps = dataMapFactory.getDataMaps(segment);
+        List<IndexDataMap> dataMaps = indexDataMapFactory.getDataMaps(segment);
         segmentProperties = segmentPropertiesFetcher.getSegmentProperties(segment);
-        for (DataMap dataMap : dataMaps) {
+        for (IndexDataMap dataMap : dataMaps) {
           pruneBlocklets.addAll(dataMap.prune(filterExp, segmentProperties, partitions));
         }
       }
@@ -115,7 +115,7 @@ public final class TableDataMap extends OperationEventListener {
   public List<DataMapDistributable> toDistributable(List<Segment> segments) throws IOException {
     List<DataMapDistributable> distributables = new ArrayList<>();
     for (Segment segment : segments) {
-      List<DataMapDistributable> list = dataMapFactory.toDistributable(segment);
+      List<DataMapDistributable> list = indexDataMapFactory.toDistributable(segment);
       for (DataMapDistributable distributable: list) {
         distributable.setDataMapSchema(dataMapSchema);
         distributable.setSegment(segment);
@@ -138,10 +138,10 @@ public final class TableDataMap extends OperationEventListener {
       FilterResolverIntf filterExp, List<PartitionSpec> partitions) throws IOException {
     List<ExtendedBlocklet> detailedBlocklets = new ArrayList<>();
     List<Blocklet> blocklets = new ArrayList<>();
-    List<DataMap> dataMaps = dataMapFactory.getDataMaps(distributable);
-    for (DataMap dataMap : dataMaps) {
+    List<IndexDataMap> indexDataMaps = indexDataMapFactory.getDataMaps(distributable);
+    for (IndexDataMap indexDataMap : indexDataMaps) {
       blocklets.addAll(
-          dataMap.prune(
+          indexDataMap.prune(
               filterExp,
               segmentPropertiesFetcher.getSegmentProperties(distributable.getSegment()),
               partitions));
@@ -150,13 +150,13 @@ public final class TableDataMap extends OperationEventListener {
     String writePath =
         identifier.getTablePath() + CarbonCommonConstants.FILE_SEPARATOR + dataMapSchema
             .getDataMapName();
-    if (dataMapFactory.getDataMapType() == DataMapType.FG) {
+    if (indexDataMapFactory.getDataMapType() == DataMapType.FG) {
       FileFactory.mkdirs(writePath, FileFactory.getFileType(writePath));
     }
     for (Blocklet blocklet : blocklets) {
       ExtendedBlocklet detailedBlocklet =
           blockletDetailsFetcher.getExtendedBlocklet(blocklet, distributable.getSegment());
-      if (dataMapFactory.getDataMapType() == DataMapType.FG) {
+      if (indexDataMapFactory.getDataMapType() == DataMapType.FG) {
         String blockletwritePath =
             writePath + CarbonCommonConstants.FILE_SEPARATOR + System.nanoTime();
         detailedBlocklet.setDataMapWriterPath(blockletwritePath);
@@ -174,7 +174,7 @@ public final class TableDataMap extends OperationEventListener {
    */
   public void clear(List<Segment> segments) {
     for (Segment segment: segments) {
-      dataMapFactory.clear(segment);
+      indexDataMapFactory.clear(segment);
     }
   }
 
@@ -182,19 +182,19 @@ public final class TableDataMap extends OperationEventListener {
    * Clears all datamap
    */
   public void clear() {
-    dataMapFactory.clear();
+    indexDataMapFactory.clear();
   }
 
   public DataMapSchema getDataMapSchema() {
     return dataMapSchema;
   }
 
-  public DataMapFactory getDataMapFactory() {
-    return dataMapFactory;
+  public IndexDataMapFactory getIndexDataMapFactory() {
+    return indexDataMapFactory;
   }
 
   @Override public void onEvent(Event event, OperationContext opContext) throws Exception {
-    dataMapFactory.fireEvent(event);
+    indexDataMapFactory.fireEvent(event);
   }
 
   /**
@@ -209,8 +209,8 @@ 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) {
+      List<IndexDataMap> dataMaps = indexDataMapFactory.getDataMaps(segment);
+      for (IndexDataMap dataMap : dataMaps) {
         if (dataMap.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

http://git-wip-us.apache.org/repos/asf/carbondata/blob/89a12af5/core/src/main/java/org/apache/carbondata/core/datamap/dev/AbstractDataMapWriter.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/carbondata/core/datamap/dev/AbstractDataMapWriter.java b/core/src/main/java/org/apache/carbondata/core/datamap/dev/AbstractDataMapWriter.java
index de6dcb1..c6961c7 100644
--- a/core/src/main/java/org/apache/carbondata/core/datamap/dev/AbstractDataMapWriter.java
+++ b/core/src/main/java/org/apache/carbondata/core/datamap/dev/AbstractDataMapWriter.java
@@ -71,7 +71,7 @@ public abstract class AbstractDataMapWriter {
 
   /**
    * Add the column pages row to the datamap, order of pages is same as `indexColumns` in
-   * DataMapMeta returned in DataMapFactory.
+   * DataMapMeta returned in IndexDataMapFactory.
    * Implementation should copy the content of `pages` as needed, because `pages` memory
    * may be freed after this method returns, if using unsafe column page.
    */

http://git-wip-us.apache.org/repos/asf/carbondata/blob/89a12af5/core/src/main/java/org/apache/carbondata/core/datamap/dev/DataMap.java
----------------------------------------------------------------------
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/DataMap.java
deleted file mode 100644
index dd5507c..0000000
--- a/core/src/main/java/org/apache/carbondata/core/datamap/dev/DataMap.java
+++ /dev/null
@@ -1,56 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *    http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.carbondata.core.datamap.dev;
-
-import java.io.IOException;
-import java.util.List;
-
-import org.apache.carbondata.core.datastore.block.SegmentProperties;
-import org.apache.carbondata.core.indexstore.Blocklet;
-import org.apache.carbondata.core.indexstore.PartitionSpec;
-import org.apache.carbondata.core.memory.MemoryException;
-import org.apache.carbondata.core.scan.filter.resolver.FilterResolverIntf;
-
-/**
- * Datamap is an entity which can store and retrieve index data.
- */
-public interface DataMap<T extends Blocklet> {
-
-  /**
-   * It is called to load the data map to memory or to initialize it.
-   */
-  void init(DataMapModel dataMapModel) throws MemoryException, IOException;
-
-  /**
-   * Prune the datamap with filter expression and partition information. It returns the list of
-   * blocklets where these filters can exist.
-   */
-  List<T> prune(FilterResolverIntf filterExp, SegmentProperties segmentProperties,
-      List<PartitionSpec> partitions);
-
-  // TODO Move this method to Abstract class
-  /**
-   * Validate whether the current segment needs to be fetching the required data
-   */
-  boolean isScanRequired(FilterResolverIntf filterExp);
-
-  /**
-   * Clear complete index table and release memory.
-   */
-  void clear();
-
-}

http://git-wip-us.apache.org/repos/asf/carbondata/blob/89a12af5/core/src/main/java/org/apache/carbondata/core/datamap/dev/DataMapFactory.java
----------------------------------------------------------------------
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/DataMapFactory.java
deleted file mode 100644
index b59de81..0000000
--- a/core/src/main/java/org/apache/carbondata/core/datamap/dev/DataMapFactory.java
+++ /dev/null
@@ -1,86 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *    http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.carbondata.core.datamap.dev;
-
-import java.io.IOException;
-import java.util.List;
-
-import org.apache.carbondata.core.datamap.DataMapDistributable;
-import org.apache.carbondata.core.datamap.DataMapMeta;
-import org.apache.carbondata.core.datamap.DataMapType;
-import org.apache.carbondata.core.datamap.Segment;
-import org.apache.carbondata.core.metadata.AbsoluteTableIdentifier;
-import org.apache.carbondata.core.metadata.schema.table.DataMapSchema;
-import org.apache.carbondata.events.Event;
-
-/**
- * Interface for datamap factory, it is responsible for creating the datamap.
- */
-public interface DataMapFactory<T extends DataMap> {
-
-  /**
-   * Initialization of Datamap factory with the identifier and datamap name
-   */
-  void init(AbsoluteTableIdentifier identifier, DataMapSchema dataMapSchema);
-
-  /**
-   * Return a new write for this datamap
-   */
-  AbstractDataMapWriter createWriter(Segment segment, String writeDirectoryPath);
-
-  /**
-   * Get the datamap for segmentid
-   */
-  List<T> getDataMaps(Segment segment) throws IOException;
-
-  /**
-   * Get datamaps for distributable object.
-   */
-  List<T> getDataMaps(DataMapDistributable distributable) throws IOException;
-
-  /**
-   * Get all distributable objects of a segmentid
-   * @return
-   */
-  List<DataMapDistributable> toDistributable(Segment segment);
-
-  /**
-   *
-   * @param event
-   */
-  void fireEvent(Event event);
-
-  /**
-   * Clears datamap of the segment
-   */
-  void clear(Segment segment);
-
-  /**
-   * Clear all datamaps from memory
-   */
-  void clear();
-
-  /**
-   * Return metadata of this datamap
-   */
-  DataMapMeta getMeta();
-
-  /**
-   *  Type of datamap whether it is FG or CG
-   */
-  DataMapType getDataMapType();
-}

http://git-wip-us.apache.org/repos/asf/carbondata/blob/89a12af5/core/src/main/java/org/apache/carbondata/core/datamap/dev/IndexDataMap.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/carbondata/core/datamap/dev/IndexDataMap.java b/core/src/main/java/org/apache/carbondata/core/datamap/dev/IndexDataMap.java
new file mode 100644
index 0000000..eed3ac9
--- /dev/null
+++ b/core/src/main/java/org/apache/carbondata/core/datamap/dev/IndexDataMap.java
@@ -0,0 +1,56 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.carbondata.core.datamap.dev;
+
+import java.io.IOException;
+import java.util.List;
+
+import org.apache.carbondata.core.datastore.block.SegmentProperties;
+import org.apache.carbondata.core.indexstore.Blocklet;
+import org.apache.carbondata.core.indexstore.PartitionSpec;
+import org.apache.carbondata.core.memory.MemoryException;
+import org.apache.carbondata.core.scan.filter.resolver.FilterResolverIntf;
+
+/**
+ * Datamap is an entity which can store and retrieve index data.
+ */
+public interface IndexDataMap<T extends Blocklet> {
+
+  /**
+   * It is called to load the data map to memory or to initialize it.
+   */
+  void init(DataMapModel dataMapModel) throws MemoryException, IOException;
+
+  /**
+   * Prune the datamap with filter expression and partition information. It returns the list of
+   * blocklets where these filters can exist.
+   */
+  List<T> prune(FilterResolverIntf filterExp, SegmentProperties segmentProperties,
+      List<PartitionSpec> partitions);
+
+  // TODO Move this method to Abstract class
+  /**
+   * Validate whether the current segment needs to be fetching the required data
+   */
+  boolean isScanRequired(FilterResolverIntf filterExp);
+
+  /**
+   * Clear complete index table and release memory.
+   */
+  void clear();
+
+}

http://git-wip-us.apache.org/repos/asf/carbondata/blob/89a12af5/core/src/main/java/org/apache/carbondata/core/datamap/dev/IndexDataMapFactory.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/carbondata/core/datamap/dev/IndexDataMapFactory.java b/core/src/main/java/org/apache/carbondata/core/datamap/dev/IndexDataMapFactory.java
new file mode 100644
index 0000000..12c9fd9
--- /dev/null
+++ b/core/src/main/java/org/apache/carbondata/core/datamap/dev/IndexDataMapFactory.java
@@ -0,0 +1,86 @@
+/*
+ * 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.datamap.dev;
+
+import java.io.IOException;
+import java.util.List;
+
+import org.apache.carbondata.core.datamap.DataMapDistributable;
+import org.apache.carbondata.core.datamap.DataMapMeta;
+import org.apache.carbondata.core.datamap.DataMapType;
+import org.apache.carbondata.core.datamap.Segment;
+import org.apache.carbondata.core.metadata.AbsoluteTableIdentifier;
+import org.apache.carbondata.core.metadata.schema.table.DataMapSchema;
+import org.apache.carbondata.events.Event;
+
+/**
+ * Interface for datamap factory, it is responsible for creating the datamap.
+ */
+public interface IndexDataMapFactory<T extends IndexDataMap> {
+
+  /**
+   * Initialization of Datamap factory with the identifier and datamap name
+   */
+  void init(AbsoluteTableIdentifier identifier, DataMapSchema dataMapSchema);
+
+  /**
+   * Return a new write for this datamap
+   */
+  AbstractDataMapWriter createWriter(Segment segment, String writeDirectoryPath);
+
+  /**
+   * Get the datamap for segmentid
+   */
+  List<T> getDataMaps(Segment segment) throws IOException;
+
+  /**
+   * Get datamaps for distributable object.
+   */
+  List<T> getDataMaps(DataMapDistributable distributable) throws IOException;
+
+  /**
+   * Get all distributable objects of a segmentid
+   * @return
+   */
+  List<DataMapDistributable> toDistributable(Segment segment);
+
+  /**
+   *
+   * @param event
+   */
+  void fireEvent(Event event);
+
+  /**
+   * Clears datamap of the segment
+   */
+  void clear(Segment segment);
+
+  /**
+   * Clear all datamaps from memory
+   */
+  void clear();
+
+  /**
+   * Return metadata of this datamap
+   */
+  DataMapMeta getMeta();
+
+  /**
+   *  Type of datamap whether it is FG or CG
+   */
+  DataMapType getDataMapType();
+}

http://git-wip-us.apache.org/repos/asf/carbondata/blob/89a12af5/core/src/main/java/org/apache/carbondata/core/datamap/dev/cgdatamap/AbstractCoarseGrainDataMap.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/carbondata/core/datamap/dev/cgdatamap/AbstractCoarseGrainDataMap.java b/core/src/main/java/org/apache/carbondata/core/datamap/dev/cgdatamap/AbstractCoarseGrainDataMap.java
deleted file mode 100644
index d79d0c6..0000000
--- a/core/src/main/java/org/apache/carbondata/core/datamap/dev/cgdatamap/AbstractCoarseGrainDataMap.java
+++ /dev/null
@@ -1,24 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *    http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.carbondata.core.datamap.dev.cgdatamap;
-
-import org.apache.carbondata.core.datamap.dev.DataMap;
-import org.apache.carbondata.core.indexstore.Blocklet;
-
-public abstract class AbstractCoarseGrainDataMap implements DataMap<Blocklet> {
-
-}

http://git-wip-us.apache.org/repos/asf/carbondata/blob/89a12af5/core/src/main/java/org/apache/carbondata/core/datamap/dev/cgdatamap/AbstractCoarseGrainDataMapFactory.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/carbondata/core/datamap/dev/cgdatamap/AbstractCoarseGrainDataMapFactory.java b/core/src/main/java/org/apache/carbondata/core/datamap/dev/cgdatamap/AbstractCoarseGrainDataMapFactory.java
deleted file mode 100644
index 9789992..0000000
--- a/core/src/main/java/org/apache/carbondata/core/datamap/dev/cgdatamap/AbstractCoarseGrainDataMapFactory.java
+++ /dev/null
@@ -1,34 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *    http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.carbondata.core.datamap.dev.cgdatamap;
-
-import org.apache.carbondata.core.datamap.DataMapType;
-import org.apache.carbondata.core.datamap.dev.DataMapFactory;
-
-/**
- *  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.
- */
-public abstract class AbstractCoarseGrainDataMapFactory
-    implements DataMapFactory<AbstractCoarseGrainDataMap> {
-
-  @Override public DataMapType getDataMapType() {
-    return DataMapType.CG;
-  }
-}

http://git-wip-us.apache.org/repos/asf/carbondata/blob/89a12af5/core/src/main/java/org/apache/carbondata/core/datamap/dev/cgdatamap/AbstractCoarseGrainIndexDataMap.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/carbondata/core/datamap/dev/cgdatamap/AbstractCoarseGrainIndexDataMap.java b/core/src/main/java/org/apache/carbondata/core/datamap/dev/cgdatamap/AbstractCoarseGrainIndexDataMap.java
new file mode 100644
index 0000000..df9d4e8
--- /dev/null
+++ b/core/src/main/java/org/apache/carbondata/core/datamap/dev/cgdatamap/AbstractCoarseGrainIndexDataMap.java
@@ -0,0 +1,24 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.carbondata.core.datamap.dev.cgdatamap;
+
+import org.apache.carbondata.core.datamap.dev.IndexDataMap;
+import org.apache.carbondata.core.indexstore.Blocklet;
+
+public abstract class AbstractCoarseGrainIndexDataMap implements IndexDataMap<Blocklet> {
+
+}

http://git-wip-us.apache.org/repos/asf/carbondata/blob/89a12af5/core/src/main/java/org/apache/carbondata/core/datamap/dev/cgdatamap/AbstractCoarseGrainIndexDataMapFactory.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/carbondata/core/datamap/dev/cgdatamap/AbstractCoarseGrainIndexDataMapFactory.java b/core/src/main/java/org/apache/carbondata/core/datamap/dev/cgdatamap/AbstractCoarseGrainIndexDataMapFactory.java
new file mode 100644
index 0000000..037c32e
--- /dev/null
+++ b/core/src/main/java/org/apache/carbondata/core/datamap/dev/cgdatamap/AbstractCoarseGrainIndexDataMapFactory.java
@@ -0,0 +1,34 @@
+/*
+ * 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.datamap.dev.cgdatamap;
+
+import org.apache.carbondata.core.datamap.DataMapType;
+import org.apache.carbondata.core.datamap.dev.IndexDataMapFactory;
+
+/**
+ *  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.
+ */
+public abstract class AbstractCoarseGrainIndexDataMapFactory
+    implements IndexDataMapFactory<AbstractCoarseGrainIndexDataMap> {
+
+  @Override public DataMapType getDataMapType() {
+    return DataMapType.CG;
+  }
+}

http://git-wip-us.apache.org/repos/asf/carbondata/blob/89a12af5/core/src/main/java/org/apache/carbondata/core/datamap/dev/expr/DataMapExprWrapperImpl.java
----------------------------------------------------------------------
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/DataMapExprWrapperImpl.java
index fac6cc1..312fc11 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/DataMapExprWrapperImpl.java
@@ -83,6 +83,6 @@ public class DataMapExprWrapperImpl implements DataMapExprWrapper {
   }
 
   @Override public DataMapType getDataMapType() {
-    return dataMap.getDataMapFactory().getDataMapType();
+    return dataMap.getIndexDataMapFactory().getDataMapType();
   }
 }

http://git-wip-us.apache.org/repos/asf/carbondata/blob/89a12af5/core/src/main/java/org/apache/carbondata/core/datamap/dev/fgdatamap/AbstractFineGrainDataMap.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/carbondata/core/datamap/dev/fgdatamap/AbstractFineGrainDataMap.java b/core/src/main/java/org/apache/carbondata/core/datamap/dev/fgdatamap/AbstractFineGrainDataMap.java
deleted file mode 100644
index 310fb3b..0000000
--- a/core/src/main/java/org/apache/carbondata/core/datamap/dev/fgdatamap/AbstractFineGrainDataMap.java
+++ /dev/null
@@ -1,24 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *    http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.carbondata.core.datamap.dev.fgdatamap;
-
-import org.apache.carbondata.core.datamap.dev.DataMap;
-import org.apache.carbondata.core.indexstore.FineGrainBlocklet;
-
-public abstract class AbstractFineGrainDataMap implements DataMap<FineGrainBlocklet> {
-
-}

http://git-wip-us.apache.org/repos/asf/carbondata/blob/89a12af5/core/src/main/java/org/apache/carbondata/core/datamap/dev/fgdatamap/AbstractFineGrainDataMapFactory.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/carbondata/core/datamap/dev/fgdatamap/AbstractFineGrainDataMapFactory.java b/core/src/main/java/org/apache/carbondata/core/datamap/dev/fgdatamap/AbstractFineGrainDataMapFactory.java
deleted file mode 100644
index 1ca7fc3..0000000
--- a/core/src/main/java/org/apache/carbondata/core/datamap/dev/fgdatamap/AbstractFineGrainDataMapFactory.java
+++ /dev/null
@@ -1,38 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *    http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.carbondata.core.datamap.dev.fgdatamap;
-
-import org.apache.carbondata.core.datamap.DataMapType;
-import org.apache.carbondata.core.datamap.dev.DataMapFactory;
-
-/**
- *  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
- *     information of block, blocklet, page and rowids information as well.
- *  3. The pruned blocklets are internally wriitten to file and returns only the block ,
- *    blocklet and filepath information as part of Splits.
- *  4. Based on the splits scanrdd schedule the tasks.
- *  5. In filterscanner we check the datamapwriterpath from split and reNoteads the
- *     bitset if exists. And pass this bitset as input to it.
- */
-public abstract class AbstractFineGrainDataMapFactory
-    implements DataMapFactory<AbstractFineGrainDataMap> {
-
-  @Override public DataMapType getDataMapType() {
-    return DataMapType.FG;
-  }
-}

http://git-wip-us.apache.org/repos/asf/carbondata/blob/89a12af5/core/src/main/java/org/apache/carbondata/core/datamap/dev/fgdatamap/AbstractFineGrainIndexDataMap.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/carbondata/core/datamap/dev/fgdatamap/AbstractFineGrainIndexDataMap.java b/core/src/main/java/org/apache/carbondata/core/datamap/dev/fgdatamap/AbstractFineGrainIndexDataMap.java
new file mode 100644
index 0000000..ea536b9
--- /dev/null
+++ b/core/src/main/java/org/apache/carbondata/core/datamap/dev/fgdatamap/AbstractFineGrainIndexDataMap.java
@@ -0,0 +1,24 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.carbondata.core.datamap.dev.fgdatamap;
+
+import org.apache.carbondata.core.datamap.dev.IndexDataMap;
+import org.apache.carbondata.core.indexstore.FineGrainBlocklet;
+
+public abstract class AbstractFineGrainIndexDataMap implements IndexDataMap<FineGrainBlocklet> {
+
+}

http://git-wip-us.apache.org/repos/asf/carbondata/blob/89a12af5/core/src/main/java/org/apache/carbondata/core/datamap/dev/fgdatamap/AbstractFineGrainIndexDataMapFactory.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/carbondata/core/datamap/dev/fgdatamap/AbstractFineGrainIndexDataMapFactory.java b/core/src/main/java/org/apache/carbondata/core/datamap/dev/fgdatamap/AbstractFineGrainIndexDataMapFactory.java
new file mode 100644
index 0000000..762c233
--- /dev/null
+++ b/core/src/main/java/org/apache/carbondata/core/datamap/dev/fgdatamap/AbstractFineGrainIndexDataMapFactory.java
@@ -0,0 +1,38 @@
+/*
+ * 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.datamap.dev.fgdatamap;
+
+import org.apache.carbondata.core.datamap.DataMapType;
+import org.apache.carbondata.core.datamap.dev.IndexDataMapFactory;
+
+/**
+ *  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
+ *     information of block, blocklet, page and rowids information as well.
+ *  3. The pruned blocklets are internally wriitten to file and returns only the block ,
+ *    blocklet and filepath information as part of Splits.
+ *  4. Based on the splits scanrdd schedule the tasks.
+ *  5. In filterscanner we check the datamapwriterpath from split and reNoteads the
+ *     bitset if exists. And pass this bitset as input to it.
+ */
+public abstract class AbstractFineGrainIndexDataMapFactory
+    implements IndexDataMapFactory<AbstractFineGrainIndexDataMap> {
+
+  @Override public DataMapType getDataMapType() {
+    return DataMapType.FG;
+  }
+}

http://git-wip-us.apache.org/repos/asf/carbondata/blob/89a12af5/core/src/main/java/org/apache/carbondata/core/indexstore/BlockletDataMapIndexStore.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/carbondata/core/indexstore/BlockletDataMapIndexStore.java b/core/src/main/java/org/apache/carbondata/core/indexstore/BlockletDataMapIndexStore.java
index defaf39..67dbe88 100644
--- a/core/src/main/java/org/apache/carbondata/core/indexstore/BlockletDataMapIndexStore.java
+++ b/core/src/main/java/org/apache/carbondata/core/indexstore/BlockletDataMapIndexStore.java
@@ -31,8 +31,8 @@ import org.apache.carbondata.core.constants.CarbonCommonConstants;
 import org.apache.carbondata.core.datastore.filesystem.AbstractDFSCarbonFile;
 import org.apache.carbondata.core.datastore.filesystem.CarbonFile;
 import org.apache.carbondata.core.datastore.impl.FileFactory;
-import org.apache.carbondata.core.indexstore.blockletindex.BlockletDataMap;
 import org.apache.carbondata.core.indexstore.blockletindex.BlockletDataMapModel;
+import org.apache.carbondata.core.indexstore.blockletindex.BlockletIndexDataMap;
 import org.apache.carbondata.core.indexstore.blockletindex.SegmentIndexFileStore;
 import org.apache.carbondata.core.memory.MemoryException;
 import org.apache.carbondata.core.metadata.blocklet.DataFileFooter;
@@ -47,7 +47,7 @@ import org.apache.hadoop.fs.RemoteIterator;
  * blocks
  */
 public class BlockletDataMapIndexStore
-    implements Cache<TableBlockIndexUniqueIdentifier, BlockletDataMap> {
+    implements Cache<TableBlockIndexUniqueIdentifier, BlockletIndexDataMap> {
   private static final LogService LOGGER =
       LogServiceFactory.getLogService(BlockletDataMapIndexStore.class.getName());
   /**
@@ -74,10 +74,10 @@ public class BlockletDataMapIndexStore
   }
 
   @Override
-  public BlockletDataMap get(TableBlockIndexUniqueIdentifier identifier)
+  public BlockletIndexDataMap get(TableBlockIndexUniqueIdentifier identifier)
       throws IOException {
     String lruCacheKey = identifier.getUniqueTableSegmentIdentifier();
-    BlockletDataMap dataMap = (BlockletDataMap) lruCache.get(lruCacheKey);
+    BlockletIndexDataMap dataMap = (BlockletIndexDataMap) lruCache.get(lruCacheKey);
     if (dataMap == null) {
       try {
         SegmentIndexFileStore indexFileStore = new SegmentIndexFileStore();
@@ -134,14 +134,15 @@ public class BlockletDataMapIndexStore
   }
 
   @Override
-  public List<BlockletDataMap> getAll(
+  public List<BlockletIndexDataMap> getAll(
       List<TableBlockIndexUniqueIdentifier> tableSegmentUniqueIdentifiers) throws IOException {
-    List<BlockletDataMap> blockletDataMaps = new ArrayList<>(tableSegmentUniqueIdentifiers.size());
+    List<BlockletIndexDataMap> blockletDataMaps =
+        new ArrayList<>(tableSegmentUniqueIdentifiers.size());
     List<TableBlockIndexUniqueIdentifier> missedIdentifiers = new ArrayList<>();
     // Get the datamaps for each indexfile from cache.
     try {
       for (TableBlockIndexUniqueIdentifier identifier : tableSegmentUniqueIdentifiers) {
-        BlockletDataMap ifPresent = getIfPresent(identifier);
+        BlockletIndexDataMap ifPresent = getIfPresent(identifier);
         if (ifPresent != null) {
           blockletDataMaps.add(ifPresent);
         } else {
@@ -159,7 +160,7 @@ public class BlockletDataMapIndexStore
         }
       }
     } catch (Throwable e) {
-      for (BlockletDataMap dataMap : blockletDataMaps) {
+      for (BlockletIndexDataMap dataMap : blockletDataMaps) {
         dataMap.clear();
       }
       throw new IOException("Problem in loading segment blocks.", e);
@@ -174,9 +175,9 @@ public class BlockletDataMapIndexStore
    * @return
    */
   @Override
-  public BlockletDataMap getIfPresent(
+  public BlockletIndexDataMap getIfPresent(
       TableBlockIndexUniqueIdentifier tableSegmentUniqueIdentifier) {
-    return (BlockletDataMap) lruCache.get(
+    return (BlockletIndexDataMap) lruCache.get(
         tableSegmentUniqueIdentifier.getUniqueTableSegmentIdentifier());
   }
 
@@ -199,7 +200,7 @@ public class BlockletDataMapIndexStore
    * @return map of taks id to segment mapping
    * @throws IOException
    */
-  private BlockletDataMap loadAndGetDataMap(
+  private BlockletIndexDataMap loadAndGetDataMap(
       TableBlockIndexUniqueIdentifier identifier,
       SegmentIndexFileStore indexFileStore,
       Map<String, BlockMetaInfo> blockMetaInfoMap)
@@ -210,9 +211,9 @@ public class BlockletDataMapIndexStore
     if (lock == null) {
       lock = addAndGetSegmentLock(uniqueTableSegmentIdentifier);
     }
-    BlockletDataMap dataMap;
+    BlockletIndexDataMap dataMap;
     synchronized (lock) {
-      dataMap = new BlockletDataMap();
+      dataMap = new BlockletIndexDataMap();
       dataMap.init(new BlockletDataMapModel(
           identifier.getIndexFilePath() + CarbonCommonConstants.FILE_SEPARATOR + identifier
               .getIndexFileName(), indexFileStore.getFileData(identifier.getIndexFileName()),
@@ -248,9 +249,9 @@ public class BlockletDataMapIndexStore
   @Override
   public void clearAccessCount(
       List<TableBlockIndexUniqueIdentifier> tableSegmentUniqueIdentifiers) {
-    for (TableBlockIndexUniqueIdentifier segmentUniqueIdentifier : tableSegmentUniqueIdentifiers) {
-      BlockletDataMap cacheable =
-          (BlockletDataMap) lruCache.get(segmentUniqueIdentifier.getUniqueTableSegmentIdentifier());
+    for (TableBlockIndexUniqueIdentifier identifier : tableSegmentUniqueIdentifiers) {
+      BlockletIndexDataMap cacheable =
+          (BlockletIndexDataMap) lruCache.get(identifier.getUniqueTableSegmentIdentifier());
       cacheable.clear();
     }
   }