You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@carbondata.apache.org by ja...@apache.org on 2018/02/24 02:17:09 UTC

[1/6] carbondata git commit: [CARBONDATA-2189] Add DataMapProvider developer interface

Repository: carbondata
Updated Branches:
  refs/heads/datamap e616162c0 -> 5a625f4ce


http://git-wip-us.apache.org/repos/asf/carbondata/blob/5a625f4c/integration/spark2/src/main/scala/org/apache/spark/sql/execution/command/preaaggregate/PreAggregateTableHelper.scala
----------------------------------------------------------------------
diff --git a/integration/spark2/src/main/scala/org/apache/spark/sql/execution/command/preaaggregate/PreAggregateTableHelper.scala b/integration/spark2/src/main/scala/org/apache/spark/sql/execution/command/preaaggregate/PreAggregateTableHelper.scala
new file mode 100644
index 0000000..96e840f
--- /dev/null
+++ b/integration/spark2/src/main/scala/org/apache/spark/sql/execution/command/preaaggregate/PreAggregateTableHelper.scala
@@ -0,0 +1,202 @@
+/*
+ * 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.spark.sql.execution.command.preaaggregate
+
+import scala.collection.JavaConverters._
+import scala.collection.mutable
+
+import org.apache.spark.sql._
+import org.apache.spark.sql.catalyst.TableIdentifier
+import org.apache.spark.sql.execution.command._
+import org.apache.spark.sql.execution.command.management.CarbonLoadDataCommand
+import org.apache.spark.sql.execution.command.table.CarbonCreateTableCommand
+import org.apache.spark.sql.execution.command.timeseries.TimeSeriesUtil
+import org.apache.spark.sql.parser.CarbonSpark2SqlParser
+
+import org.apache.carbondata.common.exceptions.sql.MalformedDataMapCommandException
+import org.apache.carbondata.core.constants.CarbonCommonConstants
+import org.apache.carbondata.core.metadata.schema.datamap.DataMapProvider
+import org.apache.carbondata.core.metadata.schema.table.{AggregationDataMapSchema, CarbonTable, DataMapSchema}
+import org.apache.carbondata.core.statusmanager.{SegmentStatus, SegmentStatusManager}
+
+/**
+ * Below helper class will be used to create pre-aggregate table
+ * and updating the parent table about the child table information
+ * It will be either success or nothing happen in case of failure:
+ * 1. failed to create pre aggregate table.
+ * 2. failed to update main table
+ *
+ */
+case class PreAggregateTableHelper(
+    var parentTable: CarbonTable,
+    dataMapName: String,
+    dataMapClassName: String,
+    dataMapProperties: java.util.Map[String, String],
+    queryString: String,
+    timeSeriesFunction: String) {
+
+  var loadCommand: CarbonLoadDataCommand = _
+
+  def initMeta(sparkSession: SparkSession): Seq[Row] = {
+    val dmProperties = dataMapProperties.asScala
+    val updatedQuery = new CarbonSpark2SqlParser().addPreAggFunction(queryString)
+    val df = sparkSession.sql(updatedQuery)
+    val fieldRelationMap = PreAggregateUtil.validateActualSelectPlanAndGetAttributes(
+      df.logicalPlan, queryString)
+    val fields = fieldRelationMap.keySet.toSeq
+    val tableProperties = mutable.Map[String, String]()
+    dmProperties.foreach(t => tableProperties.put(t._1, t._2))
+
+    val selectTable = PreAggregateUtil.getParentCarbonTable(df.logicalPlan)
+    if (!parentTable.getTableName.equalsIgnoreCase(selectTable.getTableName)) {
+      throw new MalformedDataMapCommandException(
+        "Parent table name is different in select and create")
+    }
+    var neworder = Seq[String]()
+    val parentOrder = parentTable.getSortColumns(parentTable.getTableName).asScala
+    parentOrder.foreach(parentcol =>
+      fields.filter(col => fieldRelationMap(col).aggregateFunction.isEmpty &&
+                           parentcol.equals(fieldRelationMap(col).
+                             columnTableRelationList.get(0).parentColumnName))
+        .map(cols => neworder :+= cols.column)
+    )
+    tableProperties.put(CarbonCommonConstants.SORT_COLUMNS, neworder.mkString(","))
+    tableProperties.put("sort_scope", parentTable.getTableInfo.getFactTable.
+      getTableProperties.getOrDefault("sort_scope", CarbonCommonConstants
+      .LOAD_SORT_SCOPE_DEFAULT))
+    tableProperties
+      .put(CarbonCommonConstants.TABLE_BLOCKSIZE, parentTable.getBlockSizeInMB.toString)
+    val tableIdentifier =
+      TableIdentifier(parentTable.getTableName + "_" + dataMapName,
+        Some(parentTable.getDatabaseName))
+    // prepare table model of the collected tokens
+    val tableModel: TableModel = new CarbonSpark2SqlParser().prepareTableModel(
+      ifNotExistPresent = false,
+      new CarbonSpark2SqlParser().convertDbNameToLowerCase(tableIdentifier.database),
+      tableIdentifier.table.toLowerCase,
+      fields,
+      Seq(),
+      tableProperties,
+      None,
+      isAlterFlow = false,
+      None)
+
+    // updating the relation identifier, this will be stored in child table
+    // which can be used during dropping of pre-aggreate table as parent table will
+    // also get updated
+    if(timeSeriesFunction != null) {
+      TimeSeriesUtil.validateTimeSeriesEventTime(dmProperties.toMap, parentTable)
+      TimeSeriesUtil.validateEventTimeColumnExitsInSelect(
+        fieldRelationMap,
+        dmProperties(TimeSeriesUtil.TIMESERIES_EVENTTIME))
+      TimeSeriesUtil.updateTimeColumnSelect(fieldRelationMap,
+        dmProperties(TimeSeriesUtil.TIMESERIES_EVENTTIME),
+      timeSeriesFunction)
+    }
+    tableModel.parentTable = Some(parentTable)
+    tableModel.dataMapRelation = Some(fieldRelationMap)
+    val tablePath = if (dmProperties.contains("path")) {
+      dmProperties("path")
+    } else {
+      CarbonEnv.getTablePath(tableModel.databaseNameOp, tableModel.tableName)(sparkSession)
+    }
+    CarbonCreateTableCommand(TableNewProcessor(tableModel),
+      tableModel.ifNotExistsSet, Some(tablePath)).run(sparkSession)
+
+    val table = CarbonEnv.getCarbonTable(tableIdentifier)(sparkSession)
+    val tableInfo = table.getTableInfo
+
+    // child schema object will be saved on parent table schema
+    val childSchema = tableInfo.getFactTable.buildChildSchema(
+      dataMapName,
+      DataMapProvider.PREAGGREGATE.toString,
+      tableInfo.getDatabaseName,
+      queryString,
+      "AGGREGATION")
+    dmProperties.foreach(f => childSchema.getProperties.put(f._1, f._2))
+
+    // updating the parent table about child table
+    PreAggregateUtil.updateMainTable(parentTable, childSchema, sparkSession)
+
+    // After updating the parent carbon table with data map entry extract the latest table object
+    // to be used in further create process.
+    parentTable = CarbonEnv.getCarbonTable(Some(parentTable.getDatabaseName),
+      parentTable.getTableName)(sparkSession)
+
+    val updatedLoadQuery = if (timeSeriesFunction != null) {
+      val dataMap = parentTable.getTableInfo.getDataMapSchemaList.asScala
+        .filter(p => p.getDataMapName.equalsIgnoreCase(dataMapName))
+        .head
+        .asInstanceOf[AggregationDataMapSchema]
+      PreAggregateUtil.createTimeSeriesSelectQueryFromMain(dataMap.getChildSchema,
+        parentTable.getTableName,
+        parentTable.getDatabaseName)
+    } else {
+      queryString
+    }
+    val dataFrame = sparkSession.sql(new CarbonSpark2SqlParser().addPreAggLoadFunction(
+      updatedLoadQuery)).drop("preAggLoad")
+    loadCommand = PreAggregateUtil.createLoadCommandForChild(
+      tableInfo.getFactTable.getListOfColumns,
+      tableIdentifier,
+      dataFrame,
+      isOverwrite = false,
+      sparkSession = sparkSession)
+    loadCommand.processMetadata(sparkSession)
+    Seq.empty
+  }
+
+  def initData(sparkSession: SparkSession): Seq[Row] = {
+    // load child table if parent table has existing segments
+    // This will be used to check if the parent table has any segments or not. If not then no
+    // need to fire load for pre-aggregate table. Therefore reading the load details for PARENT
+    // table.
+    SegmentStatusManager.deleteLoadsAndUpdateMetadata(parentTable, false)
+    val loadAvailable = SegmentStatusManager.readLoadMetadata(parentTable.getMetadataPath)
+    if (loadAvailable.exists(load => load.getSegmentStatus == SegmentStatus.INSERT_IN_PROGRESS ||
+      load.getSegmentStatus == SegmentStatus.INSERT_OVERWRITE_IN_PROGRESS)) {
+      throw new UnsupportedOperationException(
+        "Cannot create pre-aggregate table when insert is in progress on main table")
+    } else if (loadAvailable.nonEmpty) {
+      val updatedQuery = if (timeSeriesFunction != null) {
+        val dataMap = parentTable.getTableInfo.getDataMapSchemaList.asScala
+          .filter(p => p.getDataMapName
+            .equalsIgnoreCase(dataMapName)).head
+          .asInstanceOf[AggregationDataMapSchema]
+        PreAggregateUtil.createTimeSeriesSelectQueryFromMain(dataMap.getChildSchema,
+          parentTable.getTableName,
+          parentTable.getDatabaseName)
+      } else {
+        queryString
+      }
+      // Passing segmentToLoad as * because we want to load all the segments into the
+      // pre-aggregate table even if the user has set some segments on the parent table.
+      loadCommand.dataFrame = Some(PreAggregateUtil
+        .getDataFrame(sparkSession, loadCommand.logicalPlan.get))
+      PreAggregateUtil.startDataLoadForDataMap(
+        parentTable,
+        segmentToLoad = "*",
+        validateSegments = true,
+        sparkSession,
+        loadCommand)
+    }
+    Seq.empty
+  }
+}
+
+

http://git-wip-us.apache.org/repos/asf/carbondata/blob/5a625f4c/integration/spark2/src/main/scala/org/apache/spark/sql/execution/command/preaaggregate/PreAggregateUtil.scala
----------------------------------------------------------------------
diff --git a/integration/spark2/src/main/scala/org/apache/spark/sql/execution/command/preaaggregate/PreAggregateUtil.scala b/integration/spark2/src/main/scala/org/apache/spark/sql/execution/command/preaaggregate/PreAggregateUtil.scala
index 1073f63..845e30d 100644
--- a/integration/spark2/src/main/scala/org/apache/spark/sql/execution/command/preaaggregate/PreAggregateUtil.scala
+++ b/integration/spark2/src/main/scala/org/apache/spark/sql/execution/command/preaaggregate/PreAggregateUtil.scala
@@ -30,7 +30,7 @@ import org.apache.spark.sql.catalyst.plans.logical._
 import org.apache.spark.sql.execution.command.{ColumnTableRelation, DataMapField, Field}
 import org.apache.spark.sql.execution.command.management.CarbonLoadDataCommand
 import org.apache.spark.sql.execution.datasources.LogicalRelation
-import org.apache.spark.sql.hive.CarbonRelation
+import org.apache.spark.sql.hive.{CarbonMetaStore, CarbonRelation}
 import org.apache.spark.sql.parser.CarbonSpark2SqlParser
 import org.apache.spark.sql.types.DataType
 
@@ -405,11 +405,11 @@ object PreAggregateUtil {
    * Below method will be used to update the main table about the pre aggregate table information
    * in case of any exception it will throw error so pre aggregate table creation will fail
    *
-   * @param childSchema
-   * @param sparkSession
+   * @return the existing TableInfo object before updating, it can be used to recover if any
+   *         operation failed later
    */
   def updateMainTable(carbonTable: CarbonTable,
-      childSchema: DataMapSchema, sparkSession: SparkSession): Unit = {
+      childSchema: DataMapSchema, sparkSession: SparkSession): TableInfo = {
     val LOGGER: LogService = LogServiceFactory.getLogService(this.getClass.getCanonicalName)
     val locksToBeAcquired = List(LockUsage.METADATA_LOCK,
       LockUsage.DROP_TABLE_LOCK)
@@ -422,7 +422,7 @@ object PreAggregateUtil {
       locks = acquireLock(dbName, tableName, locksToBeAcquired, carbonTable)
       // get the latest carbon table and check for column existence
       // read the latest schema file
-      val thriftTableInfo: TableInfo = metastore.getThriftTableInfo(carbonTable)(sparkSession)
+      val thriftTableInfo: TableInfo = metastore.getThriftTableInfo(carbonTable)
       val schemaConverter = new ThriftWrapperSchemaConverterImpl()
       val wrapperTableInfo = schemaConverter.fromExternalToWrapperTableInfo(
         thriftTableInfo,
@@ -435,11 +435,11 @@ object PreAggregateUtil {
         throw new MetadataProcessException("DataMap name already exist")
       }
       wrapperTableInfo.getDataMapSchemaList.add(childSchema)
-      val thriftTable = schemaConverter
-        .fromWrapperToExternalTableInfo(wrapperTableInfo, dbName, tableName)
-      updateSchemaInfo(carbonTable,
-        thriftTable)(sparkSession)
+      val thriftTable = schemaConverter.fromWrapperToExternalTableInfo(
+        wrapperTableInfo, dbName, tableName)
+      updateSchemaInfo(carbonTable, thriftTable)(sparkSession)
       LOGGER.info(s"Parent table updated is successful for table $dbName.$tableName")
+      thriftTableInfo
     } catch {
       case e: Exception =>
         LOGGER.error(e, "Pre Aggregate Parent table update failed reverting changes")
@@ -449,7 +449,6 @@ object PreAggregateUtil {
       // release lock after command execution completion
       releaseLocks(locks)
     }
-    Seq.empty
   }
 
   /**
@@ -525,7 +524,7 @@ object PreAggregateUtil {
     val metastore = CarbonEnv.getInstance(sparkSession).carbonMetastore
     val carbonTable = CarbonEnv.getCarbonTable(Some(dbName), tableName)(sparkSession)
     carbonTable.getTableLastUpdatedTime
-    val thriftTable: TableInfo = metastore.getThriftTableInfo(carbonTable)(sparkSession)
+    val thriftTable: TableInfo = metastore.getThriftTableInfo(carbonTable)
     if (thriftTable.dataMapSchemas.size > numberOfChildSchema) {
       metastore.revertTableSchemaForPreAggCreationFailure(
         carbonTable.getAbsoluteTableIdentifier, thriftTable)(sparkSession)

http://git-wip-us.apache.org/repos/asf/carbondata/blob/5a625f4c/integration/spark2/src/main/scala/org/apache/spark/sql/execution/command/schema/CarbonAlterTableAddColumnCommand.scala
----------------------------------------------------------------------
diff --git a/integration/spark2/src/main/scala/org/apache/spark/sql/execution/command/schema/CarbonAlterTableAddColumnCommand.scala b/integration/spark2/src/main/scala/org/apache/spark/sql/execution/command/schema/CarbonAlterTableAddColumnCommand.scala
index 07917d0..07d693b 100644
--- a/integration/spark2/src/main/scala/org/apache/spark/sql/execution/command/schema/CarbonAlterTableAddColumnCommand.scala
+++ b/integration/spark2/src/main/scala/org/apache/spark/sql/execution/command/schema/CarbonAlterTableAddColumnCommand.scala
@@ -63,7 +63,7 @@ private[sql] case class CarbonAlterTableAddColumnCommand(
       OperationListenerBus.getInstance().fireEvent(alterTableAddColumnListener, operationContext)
       // get the latest carbon table and check for column existence
       // read the latest schema file
-      val thriftTableInfo: TableInfo = metastore.getThriftTableInfo(carbonTable)(sparkSession)
+      val thriftTableInfo: TableInfo = metastore.getThriftTableInfo(carbonTable)
       val schemaConverter = new ThriftWrapperSchemaConverterImpl()
       val wrapperTableInfo = schemaConverter
         .fromExternalToWrapperTableInfo(thriftTableInfo,

http://git-wip-us.apache.org/repos/asf/carbondata/blob/5a625f4c/integration/spark2/src/main/scala/org/apache/spark/sql/execution/command/schema/CarbonAlterTableDataTypeChangeCommand.scala
----------------------------------------------------------------------
diff --git a/integration/spark2/src/main/scala/org/apache/spark/sql/execution/command/schema/CarbonAlterTableDataTypeChangeCommand.scala b/integration/spark2/src/main/scala/org/apache/spark/sql/execution/command/schema/CarbonAlterTableDataTypeChangeCommand.scala
index fa8003e..51c8ec8 100644
--- a/integration/spark2/src/main/scala/org/apache/spark/sql/execution/command/schema/CarbonAlterTableDataTypeChangeCommand.scala
+++ b/integration/spark2/src/main/scala/org/apache/spark/sql/execution/command/schema/CarbonAlterTableDataTypeChangeCommand.scala
@@ -74,7 +74,7 @@ private[sql] case class CarbonAlterTableDataTypeChangeCommand(
         sys.error(s"Invalid Column: $columnName")
       }
       // read the latest schema file
-      val tableInfo: TableInfo = metastore.getThriftTableInfo(carbonTable)(sparkSession)
+      val tableInfo: TableInfo = metastore.getThriftTableInfo(carbonTable)
       // maintain the added column for schema evolution history
       var addColumnSchema: ColumnSchema = null
       var deletedColumnSchema: ColumnSchema = null

http://git-wip-us.apache.org/repos/asf/carbondata/blob/5a625f4c/integration/spark2/src/main/scala/org/apache/spark/sql/execution/command/schema/CarbonAlterTableDropColumnCommand.scala
----------------------------------------------------------------------
diff --git a/integration/spark2/src/main/scala/org/apache/spark/sql/execution/command/schema/CarbonAlterTableDropColumnCommand.scala b/integration/spark2/src/main/scala/org/apache/spark/sql/execution/command/schema/CarbonAlterTableDropColumnCommand.scala
index d848eb5..fcc1095 100644
--- a/integration/spark2/src/main/scala/org/apache/spark/sql/execution/command/schema/CarbonAlterTableDropColumnCommand.scala
+++ b/integration/spark2/src/main/scala/org/apache/spark/sql/execution/command/schema/CarbonAlterTableDropColumnCommand.scala
@@ -99,7 +99,7 @@ private[sql] case class CarbonAlterTableDropColumnCommand(
 
       // read the latest schema file
       val tableInfo: org.apache.carbondata.format.TableInfo =
-        metastore.getThriftTableInfo(carbonTable)(sparkSession)
+        metastore.getThriftTableInfo(carbonTable)
       // maintain the deleted columns for schema evolution history
       var deletedColumnSchema = ListBuffer[org.apache.carbondata.format.ColumnSchema]()
       val columnSchemaList = tableInfo.fact_table.table_columns.asScala

http://git-wip-us.apache.org/repos/asf/carbondata/blob/5a625f4c/integration/spark2/src/main/scala/org/apache/spark/sql/execution/command/schema/CarbonAlterTableRenameCommand.scala
----------------------------------------------------------------------
diff --git a/integration/spark2/src/main/scala/org/apache/spark/sql/execution/command/schema/CarbonAlterTableRenameCommand.scala b/integration/spark2/src/main/scala/org/apache/spark/sql/execution/command/schema/CarbonAlterTableRenameCommand.scala
index fc780cb..cda78ce 100644
--- a/integration/spark2/src/main/scala/org/apache/spark/sql/execution/command/schema/CarbonAlterTableRenameCommand.scala
+++ b/integration/spark2/src/main/scala/org/apache/spark/sql/execution/command/schema/CarbonAlterTableRenameCommand.scala
@@ -108,7 +108,7 @@ private[sql] case class CarbonAlterTableRenameCommand(
         sparkSession)
       OperationListenerBus.getInstance().fireEvent(alterTableRenamePreEvent, operationContext)
       val tableInfo: org.apache.carbondata.format.TableInfo =
-        metastore.getThriftTableInfo(carbonTable)(sparkSession)
+        metastore.getThriftTableInfo(carbonTable)
       val schemaEvolutionEntry = new SchemaEvolutionEntry(System.currentTimeMillis)
       schemaEvolutionEntry.setTableName(newTableName)
       timeStamp = System.currentTimeMillis()

http://git-wip-us.apache.org/repos/asf/carbondata/blob/5a625f4c/integration/spark2/src/main/scala/org/apache/spark/sql/execution/command/timeseries/TimeSeriesUtil.scala
----------------------------------------------------------------------
diff --git a/integration/spark2/src/main/scala/org/apache/spark/sql/execution/command/timeseries/TimeSeriesUtil.scala b/integration/spark2/src/main/scala/org/apache/spark/sql/execution/command/timeseries/TimeSeriesUtil.scala
index 9e0cee5..e67a98f 100644
--- a/integration/spark2/src/main/scala/org/apache/spark/sql/execution/command/timeseries/TimeSeriesUtil.scala
+++ b/integration/spark2/src/main/scala/org/apache/spark/sql/execution/command/timeseries/TimeSeriesUtil.scala
@@ -63,13 +63,13 @@ object TimeSeriesUtil {
    * @return whether find  only one granularity
    */
   def validateTimeSeriesGranularity(
-      dmProperties: Map[String, String],
+      dmProperties: java.util.Map[String, String],
       dmClassName: String): Boolean = {
     var isFound = false
 
     // 1. granularity only support one
     for (granularity <- Granularity.values()) {
-      if (dmProperties.get(granularity.getName).isDefined) {
+      if (dmProperties.containsKey(granularity.getName)) {
         if (isFound) {
           throw new MalformedDataMapCommandException(
             s"Only one granularity level can be defined")
@@ -104,14 +104,14 @@ object TimeSeriesUtil {
    * @return key and value tuple
    */
   def getTimeSeriesGranularityDetails(
-      dmProperties: Map[String, String],
+      dmProperties: java.util.Map[String, String],
       dmClassName: String): (String, String) = {
 
     val defaultValue = "1"
     for (granularity <- Granularity.values()) {
-      if (dmProperties.get(granularity.getName).isDefined &&
-        dmProperties.get(granularity.getName).get.equalsIgnoreCase(defaultValue)) {
-        return (granularity.toString.toLowerCase, dmProperties.get(granularity.getName).get)
+      if (dmProperties.containsKey(granularity.getName) &&
+        dmProperties.get(granularity.getName).equalsIgnoreCase(defaultValue)) {
+        return (granularity.toString.toLowerCase, dmProperties.get(granularity.getName))
       }
     }
 

http://git-wip-us.apache.org/repos/asf/carbondata/blob/5a625f4c/integration/spark2/src/main/scala/org/apache/spark/sql/hive/CarbonFileMetastore.scala
----------------------------------------------------------------------
diff --git a/integration/spark2/src/main/scala/org/apache/spark/sql/hive/CarbonFileMetastore.scala b/integration/spark2/src/main/scala/org/apache/spark/sql/hive/CarbonFileMetastore.scala
index fd09e48..b3438a4 100644
--- a/integration/spark2/src/main/scala/org/apache/spark/sql/hive/CarbonFileMetastore.scala
+++ b/integration/spark2/src/main/scala/org/apache/spark/sql/hive/CarbonFileMetastore.scala
@@ -523,8 +523,7 @@ class CarbonFileMetastore extends CarbonMetaStore {
   override def listAllTables(sparkSession: SparkSession): Seq[CarbonTable] =
     metadata.carbonTables
 
-  override def getThriftTableInfo(carbonTable: CarbonTable)
-    (sparkSession: SparkSession): TableInfo = {
+  override def getThriftTableInfo(carbonTable: CarbonTable): TableInfo = {
     val tableMetadataFile = CarbonTablePath.getSchemaFilePath(carbonTable.getTablePath)
     CarbonUtil.readSchemaFile(tableMetadataFile)
   }

http://git-wip-us.apache.org/repos/asf/carbondata/blob/5a625f4c/integration/spark2/src/main/scala/org/apache/spark/sql/hive/CarbonHiveMetaStore.scala
----------------------------------------------------------------------
diff --git a/integration/spark2/src/main/scala/org/apache/spark/sql/hive/CarbonHiveMetaStore.scala b/integration/spark2/src/main/scala/org/apache/spark/sql/hive/CarbonHiveMetaStore.scala
index 44f731e..4c40fee 100644
--- a/integration/spark2/src/main/scala/org/apache/spark/sql/hive/CarbonHiveMetaStore.scala
+++ b/integration/spark2/src/main/scala/org/apache/spark/sql/hive/CarbonHiveMetaStore.scala
@@ -96,8 +96,7 @@ class CarbonHiveMetaStore extends CarbonFileMetastore {
     Seq()
   }
 
-  override def getThriftTableInfo(carbonTable: CarbonTable)
-    (sparkSession: SparkSession): format.TableInfo = {
+  override def getThriftTableInfo(carbonTable: CarbonTable): format.TableInfo = {
     val schemaConverter = new ThriftWrapperSchemaConverterImpl
     schemaConverter.fromWrapperToExternalTableInfo(carbonTable.getTableInfo,
       carbonTable.getDatabaseName,

http://git-wip-us.apache.org/repos/asf/carbondata/blob/5a625f4c/integration/spark2/src/main/scala/org/apache/spark/sql/hive/CarbonMetaStore.scala
----------------------------------------------------------------------
diff --git a/integration/spark2/src/main/scala/org/apache/spark/sql/hive/CarbonMetaStore.scala b/integration/spark2/src/main/scala/org/apache/spark/sql/hive/CarbonMetaStore.scala
index 0645040..c2333f9 100644
--- a/integration/spark2/src/main/scala/org/apache/spark/sql/hive/CarbonMetaStore.scala
+++ b/integration/spark2/src/main/scala/org/apache/spark/sql/hive/CarbonMetaStore.scala
@@ -143,7 +143,7 @@ trait CarbonMetaStore {
 
   def listAllTables(sparkSession: SparkSession): Seq[CarbonTable]
 
-  def getThriftTableInfo(carbonTable: CarbonTable)(sparkSession: SparkSession): TableInfo
+  def getThriftTableInfo(carbonTable: CarbonTable): TableInfo
 
   def getTableFromMetadataCache(database: String, tableName: String): Option[CarbonTable]
 

http://git-wip-us.apache.org/repos/asf/carbondata/blob/5a625f4c/integration/spark2/src/main/scala/org/apache/spark/util/AlterTableUtil.scala
----------------------------------------------------------------------
diff --git a/integration/spark2/src/main/scala/org/apache/spark/util/AlterTableUtil.scala b/integration/spark2/src/main/scala/org/apache/spark/util/AlterTableUtil.scala
index aaa87a3..9c2c7e7 100644
--- a/integration/spark2/src/main/scala/org/apache/spark/util/AlterTableUtil.scala
+++ b/integration/spark2/src/main/scala/org/apache/spark/util/AlterTableUtil.scala
@@ -192,7 +192,7 @@ object AlterTableUtil {
     val metastore = CarbonEnv.getInstance(sparkSession).carbonMetastore
     val fileType = FileFactory.getFileType(tablePath)
     if (FileFactory.isFileExist(tablePath, fileType)) {
-      val tableInfo = metastore.getThriftTableInfo(oldCarbonTable)(sparkSession)
+      val tableInfo = metastore.getThriftTableInfo(oldCarbonTable)
       val evolutionEntryList = tableInfo.fact_table.schema_evolution.schema_evolution_history
       val updatedTime = evolutionEntryList.get(evolutionEntryList.size() - 1).time_stamp
       if (updatedTime == timeStamp) {
@@ -221,7 +221,7 @@ object AlterTableUtil {
     (sparkSession: SparkSession): Unit = {
     val metastore = CarbonEnv.getInstance(sparkSession).carbonMetastore
     val carbonTable = CarbonEnv.getCarbonTable(Some(dbName), tableName)(sparkSession)
-    val thriftTable: TableInfo = metastore.getThriftTableInfo(carbonTable)(sparkSession)
+    val thriftTable: TableInfo = metastore.getThriftTableInfo(carbonTable)
     val evolutionEntryList = thriftTable.fact_table.schema_evolution.schema_evolution_history
     val updatedTime = evolutionEntryList.get(evolutionEntryList.size() - 1).time_stamp
     if (updatedTime == timeStamp) {
@@ -246,7 +246,7 @@ object AlterTableUtil {
     (sparkSession: SparkSession): Unit = {
     val metastore = CarbonEnv.getInstance(sparkSession).carbonMetastore
     val carbonTable = CarbonEnv.getCarbonTable(Some(dbName), tableName)(sparkSession)
-    val thriftTable: TableInfo = metastore.getThriftTableInfo(carbonTable)(sparkSession)
+    val thriftTable: TableInfo = metastore.getThriftTableInfo(carbonTable)
     val evolutionEntryList = thriftTable.fact_table.schema_evolution.schema_evolution_history
     val updatedTime = evolutionEntryList.get(evolutionEntryList.size() - 1).time_stamp
     if (updatedTime == timeStamp) {
@@ -277,7 +277,7 @@ object AlterTableUtil {
     (sparkSession: SparkSession): Unit = {
     val metastore = CarbonEnv.getInstance(sparkSession).carbonMetastore
     val carbonTable = CarbonEnv.getCarbonTable(Some(dbName), tableName)(sparkSession)
-    val thriftTable: TableInfo = metastore.getThriftTableInfo(carbonTable)(sparkSession)
+    val thriftTable: TableInfo = metastore.getThriftTableInfo(carbonTable)
     val evolutionEntryList = thriftTable.fact_table.schema_evolution.schema_evolution_history
     val updatedTime = evolutionEntryList.get(evolutionEntryList.size() - 1).time_stamp
     if (updatedTime == timeStamp) {
@@ -326,7 +326,7 @@ object AlterTableUtil {
       carbonTable = CarbonEnv.getCarbonTable(Some(dbName), tableName)(sparkSession)
       // get the latest carbon table
       // read the latest schema file
-      val thriftTableInfo: TableInfo = metastore.getThriftTableInfo(carbonTable)(sparkSession)
+      val thriftTableInfo: TableInfo = metastore.getThriftTableInfo(carbonTable)
       val schemaConverter = new ThriftWrapperSchemaConverterImpl()
       val wrapperTableInfo = schemaConverter.fromExternalToWrapperTableInfo(
         thriftTableInfo,

http://git-wip-us.apache.org/repos/asf/carbondata/blob/5a625f4c/processing/src/main/java/org/apache/carbondata/processing/datamap/DataMapWriterListener.java
----------------------------------------------------------------------
diff --git a/processing/src/main/java/org/apache/carbondata/processing/datamap/DataMapWriterListener.java b/processing/src/main/java/org/apache/carbondata/processing/datamap/DataMapWriterListener.java
index e817590..f9baab3 100644
--- a/processing/src/main/java/org/apache/carbondata/processing/datamap/DataMapWriterListener.java
+++ b/processing/src/main/java/org/apache/carbondata/processing/datamap/DataMapWriterListener.java
@@ -30,13 +30,13 @@ import org.apache.carbondata.core.datamap.DataMapMeta;
 import org.apache.carbondata.core.datamap.DataMapStoreManager;
 import org.apache.carbondata.core.datamap.TableDataMap;
 import org.apache.carbondata.core.datamap.dev.AbstractDataMapWriter;
-import org.apache.carbondata.core.datamap.dev.DataMapFactory;
+import org.apache.carbondata.core.datamap.dev.IndexDataMapFactory;
 import org.apache.carbondata.core.datastore.page.ColumnPage;
 import org.apache.carbondata.core.metadata.schema.table.CarbonTable;
 import org.apache.carbondata.processing.store.TablePage;
 
 /**
- * It is for writing DataMap for one table
+ * It is for writing IndexDataMap for one table
  */
 public class DataMapWriterListener {
 
@@ -54,7 +54,7 @@ public class DataMapWriterListener {
     List<TableDataMap> tableDataMaps = DataMapStoreManager.getInstance().getAllDataMap(carbonTable);
     if (tableDataMaps != null) {
       for (TableDataMap tableDataMap : tableDataMaps) {
-        DataMapFactory factory = tableDataMap.getDataMapFactory();
+        IndexDataMapFactory factory = tableDataMap.getIndexDataMapFactory();
         register(factory, segmentId, dataWritePath);
       }
     }
@@ -63,7 +63,7 @@ public class DataMapWriterListener {
   /**
    * Register a AbstractDataMapWriter
    */
-  private void register(DataMapFactory factory, String segmentId, String dataWritePath) {
+  private void register(IndexDataMapFactory factory, String segmentId, String dataWritePath) {
     assert (factory != null);
     assert (segmentId != null);
     DataMapMeta meta = factory.getMeta();


[5/6] carbondata git commit: [CARBONDATA-2189] Add DataMapProvider developer interface

Posted by ja...@apache.org.
http://git-wip-us.apache.org/repos/asf/carbondata/blob/5a625f4c/core/src/main/java/org/apache/carbondata/core/indexstore/blockletindex/BlockletDataMap.java
----------------------------------------------------------------------
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/BlockletDataMap.java
deleted file mode 100644
index aceb372..0000000
--- a/core/src/main/java/org/apache/carbondata/core/indexstore/blockletindex/BlockletDataMap.java
+++ /dev/null
@@ -1,981 +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.indexstore.blockletindex;
-
-import java.io.ByteArrayInputStream;
-import java.io.ByteArrayOutputStream;
-import java.io.DataInputStream;
-import java.io.DataOutput;
-import java.io.DataOutputStream;
-import java.io.IOException;
-import java.io.UnsupportedEncodingException;
-import java.math.BigDecimal;
-import java.nio.ByteBuffer;
-import java.util.ArrayList;
-import java.util.Arrays;
-import java.util.BitSet;
-import java.util.Comparator;
-import java.util.List;
-
-import org.apache.carbondata.common.logging.LogService;
-import org.apache.carbondata.common.logging.LogServiceFactory;
-import org.apache.carbondata.core.cache.Cacheable;
-import org.apache.carbondata.core.constants.CarbonCommonConstants;
-import org.apache.carbondata.core.datamap.dev.DataMapModel;
-import org.apache.carbondata.core.datamap.dev.cgdatamap.AbstractCoarseGrainDataMap;
-import org.apache.carbondata.core.datastore.IndexKey;
-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.Blocklet;
-import org.apache.carbondata.core.indexstore.BlockletDetailInfo;
-import org.apache.carbondata.core.indexstore.ExtendedBlocklet;
-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.schema.CarbonRowSchema;
-import org.apache.carbondata.core.keygenerator.KeyGenException;
-import org.apache.carbondata.core.memory.MemoryException;
-import org.apache.carbondata.core.metadata.blocklet.BlockletInfo;
-import org.apache.carbondata.core.metadata.blocklet.DataFileFooter;
-import org.apache.carbondata.core.metadata.blocklet.index.BlockletIndex;
-import org.apache.carbondata.core.metadata.blocklet.index.BlockletMinMaxIndex;
-import org.apache.carbondata.core.metadata.datatype.DataType;
-import org.apache.carbondata.core.metadata.datatype.DataTypes;
-import org.apache.carbondata.core.metadata.schema.table.column.CarbonMeasure;
-import org.apache.carbondata.core.metadata.schema.table.column.ColumnSchema;
-import org.apache.carbondata.core.scan.filter.FilterExpressionProcessor;
-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.ByteUtil;
-import org.apache.carbondata.core.util.DataFileFooterConverter;
-import org.apache.carbondata.core.util.DataTypeUtil;
-
-import org.apache.commons.lang3.StringUtils;
-import org.xerial.snappy.Snappy;
-
-/**
- * Datamap implementation for blocklet.
- */
-public class BlockletDataMap extends AbstractCoarseGrainDataMap implements Cacheable {
-
-  private static final LogService LOGGER =
-      LogServiceFactory.getLogService(BlockletDataMap.class.getName());
-
-  private static int KEY_INDEX = 0;
-
-  private static int MIN_VALUES_INDEX = 1;
-
-  private static int MAX_VALUES_INDEX = 2;
-
-  private static int ROW_COUNT_INDEX = 3;
-
-  private static int FILE_PATH_INDEX = 4;
-
-  private static int PAGE_COUNT_INDEX = 5;
-
-  private static int VERSION_INDEX = 6;
-
-  private static int SCHEMA_UPADATED_TIME_INDEX = 7;
-
-  private static int BLOCK_INFO_INDEX = 8;
-
-  private static int BLOCK_FOOTER_OFFSET = 9;
-
-  private static int LOCATIONS = 10;
-
-  private static int BLOCKLET_ID_INDEX = 11;
-
-  private static int BLOCK_LENGTH = 12;
-
-  private static int TASK_MIN_VALUES_INDEX = 0;
-
-  private static int TASK_MAX_VALUES_INDEX = 1;
-
-  private static int SCHEMA = 2;
-
-  private static int PARTITION_INFO = 3;
-
-  private UnsafeMemoryDMStore unsafeMemoryDMStore;
-
-  private UnsafeMemoryDMStore unsafeMemorySummaryDMStore;
-
-  private SegmentProperties segmentProperties;
-
-  private int[] columnCardinality;
-
-  private boolean isPartitionedSegment;
-
-  @Override
-  public void init(DataMapModel dataMapModel) throws IOException, MemoryException {
-    long startTime = System.currentTimeMillis();
-    assert (dataMapModel instanceof BlockletDataMapModel);
-    BlockletDataMapModel blockletDataMapInfo = (BlockletDataMapModel) dataMapModel;
-    DataFileFooterConverter fileFooterConverter = new DataFileFooterConverter();
-    List<DataFileFooter> indexInfo = fileFooterConverter
-        .getIndexInfo(blockletDataMapInfo.getFilePath(), blockletDataMapInfo.getFileData());
-    isPartitionedSegment = blockletDataMapInfo.isPartitionedSegment();
-    DataMapRowImpl summaryRow = null;
-    byte[] schemaBinary = null;
-    // below 2 variables will be used for fetching the relative blocklet id. Relative blocklet ID
-    // is id assigned to a blocklet within a part file
-    String tempFilePath = null;
-    int relativeBlockletId = 0;
-    for (DataFileFooter fileFooter : indexInfo) {
-      if (segmentProperties == null) {
-        List<ColumnSchema> columnInTable = fileFooter.getColumnInTable();
-        schemaBinary = convertSchemaToBinary(columnInTable);
-        columnCardinality = fileFooter.getSegmentInfo().getColumnCardinality();
-        segmentProperties = new SegmentProperties(columnInTable, columnCardinality);
-        createSchema(segmentProperties);
-        createSummarySchema(segmentProperties, blockletDataMapInfo.getPartitions(), schemaBinary);
-      }
-      TableBlockInfo blockInfo = fileFooter.getBlockInfo().getTableBlockInfo();
-      BlockMetaInfo blockMetaInfo =
-          blockletDataMapInfo.getBlockMetaInfoMap().get(blockInfo.getFilePath());
-      // Here it loads info about all blocklets of index
-      // Only add if the file exists physically. There are scenarios which index file exists inside
-      // merge index but related carbondata files are deleted. In that case we first check whether
-      // the file exists physically or not
-      if (blockMetaInfo != null) {
-        if (fileFooter.getBlockletList() == null) {
-          // This is old store scenario, here blocklet information is not available in index file so
-          // load only block info
-          summaryRow =
-              loadToUnsafeBlock(fileFooter, segmentProperties, blockInfo.getFilePath(), summaryRow,
-                  blockMetaInfo);
-        } else {
-          // blocklet ID will start from 0 again only when part file path is changed
-          if (null == tempFilePath || !tempFilePath.equals(blockInfo.getFilePath())) {
-            tempFilePath = blockInfo.getFilePath();
-            relativeBlockletId = 0;
-          }
-          summaryRow =
-              loadToUnsafe(fileFooter, segmentProperties, blockInfo.getFilePath(), summaryRow,
-                  blockMetaInfo, relativeBlockletId);
-          // this is done because relative blocklet id need to be incremented based on the
-          // total number of blocklets
-          relativeBlockletId += fileFooter.getBlockletList().size();
-        }
-      }
-    }
-    if (unsafeMemoryDMStore != null) {
-      unsafeMemoryDMStore.finishWriting();
-    }
-    if (null != unsafeMemorySummaryDMStore) {
-      addTaskSummaryRowToUnsafeMemoryStore(
-          summaryRow,
-          blockletDataMapInfo.getPartitions(),
-          schemaBinary);
-      unsafeMemorySummaryDMStore.finishWriting();
-    }
-    LOGGER.info(
-        "Time taken to load blocklet datamap from file : " + dataMapModel.getFilePath() + "is " + (
-            System.currentTimeMillis() - startTime));
-  }
-
-  private DataMapRowImpl loadToUnsafe(DataFileFooter fileFooter,
-      SegmentProperties segmentProperties, String filePath, DataMapRowImpl summaryRow,
-      BlockMetaInfo blockMetaInfo, int relativeBlockletId) {
-    int[] minMaxLen = segmentProperties.getColumnsValueSize();
-    List<BlockletInfo> blockletList = fileFooter.getBlockletList();
-    CarbonRowSchema[] schema = unsafeMemoryDMStore.getSchema();
-    // Add one row to maintain task level min max for segment pruning
-    if (!blockletList.isEmpty() && summaryRow == null) {
-      summaryRow = new DataMapRowImpl(unsafeMemorySummaryDMStore.getSchema());
-    }
-    for (int index = 0; index < blockletList.size(); index++) {
-      DataMapRow row = new DataMapRowImpl(schema);
-      int ordinal = 0;
-      int taskMinMaxOrdinal = 0;
-      BlockletInfo blockletInfo = blockletList.get(index);
-
-      // add start key as index key
-      row.setByteArray(blockletInfo.getBlockletIndex().getBtreeIndex().getStartKey(), ordinal++);
-
-      BlockletMinMaxIndex minMaxIndex = blockletInfo.getBlockletIndex().getMinMaxIndex();
-      byte[][] minValues = updateMinValues(minMaxIndex.getMinValues(), minMaxLen);
-      row.setRow(addMinMax(minMaxLen, schema[ordinal], minValues), ordinal);
-      // compute and set task level min values
-      addTaskMinMaxValues(summaryRow, minMaxLen,
-          unsafeMemorySummaryDMStore.getSchema()[taskMinMaxOrdinal], minValues,
-          TASK_MIN_VALUES_INDEX, true);
-      ordinal++;
-      taskMinMaxOrdinal++;
-      byte[][] maxValues = updateMaxValues(minMaxIndex.getMaxValues(), minMaxLen);
-      row.setRow(addMinMax(minMaxLen, schema[ordinal], maxValues), ordinal);
-      // compute and set task level max values
-      addTaskMinMaxValues(summaryRow, minMaxLen,
-          unsafeMemorySummaryDMStore.getSchema()[taskMinMaxOrdinal], maxValues,
-          TASK_MAX_VALUES_INDEX, false);
-      ordinal++;
-
-      row.setInt(blockletInfo.getNumberOfRows(), ordinal++);
-
-      // add file path
-      byte[] filePathBytes = filePath.getBytes(CarbonCommonConstants.DEFAULT_CHARSET_CLASS);
-      row.setByteArray(filePathBytes, ordinal++);
-
-      // add pages
-      row.setShort((short) blockletInfo.getNumberOfPages(), ordinal++);
-
-      // add version number
-      row.setShort(fileFooter.getVersionId().number(), ordinal++);
-
-      // add schema updated time
-      row.setLong(fileFooter.getSchemaUpdatedTimeStamp(), ordinal++);
-
-      // add blocklet info
-      byte[] serializedData;
-      try {
-        ByteArrayOutputStream stream = new ByteArrayOutputStream();
-        DataOutput dataOutput = new DataOutputStream(stream);
-        blockletInfo.write(dataOutput);
-        serializedData = stream.toByteArray();
-        row.setByteArray(serializedData, ordinal++);
-        // Add block footer offset, it is used if we need to read footer of block
-        row.setLong(fileFooter.getBlockInfo().getTableBlockInfo().getBlockOffset(), ordinal++);
-        setLocations(blockMetaInfo.getLocationInfo(), row, ordinal);
-        ordinal++;
-        // for relative blockelt id i.e blocklet id that belongs to a particular part file
-        row.setShort((short) relativeBlockletId++, ordinal++);
-        // Store block size
-        row.setLong(blockMetaInfo.getSize(), ordinal);
-        unsafeMemoryDMStore.addIndexRowToUnsafe(row);
-      } catch (Exception e) {
-        throw new RuntimeException(e);
-      }
-    }
-
-    return summaryRow;
-  }
-
-  private void setLocations(String[] locations, DataMapRow row, int ordinal)
-      throws UnsupportedEncodingException {
-    // Add location info
-    String locationStr = StringUtils.join(locations, ',');
-    row.setByteArray(locationStr.getBytes(CarbonCommonConstants.DEFAULT_CHARSET), ordinal);
-  }
-
-  /**
-   * Load information for the block.It is the case can happen only for old stores
-   * where blocklet information is not available in index file. So load only block information
-   * and read blocklet information in executor.
-   */
-  private DataMapRowImpl loadToUnsafeBlock(DataFileFooter fileFooter,
-      SegmentProperties segmentProperties, String filePath, DataMapRowImpl summaryRow,
-      BlockMetaInfo blockMetaInfo) {
-    int[] minMaxLen = segmentProperties.getColumnsValueSize();
-    BlockletIndex blockletIndex = fileFooter.getBlockletIndex();
-    CarbonRowSchema[] schema = unsafeMemoryDMStore.getSchema();
-    // Add one row to maintain task level min max for segment pruning
-    if (summaryRow == null) {
-      summaryRow = new DataMapRowImpl(unsafeMemorySummaryDMStore.getSchema());
-    }
-    DataMapRow row = new DataMapRowImpl(schema);
-    int ordinal = 0;
-    int taskMinMaxOrdinal = 0;
-    // add start key as index key
-    row.setByteArray(blockletIndex.getBtreeIndex().getStartKey(), ordinal++);
-
-    BlockletMinMaxIndex minMaxIndex = blockletIndex.getMinMaxIndex();
-    byte[][] minValues = updateMinValues(minMaxIndex.getMinValues(), minMaxLen);
-    row.setRow(addMinMax(minMaxLen, schema[ordinal], minValues), ordinal);
-    // compute and set task level min values
-    addTaskMinMaxValues(summaryRow, minMaxLen,
-        unsafeMemorySummaryDMStore.getSchema()[taskMinMaxOrdinal], minValues,
-        TASK_MIN_VALUES_INDEX, true);
-    ordinal++;
-    taskMinMaxOrdinal++;
-    byte[][] maxValues = updateMaxValues(minMaxIndex.getMaxValues(), minMaxLen);
-    row.setRow(addMinMax(minMaxLen, schema[ordinal], maxValues), ordinal);
-    // compute and set task level max values
-    addTaskMinMaxValues(summaryRow, minMaxLen,
-        unsafeMemorySummaryDMStore.getSchema()[taskMinMaxOrdinal], maxValues,
-        TASK_MAX_VALUES_INDEX, false);
-    ordinal++;
-
-    row.setInt((int)fileFooter.getNumberOfRows(), ordinal++);
-
-    // add file path
-    byte[] filePathBytes = filePath.getBytes(CarbonCommonConstants.DEFAULT_CHARSET_CLASS);
-    row.setByteArray(filePathBytes, ordinal++);
-
-    // add pages
-    row.setShort((short) 0, ordinal++);
-
-    // add version number
-    row.setShort(fileFooter.getVersionId().number(), ordinal++);
-
-    // add schema updated time
-    row.setLong(fileFooter.getSchemaUpdatedTimeStamp(), ordinal++);
-
-    // add blocklet info
-    row.setByteArray(new byte[0], ordinal++);
-
-    row.setLong(fileFooter.getBlockInfo().getTableBlockInfo().getBlockOffset(), ordinal++);
-    try {
-      setLocations(blockMetaInfo.getLocationInfo(), row, ordinal);
-      ordinal++;
-      // for relative blocklet id. Value is -1 because in case of old store blocklet info will
-      // not be present in the index file and in that case we will not knwo the total number of
-      // blocklets
-      row.setShort((short) -1, ordinal++);
-
-      // store block size
-      row.setLong(blockMetaInfo.getSize(), ordinal);
-      unsafeMemoryDMStore.addIndexRowToUnsafe(row);
-    } catch (Exception e) {
-      throw new RuntimeException(e);
-    }
-
-    return summaryRow;
-  }
-
-  private void addTaskSummaryRowToUnsafeMemoryStore(DataMapRow summaryRow,
-      List<String> partitions, byte[] schemaBinary) throws IOException {
-    // write the task summary info to unsafe memory store
-    if (null != summaryRow) {
-      // Add column schema , it is useful to generate segment properties in executor.
-      // So we no need to read footer again there.
-      if (schemaBinary != null) {
-        summaryRow.setByteArray(schemaBinary, SCHEMA);
-      }
-      if (partitions != null && partitions.size() > 0) {
-        CarbonRowSchema[] minSchemas =
-            ((CarbonRowSchema.StructCarbonRowSchema) unsafeMemorySummaryDMStore
-                .getSchema()[PARTITION_INFO]).getChildSchemas();
-        DataMapRow partitionRow = new DataMapRowImpl(minSchemas);
-        for (int i = 0; i < partitions.size(); i++) {
-          partitionRow
-              .setByteArray(partitions.get(i).getBytes(CarbonCommonConstants.DEFAULT_CHARSET_CLASS),
-                  i);
-        }
-        summaryRow.setRow(partitionRow, PARTITION_INFO);
-      }
-      try {
-        unsafeMemorySummaryDMStore.addIndexRowToUnsafe(summaryRow);
-      } catch (Exception e) {
-        throw new RuntimeException(e);
-      }
-    }
-  }
-
-  /**
-   * Fill the measures min values with minimum , this is needed for backward version compatability
-   * as older versions don't store min values for measures
-   */
-  private byte[][] updateMinValues(byte[][] minValues, int[] minMaxLen) {
-    byte[][] updatedValues = minValues;
-    if (minValues.length < minMaxLen.length) {
-      updatedValues = new byte[minMaxLen.length][];
-      System.arraycopy(minValues, 0, updatedValues, 0, minValues.length);
-      List<CarbonMeasure> measures = segmentProperties.getMeasures();
-      ByteBuffer buffer = ByteBuffer.allocate(8);
-      for (int i = 0; i < measures.size(); i++) {
-        buffer.rewind();
-        DataType dataType = measures.get(i).getDataType();
-        if (dataType == DataTypes.BYTE) {
-          buffer.putLong(Byte.MIN_VALUE);
-          updatedValues[minValues.length + i] = buffer.array().clone();
-        } else if (dataType == DataTypes.SHORT) {
-          buffer.putLong(Short.MIN_VALUE);
-          updatedValues[minValues.length + i] = buffer.array().clone();
-        } else if (dataType == DataTypes.INT) {
-          buffer.putLong(Integer.MIN_VALUE);
-          updatedValues[minValues.length + i] = buffer.array().clone();
-        } else if (dataType == DataTypes.LONG) {
-          buffer.putLong(Long.MIN_VALUE);
-          updatedValues[minValues.length + i] = buffer.array().clone();
-        } else if (DataTypes.isDecimal(dataType)) {
-          updatedValues[minValues.length + i] =
-              DataTypeUtil.bigDecimalToByte(BigDecimal.valueOf(Long.MIN_VALUE));
-        } else {
-          buffer.putDouble(Double.MIN_VALUE);
-          updatedValues[minValues.length + i] = buffer.array().clone();
-        }
-      }
-    }
-    return updatedValues;
-  }
-
-  /**
-   * Fill the measures max values with maximum , this is needed for backward version compatability
-   * as older versions don't store max values for measures
-   */
-  private byte[][] updateMaxValues(byte[][] maxValues, int[] minMaxLen) {
-    byte[][] updatedValues = maxValues;
-    if (maxValues.length < minMaxLen.length) {
-      updatedValues = new byte[minMaxLen.length][];
-      System.arraycopy(maxValues, 0, updatedValues, 0, maxValues.length);
-      List<CarbonMeasure> measures = segmentProperties.getMeasures();
-      ByteBuffer buffer = ByteBuffer.allocate(8);
-      for (int i = 0; i < measures.size(); i++) {
-        buffer.rewind();
-        DataType dataType = measures.get(i).getDataType();
-        if (dataType == DataTypes.BYTE) {
-          buffer.putLong(Byte.MAX_VALUE);
-          updatedValues[maxValues.length + i] = buffer.array().clone();
-        } else if (dataType == DataTypes.SHORT) {
-          buffer.putLong(Short.MAX_VALUE);
-          updatedValues[maxValues.length + i] = buffer.array().clone();
-        } else if (dataType == DataTypes.INT) {
-          buffer.putLong(Integer.MAX_VALUE);
-          updatedValues[maxValues.length + i] = buffer.array().clone();
-        } else if (dataType == DataTypes.LONG) {
-          buffer.putLong(Long.MAX_VALUE);
-          updatedValues[maxValues.length + i] = buffer.array().clone();
-        } else if (DataTypes.isDecimal(dataType)) {
-          updatedValues[maxValues.length + i] =
-              DataTypeUtil.bigDecimalToByte(BigDecimal.valueOf(Long.MAX_VALUE));
-        } else {
-          buffer.putDouble(Double.MAX_VALUE);
-          updatedValues[maxValues.length + i] = buffer.array().clone();
-        }
-      }
-    }
-    return updatedValues;
-  }
-
-  private DataMapRow addMinMax(int[] minMaxLen, CarbonRowSchema carbonRowSchema,
-      byte[][] minValues) {
-    CarbonRowSchema[] minSchemas =
-        ((CarbonRowSchema.StructCarbonRowSchema) carbonRowSchema).getChildSchemas();
-    DataMapRow minRow = new DataMapRowImpl(minSchemas);
-    int minOrdinal = 0;
-    // min value adding
-    for (int i = 0; i < minMaxLen.length; i++) {
-      minRow.setByteArray(minValues[i], minOrdinal++);
-    }
-    return minRow;
-  }
-
-  /**
-   * This method will compute min/max values at task level
-   *
-   * @param taskMinMaxRow
-   * @param minMaxLen
-   * @param carbonRowSchema
-   * @param minMaxValue
-   * @param ordinal
-   * @param isMinValueComparison
-   */
-  private void addTaskMinMaxValues(DataMapRow taskMinMaxRow, int[] minMaxLen,
-      CarbonRowSchema carbonRowSchema, byte[][] minMaxValue, int ordinal,
-      boolean isMinValueComparison) {
-    DataMapRow row = taskMinMaxRow.getRow(ordinal);
-    byte[][] updatedMinMaxValues = minMaxValue;
-    if (null == row) {
-      CarbonRowSchema[] minSchemas =
-          ((CarbonRowSchema.StructCarbonRowSchema) carbonRowSchema).getChildSchemas();
-      row = new DataMapRowImpl(minSchemas);
-    } else {
-      byte[][] existingMinMaxValues = getMinMaxValue(taskMinMaxRow, ordinal);
-      // Compare and update min max values
-      for (int i = 0; i < minMaxLen.length; i++) {
-        int compare =
-            ByteUtil.UnsafeComparer.INSTANCE.compareTo(existingMinMaxValues[i], minMaxValue[i]);
-        if (isMinValueComparison) {
-          if (compare < 0) {
-            updatedMinMaxValues[i] = existingMinMaxValues[i];
-          }
-        } else if (compare > 0) {
-          updatedMinMaxValues[i] = existingMinMaxValues[i];
-        }
-      }
-    }
-    int minMaxOrdinal = 0;
-    // min/max value adding
-    for (int i = 0; i < minMaxLen.length; i++) {
-      row.setByteArray(updatedMinMaxValues[i], minMaxOrdinal++);
-    }
-    taskMinMaxRow.setRow(row, ordinal);
-  }
-
-  private void createSchema(SegmentProperties segmentProperties) throws MemoryException {
-    List<CarbonRowSchema> indexSchemas = new ArrayList<>();
-
-    // Index key
-    indexSchemas.add(new CarbonRowSchema.VariableCarbonRowSchema(DataTypes.BYTE_ARRAY));
-    getMinMaxSchema(segmentProperties, indexSchemas);
-
-    // for number of rows.
-    indexSchemas.add(new CarbonRowSchema.FixedCarbonRowSchema(DataTypes.INT));
-
-    // for table block path
-    indexSchemas.add(new CarbonRowSchema.VariableCarbonRowSchema(DataTypes.BYTE_ARRAY));
-
-    // for number of pages.
-    indexSchemas.add(new CarbonRowSchema.FixedCarbonRowSchema(DataTypes.SHORT));
-
-    // for version number.
-    indexSchemas.add(new CarbonRowSchema.FixedCarbonRowSchema(DataTypes.SHORT));
-
-    // for schema updated time.
-    indexSchemas.add(new CarbonRowSchema.FixedCarbonRowSchema(DataTypes.LONG));
-
-    //for blocklet info
-    indexSchemas.add(new CarbonRowSchema.VariableCarbonRowSchema(DataTypes.BYTE_ARRAY));
-
-    // for block footer offset.
-    indexSchemas.add(new CarbonRowSchema.FixedCarbonRowSchema(DataTypes.LONG));
-
-    // for locations
-    indexSchemas.add(new CarbonRowSchema.VariableCarbonRowSchema(DataTypes.BYTE_ARRAY));
-
-    // for relative blocklet id i.e. blocklet id that belongs to a particular part file.
-    indexSchemas.add(new CarbonRowSchema.FixedCarbonRowSchema(DataTypes.SHORT));
-
-    // for storing block length.
-    indexSchemas.add(new CarbonRowSchema.FixedCarbonRowSchema(DataTypes.LONG));
-
-    unsafeMemoryDMStore =
-        new UnsafeMemoryDMStore(indexSchemas.toArray(new CarbonRowSchema[indexSchemas.size()]));
-  }
-
-  /**
-   * Creates the schema to store summary information or the information which can be stored only
-   * once per datamap. It stores datamap level max/min of each column and partition information of
-   * datamap
-   * @param segmentProperties
-   * @param partitions
-   * @throws MemoryException
-   */
-  private void createSummarySchema(SegmentProperties segmentProperties, List<String> partitions,
-      byte[] schemaBinary)
-      throws MemoryException {
-    List<CarbonRowSchema> taskMinMaxSchemas = new ArrayList<>();
-    getMinMaxSchema(segmentProperties, taskMinMaxSchemas);
-    // for storing column schema
-    taskMinMaxSchemas.add(
-        new CarbonRowSchema.FixedCarbonRowSchema(DataTypes.BYTE_ARRAY, schemaBinary.length));
-    if (partitions != null && partitions.size() > 0) {
-      CarbonRowSchema[] mapSchemas = new CarbonRowSchema[partitions.size()];
-      for (int i = 0; i < mapSchemas.length; i++) {
-        mapSchemas[i] = new CarbonRowSchema.VariableCarbonRowSchema(DataTypes.BYTE_ARRAY);
-      }
-      CarbonRowSchema mapSchema =
-          new CarbonRowSchema.StructCarbonRowSchema(DataTypes.createDefaultStructType(),
-              mapSchemas);
-      taskMinMaxSchemas.add(mapSchema);
-    }
-    unsafeMemorySummaryDMStore = new UnsafeMemoryDMStore(
-        taskMinMaxSchemas.toArray(new CarbonRowSchema[taskMinMaxSchemas.size()]));
-  }
-
-  private void getMinMaxSchema(SegmentProperties segmentProperties,
-      List<CarbonRowSchema> minMaxSchemas) {
-    // Index key
-    int[] minMaxLen = segmentProperties.getColumnsValueSize();
-    // do it 2 times, one for min and one for max.
-    for (int k = 0; k < 2; k++) {
-      CarbonRowSchema[] mapSchemas = new CarbonRowSchema[minMaxLen.length];
-      for (int i = 0; i < minMaxLen.length; i++) {
-        if (minMaxLen[i] <= 0) {
-          mapSchemas[i] = new CarbonRowSchema.VariableCarbonRowSchema(DataTypes.BYTE_ARRAY);
-        } else {
-          mapSchemas[i] =
-              new CarbonRowSchema.FixedCarbonRowSchema(DataTypes.BYTE_ARRAY, minMaxLen[i]);
-        }
-      }
-      CarbonRowSchema mapSchema =
-          new CarbonRowSchema.StructCarbonRowSchema(DataTypes.createDefaultStructType(),
-              mapSchemas);
-      minMaxSchemas.add(mapSchema);
-    }
-  }
-
-  @Override
-  public boolean isScanRequired(FilterResolverIntf filterExp) {
-    FilterExecuter filterExecuter =
-        FilterUtil.getFilterExecuterTree(filterExp, segmentProperties, null);
-    for (int i = 0; i < unsafeMemorySummaryDMStore.getRowCount(); i++) {
-      DataMapRow unsafeRow = unsafeMemorySummaryDMStore.getUnsafeRow(i);
-      boolean isScanRequired = FilterExpressionProcessor.isScanRequired(
-          filterExecuter, getMinMaxValue(unsafeRow, TASK_MAX_VALUES_INDEX),
-          getMinMaxValue(unsafeRow, TASK_MIN_VALUES_INDEX));
-      if (isScanRequired) {
-        return true;
-      }
-    }
-    return false;
-  }
-
-  private List<Blocklet> prune(FilterResolverIntf filterExp, SegmentProperties segmentProperties) {
-    if (unsafeMemoryDMStore.getRowCount() == 0) {
-      return new ArrayList<>();
-    }
-    // getting the start and end index key based on filter for hitting the
-    // selected block reference nodes based on filter resolver tree.
-    if (LOGGER.isDebugEnabled()) {
-      LOGGER.debug("preparing the start and end key for finding"
-          + "start and end block as per filter resolver");
-    }
-    List<Blocklet> blocklets = new ArrayList<>();
-    Comparator<DataMapRow> comparator =
-        new BlockletDMComparator(segmentProperties.getColumnsValueSize(),
-            segmentProperties.getNumberOfSortColumns(),
-            segmentProperties.getNumberOfNoDictSortColumns());
-    List<IndexKey> listOfStartEndKeys = new ArrayList<IndexKey>(2);
-    FilterUtil
-        .traverseResolverTreeAndGetStartAndEndKey(segmentProperties, filterExp, listOfStartEndKeys);
-    // reading the first value from list which has start key
-    IndexKey searchStartKey = listOfStartEndKeys.get(0);
-    // reading the last value from list which has end key
-    IndexKey searchEndKey = listOfStartEndKeys.get(1);
-    if (null == searchStartKey && null == searchEndKey) {
-      try {
-        // TODO need to handle for no dictionary dimensions
-        searchStartKey = FilterUtil.prepareDefaultStartIndexKey(segmentProperties);
-        // TODO need to handle for no dictionary dimensions
-        searchEndKey = FilterUtil.prepareDefaultEndIndexKey(segmentProperties);
-      } catch (KeyGenException e) {
-        return null;
-      }
-    }
-    if (LOGGER.isDebugEnabled()) {
-      LOGGER.debug(
-          "Successfully retrieved the start and end key" + "Dictionary Start Key: " + Arrays
-              .toString(searchStartKey.getDictionaryKeys()) + "No Dictionary Start Key " + Arrays
-              .toString(searchStartKey.getNoDictionaryKeys()) + "Dictionary End Key: " + Arrays
-              .toString(searchEndKey.getDictionaryKeys()) + "No Dictionary End Key " + Arrays
-              .toString(searchEndKey.getNoDictionaryKeys()));
-    }
-    if (filterExp == null) {
-      int rowCount = unsafeMemoryDMStore.getRowCount();
-      for (int i = 0; i < rowCount; i++) {
-        DataMapRow safeRow = unsafeMemoryDMStore.getUnsafeRow(i).convertToSafeRow();
-        blocklets.add(createBlocklet(safeRow, safeRow.getShort(BLOCKLET_ID_INDEX)));
-      }
-    } else {
-      int startIndex = findStartIndex(convertToRow(searchStartKey), comparator);
-      int endIndex = findEndIndex(convertToRow(searchEndKey), comparator);
-      FilterExecuter filterExecuter =
-          FilterUtil.getFilterExecuterTree(filterExp, segmentProperties, null);
-      while (startIndex <= endIndex) {
-        DataMapRow safeRow = unsafeMemoryDMStore.getUnsafeRow(startIndex).convertToSafeRow();
-        int blockletId = safeRow.getShort(BLOCKLET_ID_INDEX);
-        String filePath = new String(safeRow.getByteArray(FILE_PATH_INDEX),
-            CarbonCommonConstants.DEFAULT_CHARSET_CLASS);
-        boolean isValid =
-            addBlockBasedOnMinMaxValue(filterExecuter, getMinMaxValue(safeRow, MAX_VALUES_INDEX),
-                getMinMaxValue(safeRow, MIN_VALUES_INDEX), filePath, blockletId);
-        if (isValid) {
-          blocklets.add(createBlocklet(safeRow, blockletId));
-        }
-        startIndex++;
-      }
-    }
-    return blocklets;
-  }
-
-  @Override
-  public List<Blocklet> prune(FilterResolverIntf filterExp, SegmentProperties segmentProperties,
-      List<String> partitions) {
-    if (unsafeMemoryDMStore.getRowCount() == 0) {
-      return new ArrayList<>();
-    }
-    // First get the partitions which are stored inside datamap.
-    List<String> storedPartitions = getPartitions();
-    // if it has partitioned datamap but there is no partitioned information stored, it means
-    // partitions are dropped so return empty list.
-    if (isPartitionedSegment && (storedPartitions == null || storedPartitions.size() == 0)) {
-      return new ArrayList<>();
-    }
-    if (storedPartitions != null && storedPartitions.size() > 0) {
-      // Check the exact match of partition information inside the stored partitions.
-      boolean found = false;
-      if (partitions != null && partitions.size() > 0) {
-        found = partitions.containsAll(storedPartitions);
-      }
-      if (!found) {
-        return new ArrayList<>();
-      }
-    }
-    // Prune with filters if the partitions are existed in this datamap
-    return prune(filterExp, segmentProperties);
-  }
-
-  /**
-   * select the blocks based on column min and max value
-   *
-   * @param filterExecuter
-   * @param maxValue
-   * @param minValue
-   * @param filePath
-   * @param blockletId
-   * @return
-   */
-  private boolean addBlockBasedOnMinMaxValue(FilterExecuter filterExecuter, byte[][] maxValue,
-      byte[][] minValue, String filePath, int blockletId) {
-    BitSet bitSet = null;
-    if (filterExecuter instanceof ImplicitColumnFilterExecutor) {
-      String uniqueBlockPath = filePath.substring(filePath.lastIndexOf("/Part") + 1);
-      // this case will come in case of old store where index file does not contain the
-      // blocklet information
-      if (blockletId != -1) {
-        uniqueBlockPath = uniqueBlockPath + CarbonCommonConstants.FILE_SEPARATOR + blockletId;
-      }
-      bitSet = ((ImplicitColumnFilterExecutor) filterExecuter)
-          .isFilterValuesPresentInBlockOrBlocklet(maxValue, minValue, uniqueBlockPath);
-    } else {
-      bitSet = filterExecuter.isScanRequired(maxValue, minValue);
-    }
-    if (!bitSet.isEmpty()) {
-      return true;
-    } else {
-      return false;
-    }
-  }
-
-  public ExtendedBlocklet getDetailedBlocklet(String blockletId) {
-    int index = Integer.parseInt(blockletId);
-    DataMapRow safeRow = unsafeMemoryDMStore.getUnsafeRow(index).convertToSafeRow();
-    return createBlocklet(safeRow, safeRow.getShort(BLOCKLET_ID_INDEX));
-  }
-
-  private byte[][] getMinMaxValue(DataMapRow row, int index) {
-    DataMapRow minMaxRow = row.getRow(index);
-    byte[][] minMax = new byte[minMaxRow.getColumnCount()][];
-    for (int i = 0; i < minMax.length; i++) {
-      minMax[i] = minMaxRow.getByteArray(i);
-    }
-    return minMax;
-  }
-
-  private ExtendedBlocklet createBlocklet(DataMapRow row, int blockletId) {
-    ExtendedBlocklet blocklet = new ExtendedBlocklet(
-        new String(row.getByteArray(FILE_PATH_INDEX), CarbonCommonConstants.DEFAULT_CHARSET_CLASS),
-        blockletId + "");
-    BlockletDetailInfo detailInfo = new BlockletDetailInfo();
-    detailInfo.setRowCount(row.getInt(ROW_COUNT_INDEX));
-    detailInfo.setPagesCount(row.getShort(PAGE_COUNT_INDEX));
-    detailInfo.setVersionNumber(row.getShort(VERSION_INDEX));
-    detailInfo.setBlockletId((short) blockletId);
-    detailInfo.setDimLens(columnCardinality);
-    detailInfo.setSchemaUpdatedTimeStamp(row.getLong(SCHEMA_UPADATED_TIME_INDEX));
-    byte[] byteArray = row.getByteArray(BLOCK_INFO_INDEX);
-    BlockletInfo blockletInfo = null;
-    try {
-      if (byteArray.length > 0) {
-        blockletInfo = new BlockletInfo();
-        ByteArrayInputStream stream = new ByteArrayInputStream(byteArray);
-        DataInputStream inputStream = new DataInputStream(stream);
-        blockletInfo.readFields(inputStream);
-        inputStream.close();
-      }
-      blocklet.setLocation(
-          new String(row.getByteArray(LOCATIONS), CarbonCommonConstants.DEFAULT_CHARSET)
-              .split(","));
-    } catch (IOException e) {
-      throw new RuntimeException(e);
-    }
-    detailInfo.setBlockletInfo(blockletInfo);
-    blocklet.setDetailInfo(detailInfo);
-    detailInfo.setBlockFooterOffset(row.getLong(BLOCK_FOOTER_OFFSET));
-    detailInfo.setColumnSchemaBinary(getColumnSchemaBinary());
-    detailInfo.setBlockSize(row.getLong(BLOCK_LENGTH));
-    return blocklet;
-  }
-
-  /**
-   * Binary search used to get the first tentative index row based on
-   * search key
-   *
-   * @param key search key
-   * @return first tentative block
-   */
-  private int findStartIndex(DataMapRow key, Comparator<DataMapRow> comparator) {
-    int childNodeIndex;
-    int low = 0;
-    int high = unsafeMemoryDMStore.getRowCount() - 1;
-    int mid = 0;
-    int compareRes = -1;
-    //
-    while (low <= high) {
-      mid = (low + high) >>> 1;
-      // compare the entries
-      compareRes = comparator.compare(key, unsafeMemoryDMStore.getUnsafeRow(mid));
-      if (compareRes < 0) {
-        high = mid - 1;
-      } else if (compareRes > 0) {
-        low = mid + 1;
-      } else {
-        // if key is matched then get the first entry
-        int currentPos = mid;
-        while (currentPos - 1 >= 0
-            && comparator.compare(key, unsafeMemoryDMStore.getUnsafeRow(currentPos - 1)) == 0) {
-          currentPos--;
-        }
-        mid = currentPos;
-        break;
-      }
-    }
-    // if compare result is less than zero then we
-    // and mid is more than 0 then we need to previous block as duplicates
-    // record can be present
-    if (compareRes < 0) {
-      if (mid > 0) {
-        mid--;
-      }
-      childNodeIndex = mid;
-    } else {
-      childNodeIndex = mid;
-    }
-    // get the leaf child
-    return childNodeIndex;
-  }
-
-  /**
-   * Binary search used to get the last tentative block  based on
-   * search key
-   *
-   * @param key search key
-   * @return first tentative block
-   */
-  private int findEndIndex(DataMapRow key, Comparator<DataMapRow> comparator) {
-    int childNodeIndex;
-    int low = 0;
-    int high = unsafeMemoryDMStore.getRowCount() - 1;
-    int mid = 0;
-    int compareRes = -1;
-    //
-    while (low <= high) {
-      mid = (low + high) >>> 1;
-      // compare the entries
-      compareRes = comparator.compare(key, unsafeMemoryDMStore.getUnsafeRow(mid));
-      if (compareRes < 0) {
-        high = mid - 1;
-      } else if (compareRes > 0) {
-        low = mid + 1;
-      } else {
-        int currentPos = mid;
-        // if key is matched then get the first entry
-        while (currentPos + 1 < unsafeMemoryDMStore.getRowCount()
-            && comparator.compare(key, unsafeMemoryDMStore.getUnsafeRow(currentPos + 1)) == 0) {
-          currentPos++;
-        }
-        mid = currentPos;
-        break;
-      }
-    }
-    // if compare result is less than zero then we
-    // and mid is more than 0 then we need to previous block as duplicates
-    // record can be present
-    if (compareRes < 0) {
-      if (mid > 0) {
-        mid--;
-      }
-      childNodeIndex = mid;
-    } else {
-      childNodeIndex = mid;
-    }
-    return childNodeIndex;
-  }
-
-  private DataMapRow convertToRow(IndexKey key) {
-    ByteBuffer buffer =
-        ByteBuffer.allocate(key.getDictionaryKeys().length + key.getNoDictionaryKeys().length + 8);
-    buffer.putInt(key.getDictionaryKeys().length);
-    buffer.putInt(key.getNoDictionaryKeys().length);
-    buffer.put(key.getDictionaryKeys());
-    buffer.put(key.getNoDictionaryKeys());
-    DataMapRowImpl dataMapRow = new DataMapRowImpl(unsafeMemoryDMStore.getSchema());
-    dataMapRow.setByteArray(buffer.array(), 0);
-    return dataMapRow;
-  }
-
-  private List<String> getPartitions() {
-    DataMapRow unsafeRow = unsafeMemorySummaryDMStore.getUnsafeRow(0);
-    if (unsafeRow.getColumnCount() > PARTITION_INFO) {
-      List<String> partitions = new ArrayList<>();
-      DataMapRow row = unsafeRow.getRow(PARTITION_INFO);
-      for (int i = 0; i < row.getColumnCount(); i++) {
-        partitions.add(
-            new String(row.getByteArray(i), CarbonCommonConstants.DEFAULT_CHARSET_CLASS));
-      }
-      return partitions;
-    }
-    return null;
-  }
-
-  private byte[] getColumnSchemaBinary() {
-    DataMapRow unsafeRow = unsafeMemorySummaryDMStore.getUnsafeRow(0);
-    return unsafeRow.getByteArray(SCHEMA);
-  }
-
-  /**
-   * Convert schema to binary
-   */
-  private byte[] convertSchemaToBinary(List<ColumnSchema> columnSchemas) throws IOException {
-    ByteArrayOutputStream stream = new ByteArrayOutputStream();
-    DataOutput dataOutput = new DataOutputStream(stream);
-    dataOutput.writeShort(columnSchemas.size());
-    for (ColumnSchema columnSchema : columnSchemas) {
-      if (columnSchema.getColumnReferenceId() == null) {
-        columnSchema.setColumnReferenceId(columnSchema.getColumnUniqueId());
-      }
-      columnSchema.write(dataOutput);
-    }
-    byte[] byteArray = stream.toByteArray();
-    // Compress with snappy to reduce the size of schema
-    return Snappy.rawCompress(byteArray, byteArray.length);
-  }
-
-  @Override
-  public void clear() {
-    if (unsafeMemoryDMStore != null) {
-      unsafeMemoryDMStore.freeMemory();
-      unsafeMemoryDMStore = null;
-      segmentProperties = null;
-    }
-    // clear task min/max unsafe memory
-    if (null != unsafeMemorySummaryDMStore) {
-      unsafeMemorySummaryDMStore.freeMemory();
-      unsafeMemorySummaryDMStore = null;
-    }
-  }
-
-  @Override
-  public long getFileTimeStamp() {
-    return 0;
-  }
-
-  @Override
-  public int getAccessCount() {
-    return 0;
-  }
-
-  @Override
-  public long getMemorySize() {
-    long memoryUsed = 0L;
-    if (unsafeMemoryDMStore != null) {
-      memoryUsed += unsafeMemoryDMStore.getMemoryUsed();
-    }
-    if (null != unsafeMemorySummaryDMStore) {
-      memoryUsed += unsafeMemorySummaryDMStore.getMemoryUsed();
-    }
-    return memoryUsed;
-  }
-
-  public SegmentProperties getSegmentProperties() {
-    return segmentProperties;
-  }
-
-}

http://git-wip-us.apache.org/repos/asf/carbondata/blob/5a625f4c/core/src/main/java/org/apache/carbondata/core/indexstore/blockletindex/BlockletDataMapFactory.java
----------------------------------------------------------------------
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/BlockletDataMapFactory.java
deleted file mode 100644
index 664242b..0000000
--- a/core/src/main/java/org/apache/carbondata/core/indexstore/blockletindex/BlockletDataMapFactory.java
+++ /dev/null
@@ -1,256 +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.indexstore.blockletindex;
-
-import java.io.IOException;
-import java.util.ArrayList;
-import java.util.HashMap;
-import java.util.List;
-import java.util.Map;
-
-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.DataMapMeta;
-import org.apache.carbondata.core.datamap.dev.AbstractDataMapWriter;
-import org.apache.carbondata.core.datamap.dev.DataMap;
-import org.apache.carbondata.core.datamap.dev.cgdatamap.AbstractCoarseGrainDataMap;
-import org.apache.carbondata.core.datamap.dev.cgdatamap.AbstractCoarseGrainDataMapFactory;
-import org.apache.carbondata.core.datastore.block.SegmentProperties;
-import org.apache.carbondata.core.datastore.filesystem.CarbonFile;
-import org.apache.carbondata.core.datastore.impl.FileFactory;
-import org.apache.carbondata.core.indexstore.Blocklet;
-import org.apache.carbondata.core.indexstore.BlockletDetailsFetcher;
-import org.apache.carbondata.core.indexstore.ExtendedBlocklet;
-import org.apache.carbondata.core.indexstore.SegmentPropertiesFetcher;
-import org.apache.carbondata.core.indexstore.TableBlockIndexUniqueIdentifier;
-import org.apache.carbondata.core.metadata.AbsoluteTableIdentifier;
-import org.apache.carbondata.core.metadata.schema.table.DataMapSchema;
-import org.apache.carbondata.core.util.path.CarbonTablePath;
-import org.apache.carbondata.events.Event;
-
-import org.apache.hadoop.fs.FileSystem;
-import org.apache.hadoop.fs.LocatedFileStatus;
-import org.apache.hadoop.fs.Path;
-import org.apache.hadoop.fs.RemoteIterator;
-
-/**
- * Table map for blocklet
- */
-public class BlockletDataMapFactory extends AbstractCoarseGrainDataMapFactory
-    implements BlockletDetailsFetcher,
-    SegmentPropertiesFetcher {
-
-  private static final String NAME = "clustered.btree.blocklet";
-
-  public static final DataMapSchema DATA_MAP_SCHEMA =
-      new DataMapSchema(NAME, BlockletDataMapFactory.class.getName());
-
-  private AbsoluteTableIdentifier identifier;
-
-  // segmentId -> list of index file
-  private Map<String, List<TableBlockIndexUniqueIdentifier>> segmentMap = new HashMap<>();
-
-  private Cache<TableBlockIndexUniqueIdentifier, AbstractCoarseGrainDataMap> cache;
-
-  @Override
-  public void init(AbsoluteTableIdentifier identifier, DataMapSchema dataMapSchema) {
-    this.identifier = identifier;
-    cache = CacheProvider.getInstance()
-        .createCache(CacheType.DRIVER_BLOCKLET_DATAMAP);
-  }
-
-  @Override
-  public AbstractDataMapWriter createWriter(String segmentId, String dataWriterPath) {
-    throw new UnsupportedOperationException("not implemented");
-  }
-
-  @Override
-  public List<AbstractCoarseGrainDataMap> getDataMaps(String segmentId) throws IOException {
-    List<TableBlockIndexUniqueIdentifier> tableBlockIndexUniqueIdentifiers =
-        getTableBlockIndexUniqueIdentifiers(segmentId);
-    return cache.getAll(tableBlockIndexUniqueIdentifiers);
-  }
-
-  private List<TableBlockIndexUniqueIdentifier> getTableBlockIndexUniqueIdentifiers(
-      String segmentId) throws IOException {
-    List<TableBlockIndexUniqueIdentifier> tableBlockIndexUniqueIdentifiers =
-        segmentMap.get(segmentId);
-    if (tableBlockIndexUniqueIdentifiers == null) {
-      tableBlockIndexUniqueIdentifiers = new ArrayList<>();
-      String path = CarbonTablePath.getSegmentPath(identifier.getTablePath(), segmentId);
-      List<String> indexFiles = new SegmentIndexFileStore().getIndexFilesFromSegment(path);
-      for (int i = 0; i < indexFiles.size(); i++) {
-        tableBlockIndexUniqueIdentifiers.add(
-            new TableBlockIndexUniqueIdentifier(identifier, segmentId, indexFiles.get(i)));
-      }
-      segmentMap.put(segmentId, tableBlockIndexUniqueIdentifiers);
-    }
-    return tableBlockIndexUniqueIdentifiers;
-  }
-
-  /**
-   * 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 default
-   * datamap.
-   */
-  @Override
-  public List<ExtendedBlocklet> getExtendedBlocklets(List<Blocklet> blocklets, String segmentId)
-      throws IOException {
-    List<ExtendedBlocklet> detailedBlocklets = new ArrayList<>();
-    // If it is already detailed blocklet then type cast and return same
-    if (blocklets.size() > 0 && blocklets.get(0) instanceof ExtendedBlocklet) {
-      for (Blocklet blocklet : blocklets) {
-        detailedBlocklets.add((ExtendedBlocklet) blocklet);
-      }
-      return detailedBlocklets;
-    }
-    List<TableBlockIndexUniqueIdentifier> identifiers =
-        getTableBlockIndexUniqueIdentifiers(segmentId);
-    // Retrieve each blocklets detail information from blocklet datamap
-    for (Blocklet blocklet : blocklets) {
-      detailedBlocklets.add(getExtendedBlocklet(identifiers, blocklet));
-    }
-    return detailedBlocklets;
-  }
-
-  @Override
-  public ExtendedBlocklet getExtendedBlocklet(Blocklet blocklet, String segmentId)
-      throws IOException {
-    if (blocklet instanceof ExtendedBlocklet) {
-      return (ExtendedBlocklet) blocklet;
-    }
-    List<TableBlockIndexUniqueIdentifier> identifiers =
-        getTableBlockIndexUniqueIdentifiers(segmentId);
-    return getExtendedBlocklet(identifiers, blocklet);
-  }
-
-  private ExtendedBlocklet getExtendedBlocklet(List<TableBlockIndexUniqueIdentifier> identifiers,
-      Blocklet blocklet) throws IOException {
-    String carbonIndexFileName = CarbonTablePath.getCarbonIndexFileName(blocklet.getBlockId());
-    for (TableBlockIndexUniqueIdentifier identifier : identifiers) {
-      if (identifier.getCarbonIndexFileName().equals(carbonIndexFileName)) {
-        DataMap dataMap = cache.get(identifier);
-        return ((BlockletDataMap) dataMap).getDetailedBlocklet(blocklet.getBlockletId());
-      }
-    }
-    throw new IOException("Blocklet with blockid " + blocklet.getBlockletId() + " not found ");
-  }
-
-
-
-  @Override
-  public List<DataMapDistributable> toDistributable(String segmentId) {
-    CarbonFile[] carbonIndexFiles = SegmentIndexFileStore.getCarbonIndexFiles(segmentId);
-    List<DataMapDistributable> distributables = new ArrayList<>();
-    for (int i = 0; i < carbonIndexFiles.length; i++) {
-      Path path = new Path(carbonIndexFiles[i].getPath());
-      try {
-        FileSystem fs = path.getFileSystem(FileFactory.getConfiguration());
-        RemoteIterator<LocatedFileStatus> iter = fs.listLocatedStatus(path);
-        LocatedFileStatus fileStatus = iter.next();
-        String[] location = fileStatus.getBlockLocations()[0].getHosts();
-        BlockletDataMapDistributable distributable =
-            new BlockletDataMapDistributable(path.getName());
-        distributable.setLocations(location);
-        distributables.add(distributable);
-      } catch (IOException e) {
-        throw new RuntimeException(e);
-      }
-    }
-    return distributables;
-  }
-
-  @Override public void fireEvent(Event event) {
-
-  }
-
-  @Override
-  public void clear(String segmentId) {
-    List<TableBlockIndexUniqueIdentifier> blockIndexes = segmentMap.remove(segmentId);
-    if (blockIndexes != null) {
-      for (TableBlockIndexUniqueIdentifier blockIndex : blockIndexes) {
-        DataMap dataMap = cache.getIfPresent(blockIndex);
-        if (dataMap != null) {
-          cache.invalidate(blockIndex);
-          dataMap.clear();
-        }
-      }
-    }
-  }
-
-  @Override
-  public void clear() {
-    for (String segmentId : segmentMap.keySet().toArray(new String[segmentMap.size()])) {
-      clear(segmentId);
-    }
-  }
-
-  @Override
-  public List<AbstractCoarseGrainDataMap> getDataMaps(DataMapDistributable distributable)
-      throws IOException {
-    BlockletDataMapDistributable mapDistributable = (BlockletDataMapDistributable) distributable;
-    List<TableBlockIndexUniqueIdentifier> identifiers = new ArrayList<>();
-    if (mapDistributable.getFilePath().endsWith(CarbonTablePath.INDEX_FILE_EXT)) {
-      identifiers.add(new TableBlockIndexUniqueIdentifier(identifier, distributable.getSegmentId(),
-          mapDistributable.getFilePath()));
-    } else if (mapDistributable.getFilePath().endsWith(CarbonTablePath.MERGE_INDEX_FILE_EXT)) {
-      SegmentIndexFileStore fileStore = new SegmentIndexFileStore();
-      List<String> indexFiles = fileStore.getIndexFilesFromMergeFile(
-          CarbonTablePath.getSegmentPath(identifier.getTablePath(), mapDistributable.getSegmentId())
-              + "/" + mapDistributable.getFilePath());
-      for (String indexFile : indexFiles) {
-        identifiers.add(
-            new TableBlockIndexUniqueIdentifier(identifier, distributable.getSegmentId(),
-                indexFile));
-      }
-    }
-    List<AbstractCoarseGrainDataMap> dataMaps;
-    try {
-      dataMaps = cache.getAll(identifiers);
-    } catch (IOException e) {
-      throw new RuntimeException(e);
-    }
-    return dataMaps;
-  }
-
-  @Override
-  public DataMapMeta getMeta() {
-    // TODO: pass SORT_COLUMNS into this class
-    return null;
-  }
-
-  @Override public SegmentProperties getSegmentProperties(String segmentId) throws IOException {
-    List<AbstractCoarseGrainDataMap> dataMaps = getDataMaps(segmentId);
-    assert (dataMaps.size() > 0);
-    AbstractCoarseGrainDataMap coarseGrainDataMap = dataMaps.get(0);
-    assert (coarseGrainDataMap instanceof BlockletDataMap);
-    BlockletDataMap dataMap = (BlockletDataMap) coarseGrainDataMap;
-    return dataMap.getSegmentProperties();
-  }
-
-  @Override public List<Blocklet> getAllBlocklets(String segmentId, List<String> partitions)
-      throws IOException {
-    List<Blocklet> blocklets = new ArrayList<>();
-    List<AbstractCoarseGrainDataMap> dataMaps = getDataMaps(segmentId);
-    for (AbstractCoarseGrainDataMap dataMap : dataMaps) {
-      blocklets.addAll(dataMap.prune(null, getSegmentProperties(segmentId), partitions));
-    }
-    return blocklets;
-  }
-}

http://git-wip-us.apache.org/repos/asf/carbondata/blob/5a625f4c/core/src/main/java/org/apache/carbondata/core/indexstore/blockletindex/BlockletDataMapModel.java
----------------------------------------------------------------------
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/BlockletDataMapModel.java
index b3a7f8c..f9a44ec 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/BlockletDataMapModel.java
@@ -23,7 +23,7 @@ import org.apache.carbondata.core.datamap.dev.DataMapModel;
 import org.apache.carbondata.core.indexstore.BlockMetaInfo;
 
 /**
- * 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 BlockletIndexDataMap.
  */
 public class BlockletDataMapModel extends DataMapModel {
 


[3/6] carbondata git commit: [CARBONDATA-2189] Add DataMapProvider developer interface

Posted by ja...@apache.org.
http://git-wip-us.apache.org/repos/asf/carbondata/blob/5a625f4c/hadoop/src/main/java/org/apache/carbondata/hadoop/api/CarbonTableInputFormat.java
----------------------------------------------------------------------
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 04902f9..98c3398 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
@@ -42,7 +42,7 @@ import org.apache.carbondata.core.datastore.filesystem.CarbonFile;
 import org.apache.carbondata.core.datastore.impl.FileFactory;
 import org.apache.carbondata.core.exception.InvalidConfigurationException;
 import org.apache.carbondata.core.indexstore.ExtendedBlocklet;
-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.ColumnarFormatVersion;
 import org.apache.carbondata.core.metadata.schema.PartitionInfo;
@@ -732,7 +732,7 @@ public class CarbonTableInputFormat<T> extends FileInputFormat<Void, T> {
       DistributableDataMapFormat datamapDstr =
           new DistributableDataMapFormat(absoluteTableIdentifier, dataMapExprWrapper,
               segmentIds, partitionsToPrune,
-              BlockletDataMapFactory.class.getName());
+              BlockletIndexDataMapFactory.class.getName());
       prunedBlocklets = dataMapJob.execute(datamapDstr, resolver);
       // Apply expression on the blocklets.
       prunedBlocklets = dataMapExprWrapper.pruneBlocklets(prunedBlocklets);

http://git-wip-us.apache.org/repos/asf/carbondata/blob/5a625f4c/integration/spark-common-test/src/test/scala/org/apache/carbondata/integration/spark/testsuite/preaggregate/TestPreAggCreateCommand.scala
----------------------------------------------------------------------
diff --git a/integration/spark-common-test/src/test/scala/org/apache/carbondata/integration/spark/testsuite/preaggregate/TestPreAggCreateCommand.scala b/integration/spark-common-test/src/test/scala/org/apache/carbondata/integration/spark/testsuite/preaggregate/TestPreAggCreateCommand.scala
index 0d33797..d84c0e7 100644
--- a/integration/spark-common-test/src/test/scala/org/apache/carbondata/integration/spark/testsuite/preaggregate/TestPreAggCreateCommand.scala
+++ b/integration/spark-common-test/src/test/scala/org/apache/carbondata/integration/spark/testsuite/preaggregate/TestPreAggCreateCommand.scala
@@ -265,7 +265,7 @@ class TestPreAggCreateCommand extends QueryTest with BeforeAndAfterAll {
            | GROUP BY dob,name
        """.stripMargin)
     }
-    assert(e.getMessage.contains(s"$timeSeries keyword missing"))
+    assert(e.getMessage.contains("Only 'path' dmproperty is allowed for this datamap"))
     sql("DROP TABLE IF EXISTS maintabletime")
   }
 
@@ -282,7 +282,7 @@ class TestPreAggCreateCommand extends QueryTest with BeforeAndAfterAll {
           | GROUP BY column3,column5,column2
         """.stripMargin)
     }
-    assert(e.getMessage.contains("DataMap class 'abc' not found"))
+    assert(e.getMessage.contains("DataMap 'abc' not found"))
     sql("DROP DATAMAP IF EXISTS agg0 ON TABLE maintable")
   }
 

http://git-wip-us.apache.org/repos/asf/carbondata/blob/5a625f4c/integration/spark-common-test/src/test/scala/org/apache/carbondata/integration/spark/testsuite/preaggregate/TestPreAggregateTableSelection.scala
----------------------------------------------------------------------
diff --git a/integration/spark-common-test/src/test/scala/org/apache/carbondata/integration/spark/testsuite/preaggregate/TestPreAggregateTableSelection.scala b/integration/spark-common-test/src/test/scala/org/apache/carbondata/integration/spark/testsuite/preaggregate/TestPreAggregateTableSelection.scala
index f9ac354..a973bfd 100644
--- a/integration/spark-common-test/src/test/scala/org/apache/carbondata/integration/spark/testsuite/preaggregate/TestPreAggregateTableSelection.scala
+++ b/integration/spark-common-test/src/test/scala/org/apache/carbondata/integration/spark/testsuite/preaggregate/TestPreAggregateTableSelection.scala
@@ -293,8 +293,8 @@ test("test PreAggregate table selection with timeseries and normal together") {
        | GROUP BY dob,name
        """.stripMargin)
 
-    val df = sql("SELECT timeseries(dob,'year') FROM maintabletime GROUP BY timeseries(dob,'year')")
-    preAggTableValidator(df.queryExecution.analyzed, "maintabletime_agg1_year")
+  val df = sql("SELECT timeseries(dob,'year') FROM maintabletime GROUP BY timeseries(dob,'year')")
+  preAggTableValidator(df.queryExecution.analyzed, "maintabletime_agg1_year")
   sql("DROP TABLE IF EXISTS maintabletime")
 
   }

http://git-wip-us.apache.org/repos/asf/carbondata/blob/5a625f4c/integration/spark-common-test/src/test/scala/org/apache/carbondata/integration/spark/testsuite/timeseries/TestTimeSeriesCreateTable.scala
----------------------------------------------------------------------
diff --git a/integration/spark-common-test/src/test/scala/org/apache/carbondata/integration/spark/testsuite/timeseries/TestTimeSeriesCreateTable.scala b/integration/spark-common-test/src/test/scala/org/apache/carbondata/integration/spark/testsuite/timeseries/TestTimeSeriesCreateTable.scala
index 49cabea..0868eed 100644
--- a/integration/spark-common-test/src/test/scala/org/apache/carbondata/integration/spark/testsuite/timeseries/TestTimeSeriesCreateTable.scala
+++ b/integration/spark-common-test/src/test/scala/org/apache/carbondata/integration/spark/testsuite/timeseries/TestTimeSeriesCreateTable.scala
@@ -200,7 +200,7 @@ class TestTimeSeriesCreateTable extends QueryTest with BeforeAndAfterAll {
           | GROUP BY dataTime
         """.stripMargin)
     }
-    assert(e.getMessage.equals("DataMap class 'abc' not found"))
+    assert(e.getMessage.equals("DataMap 'abc' not found"))
   }
 
   test("test timeseries create table: USING and catch MalformedCarbonCommandException") {
@@ -215,7 +215,7 @@ class TestTimeSeriesCreateTable extends QueryTest with BeforeAndAfterAll {
           | GROUP BY dataTime
         """.stripMargin)
     }
-    assert(e.getMessage.equals("DataMap class 'abc' not found"))
+    assert(e.getMessage.equals("DataMap 'abc' not found"))
   }
 
   test("test timeseries create table: Only one granularity level can be defined 1") {

http://git-wip-us.apache.org/repos/asf/carbondata/blob/5a625f4c/integration/spark-common-test/src/test/scala/org/apache/carbondata/spark/testsuite/datamap/CGDataMapTestCase.scala
----------------------------------------------------------------------
diff --git a/integration/spark-common-test/src/test/scala/org/apache/carbondata/spark/testsuite/datamap/CGDataMapTestCase.scala b/integration/spark-common-test/src/test/scala/org/apache/carbondata/spark/testsuite/datamap/CGDataMapTestCase.scala
deleted file mode 100644
index d4c49d2..0000000
--- a/integration/spark-common-test/src/test/scala/org/apache/carbondata/spark/testsuite/datamap/CGDataMapTestCase.scala
+++ /dev/null
@@ -1,379 +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.spark.testsuite.datamap
-
-import java.io.{ByteArrayInputStream, DataOutputStream, ObjectInputStream, ObjectOutputStream}
-
-import scala.collection.JavaConverters._
-import scala.collection.mutable.ArrayBuffer
-
-import com.sun.xml.internal.messaging.saaj.util.ByteOutputStream
-import org.apache.spark.sql.test.util.QueryTest
-import org.scalatest.BeforeAndAfterAll
-
-import org.apache.carbondata.core.datamap.dev.cgdatamap.{AbstractCoarseGrainDataMap, AbstractCoarseGrainDataMapFactory}
-import org.apache.carbondata.core.datamap.dev.{AbstractDataMapWriter, DataMapModel}
-import org.apache.carbondata.core.datamap.{DataMapDistributable, DataMapMeta, DataMapStoreManager}
-import org.apache.carbondata.core.datastore.FileReader
-import org.apache.carbondata.core.datastore.block.SegmentProperties
-import org.apache.carbondata.core.datastore.compression.SnappyCompressor
-import org.apache.carbondata.core.datastore.filesystem.{CarbonFile, CarbonFileFilter}
-import org.apache.carbondata.core.datastore.impl.FileFactory
-import org.apache.carbondata.core.datastore.page.ColumnPage
-import org.apache.carbondata.core.indexstore.Blocklet
-import org.apache.carbondata.core.indexstore.blockletindex.BlockletDataMapDistributable
-import org.apache.carbondata.core.metadata.schema.table.DataMapSchema
-import org.apache.carbondata.core.metadata.{AbsoluteTableIdentifier, CarbonMetadata}
-import org.apache.carbondata.core.scan.expression.Expression
-import org.apache.carbondata.core.scan.expression.conditional.EqualToExpression
-import org.apache.carbondata.core.scan.filter.intf.ExpressionType
-import org.apache.carbondata.core.scan.filter.resolver.FilterResolverIntf
-import org.apache.carbondata.core.util.ByteUtil
-import org.apache.carbondata.core.util.path.CarbonTablePath
-import org.apache.carbondata.events.Event
-import org.apache.carbondata.spark.testsuite.datacompaction.CompactionSupportGlobalSortBigFileTest
-
-class CGDataMapFactory extends AbstractCoarseGrainDataMapFactory {
-  var identifier: AbsoluteTableIdentifier = _
-  var dataMapSchema: DataMapSchema = _
-
-  /**
-   * Initialization of Datamap factory with the identifier and datamap name
-   */
-  override def init(identifier: AbsoluteTableIdentifier, dataMapSchema: DataMapSchema): Unit = {
-    this.identifier = identifier
-    this.dataMapSchema = dataMapSchema
-  }
-
-  /**
-   * Return a new write for this datamap
-   */
-  override def createWriter(segmentId: String, dataWritePath: String): AbstractDataMapWriter = {
-    new CGDataMapWriter(identifier, segmentId, dataWritePath, dataMapSchema)
-  }
-
-  /**
-   * Get the datamap for segmentid
-   */
-  override def getDataMaps(segmentId: String) = {
-    val file = FileFactory.getCarbonFile(CarbonTablePath.getSegmentPath(identifier.getTablePath, segmentId))
-
-    val files = file.listFiles(new CarbonFileFilter {
-      override def accept(file: CarbonFile): Boolean = file.getName.endsWith(".datamap")
-    })
-    files.map {f =>
-      val dataMap: AbstractCoarseGrainDataMap = new CGDataMap()
-      dataMap.init(new DataMapModel(f.getCanonicalPath))
-      dataMap
-    }.toList.asJava
-  }
-
-
-  /**
-   * Get datamaps for distributable object.
-   */
-  override def getDataMaps(
-      distributable: DataMapDistributable): java.util.List[AbstractCoarseGrainDataMap] = {
-    val mapDistributable = distributable.asInstanceOf[BlockletDataMapDistributable]
-    val dataMap: AbstractCoarseGrainDataMap = new CGDataMap()
-    dataMap.init(new DataMapModel(mapDistributable.getFilePath))
-    Seq(dataMap).asJava
-  }
-
-  /**
-   *
-   * @param event
-   */
-  override def fireEvent(event: Event): Unit = {
-    ???
-  }
-
-  /**
-   * Get all distributable objects of a segmentid
-   *
-   * @return
-   */
-  override def toDistributable(segmentId: String) = {
-    val file = FileFactory.getCarbonFile(CarbonTablePath.getSegmentPath(identifier.getTablePath, segmentId))
-
-    val files = file.listFiles(new CarbonFileFilter {
-      override def accept(file: CarbonFile): Boolean = file.getName.endsWith(".datamap")
-    })
-    files.map { f =>
-      val d:DataMapDistributable = new BlockletDataMapDistributable(f.getCanonicalPath)
-      d
-    }.toList.asJava
-  }
-
-
-  /**
-   * Clears datamap of the segment
-   */
-  override def clear(segmentId: String): Unit = {
-
-  }
-
-  /**
-   * Clear all datamaps from memory
-   */
-  override def clear(): Unit = {
-
-  }
-
-  /**
-   * Return metadata of this datamap
-   */
-  override def getMeta: DataMapMeta = {
-    new DataMapMeta(dataMapSchema.getProperties.get("indexcolumns").split(",").toList.asJava,
-      List(ExpressionType.EQUALS, ExpressionType.IN).asJava)
-  }
-}
-
-class CGDataMap extends AbstractCoarseGrainDataMap {
-
-  var maxMin: ArrayBuffer[(String, Int, (Array[Byte], Array[Byte]))] = _
-  var FileReader: FileReader = _
-  var filePath: String = _
-  val compressor = new SnappyCompressor
-
-  /**
-   * It is called to load the data map to memory or to initialize it.
-   */
-  override def init(dataMapModel: DataMapModel): Unit = {
-    this.filePath = dataMapModel.getFilePath
-    val size = FileFactory.getCarbonFile(filePath).getSize
-    FileReader = FileFactory.getFileHolder(FileFactory.getFileType(filePath))
-    val footerLen = FileReader.readInt(filePath, size-4)
-    val bytes = FileReader.readByteArray(filePath, size-footerLen-4, footerLen)
-    val in = new ByteArrayInputStream(compressor.unCompressByte(bytes))
-    val obj = new ObjectInputStream(in)
-    maxMin = obj.readObject().asInstanceOf[ArrayBuffer[(String, Int, (Array[Byte], Array[Byte]))]]
-  }
-
-  /**
-   * Prune the datamap with filter expression. It returns the list of
-   * blocklets where these filters can exist.
-   *
-   * @param filterExp
-   * @return
-   */
-  override def prune(
-      filterExp: FilterResolverIntf,
-      segmentProperties: SegmentProperties,
-      partitions: java.util.List[String]): java.util.List[Blocklet] = {
-    val buffer: ArrayBuffer[Expression] = new ArrayBuffer[Expression]()
-    val expression = filterExp.getFilterExpression
-    getEqualToExpression(expression, buffer)
-    val value = buffer.map { f =>
-      f.getChildren.get(1).evaluate(null).getString
-    }
-    val meta = findMeta(value(0).getBytes)
-    meta.map { f=>
-      new Blocklet(f._1, f._2+"")
-    }.asJava
-  }
-
-
-  private def findMeta(value: Array[Byte]) = {
-    val tuples = maxMin.filter { f =>
-      ByteUtil.UnsafeComparer.INSTANCE.compareTo(value, f._3._1) <= 0 &&
-      ByteUtil.UnsafeComparer.INSTANCE.compareTo(value, f._3._2) >= 0
-    }
-    tuples
-  }
-
-  private def getEqualToExpression(expression: Expression, buffer: ArrayBuffer[Expression]): Unit = {
-    if (expression.isInstanceOf[EqualToExpression]) {
-      buffer += expression
-    } else {
-      if (expression.getChildren != null) {
-        expression.getChildren.asScala.map { f =>
-          if (f.isInstanceOf[EqualToExpression]) {
-            buffer += f
-          }
-          getEqualToExpression(f, buffer)
-        }
-      }
-    }
-  }
-
-  /**
-   * Clear complete index table and release memory.
-   */
-  override def clear() = {
-    ???
-  }
-
-  override def isScanRequired(filterExp: FilterResolverIntf): Boolean = ???
-}
-
-class CGDataMapWriter(identifier: AbsoluteTableIdentifier,
-    segmentId: String,
-    dataWritePath: String,
-    dataMapSchema: DataMapSchema)
-  extends AbstractDataMapWriter(identifier, segmentId, dataWritePath) {
-
-  var currentBlockId: String = null
-  val cgwritepath = dataWritePath + "/" +
-                    dataMapSchema.getDataMapName + System.nanoTime() + ".datamap"
-  lazy val stream: DataOutputStream = FileFactory
-    .getDataOutputStream(cgwritepath, FileFactory.getFileType(cgwritepath))
-  val blockletList = new ArrayBuffer[Array[Byte]]()
-  val maxMin = new ArrayBuffer[(String, Int, (Array[Byte], Array[Byte]))]()
-  val compressor = new SnappyCompressor
-
-  /**
-   * Start of new block notification.
-   *
-   * @param blockId file name of the carbondata file
-   */
-  override def onBlockStart(blockId: String): Unit = {
-    currentBlockId = blockId
-  }
-
-  /**
-   * End of block notification
-   */
-  override def onBlockEnd(blockId: String): Unit = {
-
-  }
-
-  /**
-   * Start of new blocklet notification.
-   *
-   * @param blockletId sequence number of blocklet in the block
-   */
-  override def onBlockletStart(blockletId: Int): Unit = {
-
-  }
-
-  /**
-   * End of blocklet notification
-   *
-   * @param blockletId sequence number of blocklet in the block
-   */
-  override def onBlockletEnd(blockletId: Int): Unit = {
-    val sorted = blockletList
-      .sortWith((l, r) => ByteUtil.UnsafeComparer.INSTANCE.compareTo(l, r) <= 0)
-    maxMin +=
-    ((currentBlockId+"", blockletId, (sorted.last, sorted.head)))
-    blockletList.clear()
-  }
-
-  /**
-   * Add the column pages row to the datamap, order of pages is same as `indexColumns` in
-   * DataMapMeta returned in DataMapFactory.
-   *
-   * Implementation should copy the content of `pages` as needed, because `pages` memory
-   * may be freed after this method returns, if using unsafe column page.
-   */
-  override def onPageAdded(blockletId: Int,
-      pageId: Int,
-      pages: Array[ColumnPage]): Unit = {
-    val size = pages(0).getPageSize
-    val list = new ArrayBuffer[Array[Byte]]()
-    var i = 0
-    while (i < size) {
-      val bytes = pages(0).getBytes(i)
-      val newBytes = new Array[Byte](bytes.length - 2)
-      System.arraycopy(bytes, 2, newBytes, 0, newBytes.length)
-      list += newBytes
-      i = i + 1
-    }
-    // Sort based on the column data in order to create index.
-    val sorted = list
-      .sortWith((l, r) => ByteUtil.UnsafeComparer.INSTANCE.compareTo(l, r) <= 0)
-    blockletList += sorted.head
-    blockletList += sorted.last
-  }
-
-
-  /**
-   * This is called during closing of writer.So after this call no more data will be sent to this
-   * class.
-   */
-  override def finish(): Unit = {
-    val out = new ByteOutputStream()
-    val outStream = new ObjectOutputStream(out)
-    outStream.writeObject(maxMin)
-    outStream.close()
-    val bytes = compressor.compressByte(out.getBytes)
-    stream.write(bytes)
-    stream.writeInt(bytes.length)
-    stream.close()
-    commitFile(cgwritepath)
-  }
-
-
-}
-
-class CGDataMapTestCase extends QueryTest with BeforeAndAfterAll {
-
-  val file2 = resourcesPath + "/compaction/fil2.csv"
-  override protected def beforeAll(): Unit = {
-    //n should be about 5000000 of reset if size is default 1024
-    val n = 150000
-    CompactionSupportGlobalSortBigFileTest.createFile(file2, n * 4, n)
-    sql("DROP TABLE IF EXISTS normal_test")
-    sql(
-      """
-        | CREATE TABLE normal_test(id INT, name STRING, city STRING, age INT)
-        | STORED BY 'org.apache.carbondata.format'
-        | TBLPROPERTIES('SORT_COLUMNS'='city,name', 'SORT_SCOPE'='LOCAL_SORT')
-      """.stripMargin)
-    sql(s"LOAD DATA LOCAL INPATH '$file2' INTO TABLE normal_test OPTIONS('header'='false')")
-  }
-
-  test("test cg datamap") {
-    sql("DROP TABLE IF EXISTS datamap_test_cg")
-    sql(
-      """
-        | CREATE TABLE datamap_test_cg(id INT, name STRING, city STRING, age INT)
-        | STORED BY 'org.apache.carbondata.format'
-        | TBLPROPERTIES('SORT_COLUMNS'='city,name', 'SORT_SCOPE'='LOCAL_SORT')
-      """.stripMargin)
-    val table = CarbonMetadata.getInstance().getCarbonTable("default_datamap_test_cg")
-    // register datamap writer
-    sql(s"create datamap cgdatamap on table datamap_test_cg using '${classOf[CGDataMapFactory].getName}' DMPROPERTIES('indexcolumns'='name')")
-    sql(s"LOAD DATA LOCAL INPATH '$file2' INTO TABLE datamap_test_cg OPTIONS('header'='false')")
-    checkAnswer(sql("select * from datamap_test_cg where name='n502670'"),
-      sql("select * from normal_test where name='n502670'"))
-  }
-
-  test("test cg datamap with 2 datamaps ") {
-    sql("DROP TABLE IF EXISTS datamap_test")
-    sql(
-      """
-        | CREATE TABLE datamap_test(id INT, name STRING, city STRING, age INT)
-        | STORED BY 'org.apache.carbondata.format'
-        | TBLPROPERTIES('SORT_COLUMNS'='city,name', 'SORT_SCOPE'='LOCAL_SORT')
-      """.stripMargin)
-    val table = CarbonMetadata.getInstance().getCarbonTable("default_datamap_test")
-    // register datamap writer
-    sql(s"create datamap ggdatamap1 on table datamap_test using '${classOf[CGDataMapFactory].getName}' DMPROPERTIES('indexcolumns'='name')")
-    sql(s"create datamap ggdatamap2 on table datamap_test using '${classOf[CGDataMapFactory].getName}' DMPROPERTIES('indexcolumns'='city')")
-    sql(s"LOAD DATA LOCAL INPATH '$file2' INTO TABLE datamap_test OPTIONS('header'='false')")
-    checkAnswer(sql("select * from datamap_test where name='n502670' and city='c2670'"),
-      sql("select * from normal_test where name='n502670' and city='c2670'"))
-  }
-
-  override protected def afterAll(): Unit = {
-    CompactionSupportGlobalSortBigFileTest.deleteFile(file2)
-    sql("DROP TABLE IF EXISTS normal_test")
-    sql("DROP TABLE IF EXISTS datamap_test_cg")
-  }
-}

http://git-wip-us.apache.org/repos/asf/carbondata/blob/5a625f4c/integration/spark-common-test/src/test/scala/org/apache/carbondata/spark/testsuite/datamap/CGIndexDataMapTestCase.scala
----------------------------------------------------------------------
diff --git a/integration/spark-common-test/src/test/scala/org/apache/carbondata/spark/testsuite/datamap/CGIndexDataMapTestCase.scala b/integration/spark-common-test/src/test/scala/org/apache/carbondata/spark/testsuite/datamap/CGIndexDataMapTestCase.scala
new file mode 100644
index 0000000..5e944fb
--- /dev/null
+++ b/integration/spark-common-test/src/test/scala/org/apache/carbondata/spark/testsuite/datamap/CGIndexDataMapTestCase.scala
@@ -0,0 +1,379 @@
+/*
+ * 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.spark.testsuite.datamap
+
+import java.io.{ByteArrayInputStream, DataOutputStream, ObjectInputStream, ObjectOutputStream}
+
+import scala.collection.JavaConverters._
+import scala.collection.mutable.ArrayBuffer
+
+import com.sun.xml.internal.messaging.saaj.util.ByteOutputStream
+import org.apache.spark.sql.test.util.QueryTest
+import org.scalatest.BeforeAndAfterAll
+
+import org.apache.carbondata.core.datamap.{DataMapDistributable, DataMapMeta, DataMapStoreManager}
+import org.apache.carbondata.core.datamap.dev.{AbstractDataMapWriter, DataMapModel}
+import org.apache.carbondata.core.datamap.dev.cgdatamap.{AbstractCoarseGrainIndexDataMap, AbstractCoarseGrainIndexDataMapFactory}
+import org.apache.carbondata.core.datastore.FileReader
+import org.apache.carbondata.core.datastore.block.SegmentProperties
+import org.apache.carbondata.core.datastore.compression.SnappyCompressor
+import org.apache.carbondata.core.datastore.filesystem.{CarbonFile, CarbonFileFilter}
+import org.apache.carbondata.core.datastore.impl.FileFactory
+import org.apache.carbondata.core.datastore.page.ColumnPage
+import org.apache.carbondata.core.indexstore.Blocklet
+import org.apache.carbondata.core.indexstore.blockletindex.BlockletDataMapDistributable
+import org.apache.carbondata.core.metadata.schema.table.DataMapSchema
+import org.apache.carbondata.core.metadata.{AbsoluteTableIdentifier, CarbonMetadata}
+import org.apache.carbondata.core.scan.expression.Expression
+import org.apache.carbondata.core.scan.expression.conditional.EqualToExpression
+import org.apache.carbondata.core.scan.filter.intf.ExpressionType
+import org.apache.carbondata.core.scan.filter.resolver.FilterResolverIntf
+import org.apache.carbondata.core.util.ByteUtil
+import org.apache.carbondata.core.util.path.CarbonTablePath
+import org.apache.carbondata.events.Event
+import org.apache.carbondata.spark.testsuite.datacompaction.CompactionSupportGlobalSortBigFileTest
+
+class CGIndexDataMapFactory extends AbstractCoarseGrainIndexDataMapFactory {
+  var identifier: AbsoluteTableIdentifier = _
+  var dataMapSchema: DataMapSchema = _
+
+  /**
+   * Initialization of Datamap factory with the identifier and datamap name
+   */
+  override def init(identifier: AbsoluteTableIdentifier, dataMapSchema: DataMapSchema): Unit = {
+    this.identifier = identifier
+    this.dataMapSchema = dataMapSchema
+  }
+
+  /**
+   * Return a new write for this datamap
+   */
+  override def createWriter(segmentId: String, dataWritePath: String): AbstractDataMapWriter = {
+    new CGDataMapWriter(identifier, segmentId, dataWritePath, dataMapSchema)
+  }
+
+  /**
+   * Get the datamap for segmentid
+   */
+  override def getDataMaps(segmentId: String) = {
+    val file = FileFactory.getCarbonFile(CarbonTablePath.getSegmentPath(identifier.getTablePath, segmentId))
+
+    val files = file.listFiles(new CarbonFileFilter {
+      override def accept(file: CarbonFile): Boolean = file.getName.endsWith(".datamap")
+    })
+    files.map {f =>
+      val dataMap: AbstractCoarseGrainIndexDataMap = new CGIndexDataMap()
+      dataMap.init(new DataMapModel(f.getCanonicalPath))
+      dataMap
+    }.toList.asJava
+  }
+
+
+  /**
+   * Get datamaps for distributable object.
+   */
+  override def getDataMaps(
+      distributable: DataMapDistributable): java.util.List[AbstractCoarseGrainIndexDataMap] = {
+    val mapDistributable = distributable.asInstanceOf[BlockletDataMapDistributable]
+    val dataMap: AbstractCoarseGrainIndexDataMap = new CGIndexDataMap()
+    dataMap.init(new DataMapModel(mapDistributable.getFilePath))
+    Seq(dataMap).asJava
+  }
+
+  /**
+   *
+   * @param event
+   */
+  override def fireEvent(event: Event): Unit = {
+    ???
+  }
+
+  /**
+   * Get all distributable objects of a segmentid
+   *
+   * @return
+   */
+  override def toDistributable(segmentId: String) = {
+    val file = FileFactory.getCarbonFile(CarbonTablePath.getSegmentPath(identifier.getTablePath, segmentId))
+
+    val files = file.listFiles(new CarbonFileFilter {
+      override def accept(file: CarbonFile): Boolean = file.getName.endsWith(".datamap")
+    })
+    files.map { f =>
+      val d:DataMapDistributable = new BlockletDataMapDistributable(f.getCanonicalPath)
+      d
+    }.toList.asJava
+  }
+
+
+  /**
+   * Clears datamap of the segment
+   */
+  override def clear(segmentId: String): Unit = {
+
+  }
+
+  /**
+   * Clear all datamaps from memory
+   */
+  override def clear(): Unit = {
+
+  }
+
+  /**
+   * Return metadata of this datamap
+   */
+  override def getMeta: DataMapMeta = {
+    new DataMapMeta(dataMapSchema.getProperties.get("indexcolumns").split(",").toList.asJava,
+      List(ExpressionType.EQUALS, ExpressionType.IN).asJava)
+  }
+}
+
+class CGIndexDataMap extends AbstractCoarseGrainIndexDataMap {
+
+  var maxMin: ArrayBuffer[(String, Int, (Array[Byte], Array[Byte]))] = _
+  var FileReader: FileReader = _
+  var filePath: String = _
+  val compressor = new SnappyCompressor
+
+  /**
+   * It is called to load the data map to memory or to initialize it.
+   */
+  override def init(dataMapModel: DataMapModel): Unit = {
+    this.filePath = dataMapModel.getFilePath
+    val size = FileFactory.getCarbonFile(filePath).getSize
+    FileReader = FileFactory.getFileHolder(FileFactory.getFileType(filePath))
+    val footerLen = FileReader.readInt(filePath, size-4)
+    val bytes = FileReader.readByteArray(filePath, size-footerLen-4, footerLen)
+    val in = new ByteArrayInputStream(compressor.unCompressByte(bytes))
+    val obj = new ObjectInputStream(in)
+    maxMin = obj.readObject().asInstanceOf[ArrayBuffer[(String, Int, (Array[Byte], Array[Byte]))]]
+  }
+
+  /**
+   * Prune the datamap with filter expression. It returns the list of
+   * blocklets where these filters can exist.
+   *
+   * @param filterExp
+   * @return
+   */
+  override def prune(
+      filterExp: FilterResolverIntf,
+      segmentProperties: SegmentProperties,
+      partitions: java.util.List[String]): java.util.List[Blocklet] = {
+    val buffer: ArrayBuffer[Expression] = new ArrayBuffer[Expression]()
+    val expression = filterExp.getFilterExpression
+    getEqualToExpression(expression, buffer)
+    val value = buffer.map { f =>
+      f.getChildren.get(1).evaluate(null).getString
+    }
+    val meta = findMeta(value(0).getBytes)
+    meta.map { f=>
+      new Blocklet(f._1, f._2+"")
+    }.asJava
+  }
+
+
+  private def findMeta(value: Array[Byte]) = {
+    val tuples = maxMin.filter { f =>
+      ByteUtil.UnsafeComparer.INSTANCE.compareTo(value, f._3._1) <= 0 &&
+      ByteUtil.UnsafeComparer.INSTANCE.compareTo(value, f._3._2) >= 0
+    }
+    tuples
+  }
+
+  private def getEqualToExpression(expression: Expression, buffer: ArrayBuffer[Expression]): Unit = {
+    if (expression.isInstanceOf[EqualToExpression]) {
+      buffer += expression
+    } else {
+      if (expression.getChildren != null) {
+        expression.getChildren.asScala.map { f =>
+          if (f.isInstanceOf[EqualToExpression]) {
+            buffer += f
+          }
+          getEqualToExpression(f, buffer)
+        }
+      }
+    }
+  }
+
+  /**
+   * Clear complete index table and release memory.
+   */
+  override def clear() = {
+    ???
+  }
+
+  override def isScanRequired(filterExp: FilterResolverIntf): Boolean = ???
+}
+
+class CGDataMapWriter(identifier: AbsoluteTableIdentifier,
+    segmentId: String,
+    dataWritePath: String,
+    dataMapSchema: DataMapSchema)
+  extends AbstractDataMapWriter(identifier, segmentId, dataWritePath) {
+
+  var currentBlockId: String = null
+  val cgwritepath = dataWritePath + "/" +
+                    dataMapSchema.getDataMapName + System.nanoTime() + ".datamap"
+  lazy val stream: DataOutputStream = FileFactory
+    .getDataOutputStream(cgwritepath, FileFactory.getFileType(cgwritepath))
+  val blockletList = new ArrayBuffer[Array[Byte]]()
+  val maxMin = new ArrayBuffer[(String, Int, (Array[Byte], Array[Byte]))]()
+  val compressor = new SnappyCompressor
+
+  /**
+   * Start of new block notification.
+   *
+   * @param blockId file name of the carbondata file
+   */
+  override def onBlockStart(blockId: String): Unit = {
+    currentBlockId = blockId
+  }
+
+  /**
+   * End of block notification
+   */
+  override def onBlockEnd(blockId: String): Unit = {
+
+  }
+
+  /**
+   * Start of new blocklet notification.
+   *
+   * @param blockletId sequence number of blocklet in the block
+   */
+  override def onBlockletStart(blockletId: Int): Unit = {
+
+  }
+
+  /**
+   * End of blocklet notification
+   *
+   * @param blockletId sequence number of blocklet in the block
+   */
+  override def onBlockletEnd(blockletId: Int): Unit = {
+    val sorted = blockletList
+      .sortWith((l, r) => ByteUtil.UnsafeComparer.INSTANCE.compareTo(l, r) <= 0)
+    maxMin +=
+    ((currentBlockId+"", blockletId, (sorted.last, sorted.head)))
+    blockletList.clear()
+  }
+
+  /**
+   * Add the column pages row to the datamap, order of pages is same as `indexColumns` in
+   * 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.
+   */
+  override def onPageAdded(blockletId: Int,
+      pageId: Int,
+      pages: Array[ColumnPage]): Unit = {
+    val size = pages(0).getPageSize
+    val list = new ArrayBuffer[Array[Byte]]()
+    var i = 0
+    while (i < size) {
+      val bytes = pages(0).getBytes(i)
+      val newBytes = new Array[Byte](bytes.length - 2)
+      System.arraycopy(bytes, 2, newBytes, 0, newBytes.length)
+      list += newBytes
+      i = i + 1
+    }
+    // Sort based on the column data in order to create index.
+    val sorted = list
+      .sortWith((l, r) => ByteUtil.UnsafeComparer.INSTANCE.compareTo(l, r) <= 0)
+    blockletList += sorted.head
+    blockletList += sorted.last
+  }
+
+
+  /**
+   * This is called during closing of writer.So after this call no more data will be sent to this
+   * class.
+   */
+  override def finish(): Unit = {
+    val out = new ByteOutputStream()
+    val outStream = new ObjectOutputStream(out)
+    outStream.writeObject(maxMin)
+    outStream.close()
+    val bytes = compressor.compressByte(out.getBytes)
+    stream.write(bytes)
+    stream.writeInt(bytes.length)
+    stream.close()
+    commitFile(cgwritepath)
+  }
+
+
+}
+
+class CGIndexDataMapTestCase extends QueryTest with BeforeAndAfterAll {
+
+  val file2 = resourcesPath + "/compaction/fil2.csv"
+  override protected def beforeAll(): Unit = {
+    //n should be about 5000000 of reset if size is default 1024
+    val n = 150000
+    CompactionSupportGlobalSortBigFileTest.createFile(file2, n * 4, n)
+    sql("DROP TABLE IF EXISTS normal_test")
+    sql(
+      """
+        | CREATE TABLE normal_test(id INT, name STRING, city STRING, age INT)
+        | STORED BY 'org.apache.carbondata.format'
+        | TBLPROPERTIES('SORT_COLUMNS'='city,name', 'SORT_SCOPE'='LOCAL_SORT')
+      """.stripMargin)
+    sql(s"LOAD DATA LOCAL INPATH '$file2' INTO TABLE normal_test OPTIONS('header'='false')")
+  }
+
+  test("test cg datamap") {
+    sql("DROP TABLE IF EXISTS datamap_test_cg")
+    sql(
+      """
+        | CREATE TABLE datamap_test_cg(id INT, name STRING, city STRING, age INT)
+        | STORED BY 'org.apache.carbondata.format'
+        | TBLPROPERTIES('SORT_COLUMNS'='city,name', 'SORT_SCOPE'='LOCAL_SORT')
+      """.stripMargin)
+    val table = CarbonMetadata.getInstance().getCarbonTable("default_datamap_test_cg")
+    // register datamap writer
+    sql(s"create datamap cgdatamap on table datamap_test_cg using '${classOf[CGIndexDataMapFactory].getName}' DMPROPERTIES('indexcolumns'='name')")
+    sql(s"LOAD DATA LOCAL INPATH '$file2' INTO TABLE datamap_test_cg OPTIONS('header'='false')")
+    checkAnswer(sql("select * from datamap_test_cg where name='n502670'"),
+      sql("select * from normal_test where name='n502670'"))
+  }
+
+  test("test cg datamap with 2 datamaps ") {
+    sql("DROP TABLE IF EXISTS datamap_test")
+    sql(
+      """
+        | CREATE TABLE datamap_test(id INT, name STRING, city STRING, age INT)
+        | STORED BY 'org.apache.carbondata.format'
+        | TBLPROPERTIES('SORT_COLUMNS'='city,name', 'SORT_SCOPE'='LOCAL_SORT')
+      """.stripMargin)
+    val table = CarbonMetadata.getInstance().getCarbonTable("default_datamap_test")
+    // register datamap writer
+    sql(s"create datamap ggdatamap1 on table datamap_test using '${classOf[CGIndexDataMapFactory].getName}' DMPROPERTIES('indexcolumns'='name')")
+    sql(s"create datamap ggdatamap2 on table datamap_test using '${classOf[CGIndexDataMapFactory].getName}' DMPROPERTIES('indexcolumns'='city')")
+    sql(s"LOAD DATA LOCAL INPATH '$file2' INTO TABLE datamap_test OPTIONS('header'='false')")
+    checkAnswer(sql("select * from datamap_test where name='n502670' and city='c2670'"),
+      sql("select * from normal_test where name='n502670' and city='c2670'"))
+  }
+
+  override protected def afterAll(): Unit = {
+    CompactionSupportGlobalSortBigFileTest.deleteFile(file2)
+    sql("DROP TABLE IF EXISTS normal_test")
+    sql("DROP TABLE IF EXISTS datamap_test_cg")
+  }
+}

http://git-wip-us.apache.org/repos/asf/carbondata/blob/5a625f4c/integration/spark-common-test/src/test/scala/org/apache/carbondata/spark/testsuite/datamap/DataMapWriterSuite.scala
----------------------------------------------------------------------
diff --git a/integration/spark-common-test/src/test/scala/org/apache/carbondata/spark/testsuite/datamap/DataMapWriterSuite.scala b/integration/spark-common-test/src/test/scala/org/apache/carbondata/spark/testsuite/datamap/DataMapWriterSuite.scala
deleted file mode 100644
index 903610a..0000000
--- a/integration/spark-common-test/src/test/scala/org/apache/carbondata/spark/testsuite/datamap/DataMapWriterSuite.scala
+++ /dev/null
@@ -1,216 +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.spark.testsuite.datamap
-
-import java.util
-
-import scala.collection.JavaConverters._
-import org.apache.spark.sql.test.util.QueryTest
-import org.apache.spark.sql.{DataFrame, SaveMode}
-import org.scalatest.BeforeAndAfterAll
-
-import org.apache.carbondata.core.constants.CarbonCommonConstants
-import org.apache.carbondata.core.datamap.dev.AbstractDataMapWriter
-import org.apache.carbondata.core.datamap.dev.cgdatamap.{AbstractCoarseGrainDataMap, AbstractCoarseGrainDataMapFactory}
-import org.apache.carbondata.core.datamap.{DataMapDistributable, DataMapMeta, DataMapStoreManager}
-import org.apache.carbondata.core.datastore.page.ColumnPage
-import org.apache.carbondata.core.metadata.AbsoluteTableIdentifier
-import org.apache.carbondata.core.metadata.datatype.DataTypes
-import org.apache.carbondata.core.metadata.schema.table.DataMapSchema
-import org.apache.carbondata.core.scan.filter.intf.ExpressionType
-import org.apache.carbondata.core.metadata.datatype.DataTypes
-import org.apache.carbondata.core.util.CarbonProperties
-import org.apache.carbondata.events.Event
-
-class C2DataMapFactory() extends AbstractCoarseGrainDataMapFactory {
-
-  var identifier: AbsoluteTableIdentifier = _
-
-  override def init(identifier: AbsoluteTableIdentifier,
-      dataMapSchema: DataMapSchema): Unit = {
-    this.identifier = identifier
-  }
-
-  override def fireEvent(event: Event): Unit = ???
-
-  override def clear(segmentId: String): Unit = {}
-
-  override def clear(): Unit = {}
-
-  override def getDataMaps(distributable: DataMapDistributable): java.util.List[AbstractCoarseGrainDataMap] = ???
-
-  override def getDataMaps(segmentId: String): util.List[AbstractCoarseGrainDataMap] = ???
-
-  override def createWriter(segmentId: String, dataWritePath: String): AbstractDataMapWriter =
-    DataMapWriterSuite.dataMapWriterC2Mock(identifier, segmentId, dataWritePath)
-
-  override def getMeta: DataMapMeta = new DataMapMeta(List("c2").asJava, List(ExpressionType.EQUALS).asJava)
-
-  /**
-   * Get all distributable objects of a segmentid
-   *
-   * @return
-   */
-  override def toDistributable(segmentId: String): util.List[DataMapDistributable] = {
-    ???
-  }
-
-}
-
-class DataMapWriterSuite extends QueryTest with BeforeAndAfterAll {
-  def buildTestData(numRows: Int): DataFrame = {
-    import sqlContext.implicits._
-    sqlContext.sparkContext.parallelize(1 to numRows, 1)
-      .map(x => ("a" + x, "b", x))
-      .toDF("c1", "c2", "c3")
-  }
-
-  def dropTable(): Unit = {
-    sql("DROP TABLE IF EXISTS carbon1")
-    sql("DROP TABLE IF EXISTS carbon2")
-  }
-
-  override def beforeAll {
-    dropTable()
-  }
-
-  test("test write datamap 2 pages") {
-    sql(s"CREATE TABLE carbon1(c1 STRING, c2 STRING, c3 INT) STORED BY 'org.apache.carbondata.format'")
-    // register datamap writer
-    sql(s"CREATE DATAMAP test ON TABLE carbon1 USING '${classOf[C2DataMapFactory].getName}'")
-    val df = buildTestData(33000)
-
-    // save dataframe to carbon file
-    df.write
-      .format("carbondata")
-      .option("tableName", "carbon1")
-      .option("tempCSV", "false")
-      .option("sort_columns","c1")
-      .mode(SaveMode.Overwrite)
-      .save()
-
-    assert(DataMapWriterSuite.callbackSeq.head.contains("block start"))
-    assert(DataMapWriterSuite.callbackSeq.last.contains("block end"))
-    assert(
-      DataMapWriterSuite.callbackSeq.slice(1, DataMapWriterSuite.callbackSeq.length - 1) == Seq(
-        "blocklet start 0",
-        "add page data: blocklet 0, page 0",
-        "add page data: blocklet 0, page 1",
-        "blocklet end: 0"
-      ))
-    DataMapWriterSuite.callbackSeq = Seq()
-  }
-
-  test("test write datamap 2 blocklet") {
-    sql(s"CREATE TABLE carbon2(c1 STRING, c2 STRING, c3 INT) STORED BY 'org.apache.carbondata.format'")
-    sql(s"CREATE DATAMAP test ON TABLE carbon2 USING '${classOf[C2DataMapFactory].getName}'")
-
-    CarbonProperties.getInstance()
-      .addProperty("carbon.blockletgroup.size.in.mb", "1")
-    CarbonProperties.getInstance()
-      .addProperty("carbon.number.of.cores.while.loading",
-        CarbonCommonConstants.NUM_CORES_DEFAULT_VAL)
-
-    val df = buildTestData(300000)
-
-    // save dataframe to carbon file
-    df.write
-      .format("carbondata")
-      .option("tableName", "carbon2")
-      .option("tempCSV", "false")
-      .option("sort_columns","c1")
-      .option("SORT_SCOPE","GLOBAL_SORT")
-      .mode(SaveMode.Overwrite)
-      .save()
-
-    assert(DataMapWriterSuite.callbackSeq.head.contains("block start"))
-    assert(DataMapWriterSuite.callbackSeq.last.contains("block end"))
-    // corrected test case the min "carbon.blockletgroup.size.in.mb" size could not be less than
-    // 64 MB
-    assert(
-      DataMapWriterSuite.callbackSeq.slice(1, DataMapWriterSuite.callbackSeq.length - 1) == Seq(
-        "blocklet start 0",
-        "add page data: blocklet 0, page 0",
-        "add page data: blocklet 0, page 1",
-        "add page data: blocklet 0, page 2",
-        "add page data: blocklet 0, page 3",
-        "add page data: blocklet 0, page 4",
-        "add page data: blocklet 0, page 5",
-        "add page data: blocklet 0, page 6",
-        "add page data: blocklet 0, page 7",
-        "add page data: blocklet 0, page 8",
-        "add page data: blocklet 0, page 9",
-        "blocklet end: 0"
-      ))
-    DataMapWriterSuite.callbackSeq = Seq()
-  }
-
-  override def afterAll {
-    dropTable()
-  }
-}
-
-object DataMapWriterSuite {
-
-  var callbackSeq: Seq[String] = Seq[String]()
-
-  def dataMapWriterC2Mock(identifier: AbsoluteTableIdentifier, segmentId: String,
-      dataWritePath: String) =
-    new AbstractDataMapWriter(identifier, segmentId, dataWritePath) {
-
-    override def onPageAdded(
-        blockletId: Int,
-        pageId: Int,
-        pages: Array[ColumnPage]): Unit = {
-      assert(pages.length == 1)
-      assert(pages(0).getDataType == DataTypes.STRING)
-      val bytes: Array[Byte] = pages(0).getByteArrayPage()(0)
-      assert(bytes.sameElements(Seq(0, 1, 'b'.toByte)))
-      callbackSeq :+= s"add page data: blocklet $blockletId, page $pageId"
-    }
-
-    override def onBlockletEnd(blockletId: Int): Unit = {
-      callbackSeq :+= s"blocklet end: $blockletId"
-    }
-
-    override def onBlockEnd(blockId: String): Unit = {
-      callbackSeq :+= s"block end $blockId"
-    }
-
-    override def onBlockletStart(blockletId: Int): Unit = {
-      callbackSeq :+= s"blocklet start $blockletId"
-    }
-
-    /**
-     * Start of new block notification.
-     *
-     * @param blockId file name of the carbondata file
-     */
-    override def onBlockStart(blockId: String) = {
-      callbackSeq :+= s"block start $blockId"
-    }
-
-    /**
-     * This is called during closing of writer.So after this call no more data will be sent to this
-     * class.
-     */
-    override def finish() = {
-
-    }
-  }
-}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/carbondata/blob/5a625f4c/integration/spark-common-test/src/test/scala/org/apache/carbondata/spark/testsuite/datamap/FGDataMapTestCase.scala
----------------------------------------------------------------------
diff --git a/integration/spark-common-test/src/test/scala/org/apache/carbondata/spark/testsuite/datamap/FGDataMapTestCase.scala b/integration/spark-common-test/src/test/scala/org/apache/carbondata/spark/testsuite/datamap/FGDataMapTestCase.scala
deleted file mode 100644
index 8031dc2..0000000
--- a/integration/spark-common-test/src/test/scala/org/apache/carbondata/spark/testsuite/datamap/FGDataMapTestCase.scala
+++ /dev/null
@@ -1,476 +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.spark.testsuite.datamap
-
-import java.io.{ByteArrayInputStream, DataOutputStream, ObjectInputStream, ObjectOutputStream}
-
-import scala.collection.JavaConverters._
-import scala.collection.mutable.ArrayBuffer
-
-import com.sun.xml.internal.messaging.saaj.util.ByteOutputStream
-import org.apache.spark.sql.test.util.QueryTest
-import org.scalatest.BeforeAndAfterAll
-
-import org.apache.carbondata.core.datamap.dev.fgdatamap.{AbstractFineGrainDataMap, AbstractFineGrainDataMapFactory}
-import org.apache.carbondata.core.datamap.dev.{AbstractDataMapWriter, DataMapModel}
-import org.apache.carbondata.core.datamap.{DataMapDistributable, DataMapMeta, DataMapStoreManager}
-import org.apache.carbondata.core.datastore.FileReader
-import org.apache.carbondata.core.datastore.block.SegmentProperties
-import org.apache.carbondata.core.datastore.compression.SnappyCompressor
-import org.apache.carbondata.core.datastore.filesystem.{CarbonFile, CarbonFileFilter}
-import org.apache.carbondata.core.datastore.impl.FileFactory
-import org.apache.carbondata.core.datastore.page.ColumnPage
-import org.apache.carbondata.core.indexstore.FineGrainBlocklet
-import org.apache.carbondata.core.indexstore.blockletindex.BlockletDataMapDistributable
-import org.apache.carbondata.core.metadata.schema.table.DataMapSchema
-import org.apache.carbondata.core.metadata.{AbsoluteTableIdentifier, CarbonMetadata}
-import org.apache.carbondata.core.scan.expression.Expression
-import org.apache.carbondata.core.scan.expression.conditional.EqualToExpression
-import org.apache.carbondata.core.scan.filter.intf.ExpressionType
-import org.apache.carbondata.core.scan.filter.resolver.FilterResolverIntf
-import org.apache.carbondata.core.util.ByteUtil
-import org.apache.carbondata.core.util.path.CarbonTablePath
-import org.apache.carbondata.events.Event
-import org.apache.carbondata.spark.testsuite.datacompaction.CompactionSupportGlobalSortBigFileTest
-
-class FGDataMapFactory extends AbstractFineGrainDataMapFactory {
-  var identifier: AbsoluteTableIdentifier = _
-  var dataMapSchema: DataMapSchema = _
-
-  /**
-   * Initialization of Datamap factory with the identifier and datamap name
-   */
-  override def init(identifier: AbsoluteTableIdentifier, dataMapSchema: DataMapSchema): Unit = {
-    this.identifier = identifier
-    this.dataMapSchema = dataMapSchema
-  }
-
-  /**
-   * Return a new write for this datamap
-   */
-  override def createWriter(segmentId: String, dataWritePath: String): AbstractDataMapWriter = {
-    new FGDataMapWriter(identifier, segmentId, dataWritePath, dataMapSchema)
-  }
-
-  /**
-   * Get the datamap for segmentid
-   */
-  override def getDataMaps(segmentId: String): java.util.List[AbstractFineGrainDataMap] = {
-    val file = FileFactory
-      .getCarbonFile(CarbonTablePath.getSegmentPath(identifier.getTablePath, segmentId))
-
-    val files = file.listFiles(new CarbonFileFilter {
-      override def accept(file: CarbonFile): Boolean = file.getName.endsWith(".datamap")
-    })
-    files.map { f =>
-      val dataMap: AbstractFineGrainDataMap = new FGDataMap()
-      dataMap.init(new DataMapModel(f.getCanonicalPath))
-      dataMap
-    }.toList.asJava
-  }
-
-  /**
-   * Get datamap for distributable object.
-   */
-  override def getDataMaps(
-      distributable: DataMapDistributable): java.util.List[AbstractFineGrainDataMap]= {
-    val mapDistributable = distributable.asInstanceOf[BlockletDataMapDistributable]
-    val dataMap: AbstractFineGrainDataMap = new FGDataMap()
-    dataMap.init(new DataMapModel(mapDistributable.getFilePath))
-    Seq(dataMap).asJava
-  }
-
-  /**
-   * Get all distributable objects of a segmentid
-   *
-   * @return
-   */
-  override def toDistributable(segmentId: String): java.util.List[DataMapDistributable] = {
-    val file = FileFactory
-      .getCarbonFile(CarbonTablePath.getSegmentPath(identifier.getTablePath, segmentId))
-
-    val files = file.listFiles(new CarbonFileFilter {
-      override def accept(file: CarbonFile): Boolean = file.getName.endsWith(".datamap")
-    })
-    files.map { f =>
-      val d: DataMapDistributable = new BlockletDataMapDistributable(f.getCanonicalPath)
-      d
-    }.toList.asJava
-  }
-
-
-  /**
-   *
-   * @param event
-   */
-  override def fireEvent(event: Event):Unit = {
-    ???
-  }
-
-  /**
-   * Clears datamap of the segment
-   */
-  override def clear(segmentId: String): Unit = {
-  }
-
-  /**
-   * Clear all datamaps from memory
-   */
-  override def clear(): Unit = {
-  }
-
-  /**
-   * Return metadata of this datamap
-   */
-  override def getMeta: DataMapMeta = {
-    new DataMapMeta(dataMapSchema.getProperties.get("indexcolumns").split(",").toList.asJava,
-      List(ExpressionType.EQUALS, ExpressionType.IN).asJava)
-  }
-}
-
-class FGDataMap extends AbstractFineGrainDataMap {
-
-  var maxMin: ArrayBuffer[(String, Int, (Array[Byte], Array[Byte]), Long, Int)] = _
-  var FileReader: FileReader = _
-  var filePath: String = _
-  val compressor = new SnappyCompressor
-
-  /**
-   * It is called to load the data map to memory or to initialize it.
-   */
-  override def init(dataMapModel: DataMapModel): Unit = {
-    this.filePath = dataMapModel.getFilePath
-    val size = FileFactory.getCarbonFile(filePath).getSize
-    FileReader = FileFactory.getFileHolder(FileFactory.getFileType(filePath))
-    val footerLen = FileReader.readInt(filePath, size - 4)
-    val bytes = FileReader.readByteArray(filePath, size - footerLen - 4, footerLen)
-    val in = new ByteArrayInputStream(compressor.unCompressByte(bytes))
-    val obj = new ObjectInputStream(in)
-    maxMin = obj.readObject()
-      .asInstanceOf[ArrayBuffer[(String, Int, (Array[Byte], Array[Byte]), Long, Int)]]
-  }
-
-  /**
-   * Prune the datamap with filter expression. It returns the list of
-   * blocklets where these filters can exist.
-   *
-   * @param filterExp
-   * @return
-   */
-  override def prune(
-      filterExp: FilterResolverIntf,
-      segmentProperties: SegmentProperties,
-      partitions: java.util.List[String]): java.util.List[FineGrainBlocklet] = {
-    val buffer: ArrayBuffer[Expression] = new ArrayBuffer[Expression]()
-    val expression = filterExp.getFilterExpression
-    getEqualToExpression(expression, buffer)
-    val value = buffer.map { f =>
-      f.getChildren.get(1).evaluate(null).getString
-    }
-    val meta = findMeta(value(0).getBytes)
-    meta.map { f =>
-      readAndFindData(f, value(0).getBytes())
-    }.filter(_.isDefined).map(_.get).asJava
-  }
-
-  private def readAndFindData(meta: (String, Int, (Array[Byte], Array[Byte]), Long, Int),
-      value: Array[Byte]): Option[FineGrainBlocklet] = {
-    val bytes = FileReader.readByteArray(filePath, meta._4, meta._5)
-    val outputStream = new ByteArrayInputStream(compressor.unCompressByte(bytes))
-    val obj = new ObjectInputStream(outputStream)
-    val blockletsData = obj.readObject()
-      .asInstanceOf[ArrayBuffer[(Array[Byte], Seq[Seq[Int]], Seq[Int])]]
-
-    import scala.collection.Searching._
-    val searching = blockletsData
-      .search[(Array[Byte], Seq[Seq[Int]], Seq[Int])]((value, Seq(Seq(0)), Seq(0)))(new Ordering[
-      (Array[Byte], Seq[Seq[Int]], Seq[Int])] {
-      override def compare(x: (Array[Byte], Seq[Seq[Int]], Seq[Int]),
-          y: (Array[Byte], Seq[Seq[Int]], Seq[Int])) = {
-        ByteUtil.UnsafeComparer.INSTANCE.compareTo(x._1, y._1)
-      }
-    })
-    if (searching.insertionPoint >= 0) {
-      val f = blockletsData(searching.insertionPoint)
-      val pages = f._3.zipWithIndex.map { p =>
-        val pg = new FineGrainBlocklet.Page
-        pg.setPageId(p._1)
-        pg.setRowId(f._2(p._2).toArray)
-        pg
-      }
-      pages
-      Some(new FineGrainBlocklet(meta._1, meta._2.toString, pages.toList.asJava))
-    } else {
-      None
-    }
-
-  }
-
-  private def findMeta(value: Array[Byte]) = {
-    val tuples = maxMin.filter { f =>
-      ByteUtil.UnsafeComparer.INSTANCE.compareTo(value, f._3._1) >= 0 &&
-      ByteUtil.UnsafeComparer.INSTANCE.compareTo(value, f._3._2) <= 0
-    }
-    tuples
-  }
-
-  def getEqualToExpression(expression: Expression, buffer: ArrayBuffer[Expression]): Unit = {
-    if (expression.isInstanceOf[EqualToExpression]) {
-      buffer += expression
-    } else {
-      if (expression.getChildren != null) {
-        expression.getChildren.asScala.map { f =>
-          if (f.isInstanceOf[EqualToExpression]) {
-            buffer += f
-          }
-          getEqualToExpression(f, buffer)
-        }
-      }
-    }
-  }
-
-  /**
-   * Clear complete index table and release memory.
-   */
-  override def clear():Unit = {
-    ???
-  }
-
-  override def isScanRequired(filterExp: FilterResolverIntf): Boolean = ???
-}
-
-class FGDataMapWriter(identifier: AbsoluteTableIdentifier,
-    segmentId: String, dataWriterPath: String, dataMapSchema: DataMapSchema)
-  extends AbstractDataMapWriter(identifier, segmentId, dataWriterPath) {
-
-  var currentBlockId: String = null
-  val fgwritepath = dataWriterPath + "/" + dataMapSchema.getDataMapName + System.nanoTime() +
-                    ".datamap"
-  val stream: DataOutputStream = FileFactory
-    .getDataOutputStream(fgwritepath, FileFactory.getFileType(fgwritepath))
-  val blockletList = new ArrayBuffer[(Array[Byte], Seq[Int], Seq[Int])]()
-  val maxMin = new ArrayBuffer[(String, Int, (Array[Byte], Array[Byte]), Long, Int)]()
-  var position: Long = 0
-  val compressor = new SnappyCompressor
-
-  /**
-   * Start of new block notification.
-   *
-   * @param blockId file name of the carbondata file
-   */
-  override def onBlockStart(blockId: String): Unit = {
-    currentBlockId = blockId
-  }
-
-  /**
-   * End of block notification
-   */
-  override def onBlockEnd(blockId: String): Unit = {
-
-  }
-
-  /**
-   * Start of new blocklet notification.
-   *
-   * @param blockletId sequence number of blocklet in the block
-   */
-  override def onBlockletStart(blockletId: Int): Unit = {
-
-  }
-
-  /**
-   * End of blocklet notification
-   *
-   * @param blockletId sequence number of blocklet in the block
-   */
-  override def onBlockletEnd(blockletId: Int): Unit = {
-    val sorted = blockletList
-      .sortWith((l, r) => ByteUtil.UnsafeComparer.INSTANCE.compareTo(l._1, r._1) <= 0)
-    var oldValue: (Array[Byte], Seq[Seq[Int]], Seq[Int]) = null
-    var addedLast: Boolean = false
-    val blockletListUpdated = new ArrayBuffer[(Array[Byte], Seq[Seq[Int]], Seq[Int])]()
-    // Merge all same column values to single row.
-    sorted.foreach { f =>
-      if (oldValue != null) {
-        if (ByteUtil.UnsafeComparer.INSTANCE.compareTo(f._1, oldValue._1) == 0) {
-          oldValue = (oldValue._1, oldValue._2 ++ Seq(f._2), oldValue._3 ++ f._3)
-          addedLast = false
-        } else {
-          blockletListUpdated += oldValue
-          oldValue = (f._1, Seq(f._2), f._3)
-          addedLast = true
-        }
-      } else {
-        oldValue = (f._1, Seq(f._2), f._3)
-        addedLast = false
-      }
-    }
-    if (!addedLast && oldValue != null) {
-      blockletListUpdated += oldValue
-    }
-
-    val out = new ByteOutputStream()
-    val outStream = new ObjectOutputStream(out)
-    outStream.writeObject(blockletListUpdated)
-    outStream.close()
-    val bytes = compressor.compressByte(out.getBytes)
-    stream.write(bytes)
-    maxMin +=
-    ((currentBlockId + "", blockletId, (blockletListUpdated.head._1, blockletListUpdated.last
-      ._1), position, bytes.length))
-    position += bytes.length
-    blockletList.clear()
-  }
-
-  /**
-   * Add the column pages row to the datamap, order of pages is same as `indexColumns` in
-   * DataMapMeta returned in DataMapFactory.
-   *
-   * Implementation should copy the content of `pages` as needed, because `pages` memory
-   * may be freed after this method returns, if using unsafe column page.
-   */
-  override def onPageAdded(blockletId: Int,
-      pageId: Int,
-      pages: Array[ColumnPage]): Unit = {
-    val size = pages(0).getPageSize
-    val list = new ArrayBuffer[(Array[Byte], Int)]()
-    var i = 0
-    while (i < size) {
-      val bytes = pages(0).getBytes(i)
-      val newBytes = new Array[Byte](bytes.length - 2)
-      System.arraycopy(bytes, 2, newBytes, 0, newBytes.length)
-      list += ((newBytes, i))
-      i = i + 1
-    }
-    // Sort based on the column data in order to create index.
-    val sorted = list
-      .sortWith((l, r) => ByteUtil.UnsafeComparer.INSTANCE.compareTo(l._1, r._1) <= 0)
-    var oldValue: (Array[Byte], Seq[Int], Seq[Int]) = null
-    var addedLast: Boolean = false
-    // Merge all same column values to single row.
-    sorted.foreach { f =>
-      if (oldValue != null) {
-        if (ByteUtil.UnsafeComparer.INSTANCE.compareTo(f._1, oldValue._1) == 0) {
-          oldValue = (oldValue._1, oldValue._2 ++ Seq(f._2), oldValue._3)
-          addedLast = false
-        } else {
-          blockletList += oldValue
-          oldValue = (f._1, Seq(f._2), Seq(pageId))
-          addedLast = true
-        }
-      } else {
-        oldValue = (f._1, Seq(f._2), Seq(pageId))
-        addedLast = false
-      }
-    }
-    if (!addedLast && oldValue != null) {
-      blockletList += oldValue
-    }
-  }
-
-
-  /**
-   * This is called during closing of writer.So after this call no more data will be sent to this
-   * class.
-   */
-  override def finish(): Unit = {
-    val out = new ByteOutputStream()
-    val outStream = new ObjectOutputStream(out)
-    outStream.writeObject(maxMin)
-    outStream.close()
-    val bytes = compressor.compressByte(out.getBytes)
-    stream.write(bytes)
-    stream.writeInt(bytes.length)
-    stream.close()
-    commitFile(fgwritepath)
-  }
-
-
-}
-
-class FGDataMapTestCase extends QueryTest with BeforeAndAfterAll {
-
-  val file2 = resourcesPath + "/compaction/fil2.csv"
-
-  override protected def beforeAll(): Unit = {
-    //n should be about 5000000 of reset if size is default 1024
-    val n = 150000
-    CompactionSupportGlobalSortBigFileTest.createFile(file2, n * 4, n)
-    sql("DROP TABLE IF EXISTS normal_test")
-    sql(
-      """
-        | CREATE TABLE normal_test(id INT, name STRING, city STRING, age INT)
-        | STORED BY 'org.apache.carbondata.format'
-        | TBLPROPERTIES('SORT_COLUMNS'='city,name', 'SORT_SCOPE'='LOCAL_SORT')
-      """.stripMargin)
-    sql(s"LOAD DATA LOCAL INPATH '$file2' INTO TABLE normal_test OPTIONS('header'='false')")
-  }
-
-  test("test fg datamap") {
-    sql("DROP TABLE IF EXISTS datamap_test")
-    sql(
-      """
-        | CREATE TABLE datamap_test(id INT, name STRING, city STRING, age INT)
-        | STORED BY 'org.apache.carbondata.format'
-        | TBLPROPERTIES('SORT_COLUMNS'='city,name', 'SORT_SCOPE'='LOCAL_SORT')
-      """.stripMargin)
-    val table = CarbonMetadata.getInstance().getCarbonTable("default_datamap_test")
-    // register datamap writer
-    sql(
-      s"""
-         | CREATE DATAMAP ggdatamap ON TABLE datamap_test
-         | USING '${classOf[FGDataMapFactory].getName}'
-         | DMPROPERTIES('indexcolumns'='name')
-       """.stripMargin)
-    sql(s"LOAD DATA LOCAL INPATH '$file2' INTO TABLE datamap_test OPTIONS('header'='false')")
-    checkAnswer(sql("select * from datamap_test where name='n502670'"),
-      sql("select * from normal_test where name='n502670'"))
-  }
-
-  test("test fg datamap with 2 datamaps ") {
-    sql("DROP TABLE IF EXISTS datamap_test")
-    sql(
-      """
-        | CREATE TABLE datamap_test(id INT, name STRING, city STRING, age INT)
-        | STORED BY 'org.apache.carbondata.format'
-        | TBLPROPERTIES('SORT_COLUMNS'='city,name', 'SORT_SCOPE'='LOCAL_SORT')
-      """.stripMargin)
-    val table = CarbonMetadata.getInstance().getCarbonTable("default_datamap_test")
-    // register datamap writer
-    sql(
-      s"""
-         | CREATE DATAMAP ggdatamap1 ON TABLE datamap_test
-         | USING '${classOf[FGDataMapFactory].getName}'
-         | DMPROPERTIES('indexcolumns'='name')
-       """.stripMargin)
-    sql(
-      s"""
-         | CREATE DATAMAP ggdatamap2 ON TABLE datamap_test
-         | USING '${classOf[FGDataMapFactory].getName}'
-         | DMPROPERTIES('indexcolumns'='city')
-       """.stripMargin)
-    sql(s"LOAD DATA LOCAL INPATH '$file2' INTO TABLE datamap_test OPTIONS('header'='false')")
-    checkAnswer(sql("select * from datamap_test where name='n502670' and city='c2670'"),
-      sql("select * from normal_test where name='n502670' and city='c2670'"))
-  }
-
-  override protected def afterAll(): Unit = {
-    CompactionSupportGlobalSortBigFileTest.deleteFile(file2)
-    sql("DROP TABLE IF EXISTS normal_test")
-    sql("DROP TABLE IF EXISTS datamap_test")
-  }
-}

http://git-wip-us.apache.org/repos/asf/carbondata/blob/5a625f4c/integration/spark-common-test/src/test/scala/org/apache/carbondata/spark/testsuite/datamap/FGIndexDataMapTestCase.scala
----------------------------------------------------------------------
diff --git a/integration/spark-common-test/src/test/scala/org/apache/carbondata/spark/testsuite/datamap/FGIndexDataMapTestCase.scala b/integration/spark-common-test/src/test/scala/org/apache/carbondata/spark/testsuite/datamap/FGIndexDataMapTestCase.scala
new file mode 100644
index 0000000..8ddad75
--- /dev/null
+++ b/integration/spark-common-test/src/test/scala/org/apache/carbondata/spark/testsuite/datamap/FGIndexDataMapTestCase.scala
@@ -0,0 +1,474 @@
+/*
+ * 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.spark.testsuite.datamap
+
+import java.io.{ByteArrayInputStream, DataOutputStream, ObjectInputStream, ObjectOutputStream}
+
+import scala.collection.JavaConverters._
+import scala.collection.mutable.ArrayBuffer
+
+import com.sun.xml.internal.messaging.saaj.util.ByteOutputStream
+import org.apache.spark.sql.test.util.QueryTest
+import org.scalatest.BeforeAndAfterAll
+
+import org.apache.carbondata.core.datamap.dev.fgdatamap.{AbstractFineGrainIndexDataMap, AbstractFineGrainIndexDataMapFactory}
+import org.apache.carbondata.core.datamap.dev.{AbstractDataMapWriter, DataMapModel}
+import org.apache.carbondata.core.datamap.{DataMapDistributable, DataMapMeta}
+import org.apache.carbondata.core.datastore.FileReader
+import org.apache.carbondata.core.datastore.block.SegmentProperties
+import org.apache.carbondata.core.datastore.compression.SnappyCompressor
+import org.apache.carbondata.core.datastore.filesystem.{CarbonFile, CarbonFileFilter}
+import org.apache.carbondata.core.datastore.impl.FileFactory
+import org.apache.carbondata.core.datastore.page.ColumnPage
+import org.apache.carbondata.core.indexstore.FineGrainBlocklet
+import org.apache.carbondata.core.indexstore.blockletindex.BlockletDataMapDistributable
+import org.apache.carbondata.core.metadata.schema.table.DataMapSchema
+import org.apache.carbondata.core.metadata.{AbsoluteTableIdentifier, CarbonMetadata}
+import org.apache.carbondata.core.scan.expression.Expression
+import org.apache.carbondata.core.scan.expression.conditional.EqualToExpression
+import org.apache.carbondata.core.scan.filter.intf.ExpressionType
+import org.apache.carbondata.core.scan.filter.resolver.FilterResolverIntf
+import org.apache.carbondata.core.util.ByteUtil
+import org.apache.carbondata.core.util.path.CarbonTablePath
+import org.apache.carbondata.events.Event
+import org.apache.carbondata.spark.testsuite.datacompaction.CompactionSupportGlobalSortBigFileTest
+
+class FGIndexDataMapFactory extends AbstractFineGrainIndexDataMapFactory {
+  var identifier: AbsoluteTableIdentifier = _
+  var dataMapSchema: DataMapSchema = _
+
+  /**
+   * Initialization of Datamap factory with the identifier and datamap name
+   */
+  override def init(identifier: AbsoluteTableIdentifier, dataMapSchema: DataMapSchema): Unit = {
+    this.identifier = identifier
+    this.dataMapSchema = dataMapSchema
+  }
+
+  /**
+   * Return a new write for this datamap
+   */
+  override def createWriter(segmentId: String, dataWritePath: String): AbstractDataMapWriter = {
+    new FGDataMapWriter(identifier, segmentId, dataWritePath, dataMapSchema)
+  }
+
+  /**
+   * Get the datamap for segmentid
+   */
+  override def getDataMaps(segmentId: String): java.util.List[AbstractFineGrainIndexDataMap] = {
+    val file = FileFactory
+      .getCarbonFile(CarbonTablePath.getSegmentPath(identifier.getTablePath, segmentId))
+
+    val files = file.listFiles(new CarbonFileFilter {
+      override def accept(file: CarbonFile): Boolean = file.getName.endsWith(".datamap")
+    })
+    files.map { f =>
+      val dataMap: AbstractFineGrainIndexDataMap = new FGIndexDataMap()
+      dataMap.init(new DataMapModel(f.getCanonicalPath))
+      dataMap
+    }.toList.asJava
+  }
+
+  /**
+   * Get datamap for distributable object.
+   */
+  override def getDataMaps(
+      distributable: DataMapDistributable): java.util.List[AbstractFineGrainIndexDataMap]= {
+    val mapDistributable = distributable.asInstanceOf[BlockletDataMapDistributable]
+    val dataMap: AbstractFineGrainIndexDataMap = new FGIndexDataMap()
+    dataMap.init(new DataMapModel(mapDistributable.getFilePath))
+    Seq(dataMap).asJava
+  }
+
+  /**
+   * Get all distributable objects of a segmentid
+   *
+   * @return
+   */
+  override def toDistributable(segmentId: String): java.util.List[DataMapDistributable] = {
+    val file = FileFactory
+      .getCarbonFile(CarbonTablePath.getSegmentPath(identifier.getTablePath, segmentId))
+
+    val files = file.listFiles(new CarbonFileFilter {
+      override def accept(file: CarbonFile): Boolean = file.getName.endsWith(".datamap")
+    })
+    files.map { f =>
+      val d: DataMapDistributable = new BlockletDataMapDistributable(f.getCanonicalPath)
+      d
+    }.toList.asJava
+  }
+
+
+  /**
+   *
+   * @param event
+   */
+  override def fireEvent(event: Event):Unit = {
+    ???
+  }
+
+  /**
+   * Clears datamap of the segment
+   */
+  override def clear(segmentId: String): Unit = {
+  }
+
+  /**
+   * Clear all datamaps from memory
+   */
+  override def clear(): Unit = {
+  }
+
+  /**
+   * Return metadata of this datamap
+   */
+  override def getMeta: DataMapMeta = {
+    new DataMapMeta(dataMapSchema.getProperties.get("indexcolumns").split(",").toList.asJava,
+      List(ExpressionType.EQUALS, ExpressionType.IN).asJava)
+  }
+}
+
+class FGIndexDataMap extends AbstractFineGrainIndexDataMap {
+
+  var maxMin: ArrayBuffer[(String, Int, (Array[Byte], Array[Byte]), Long, Int)] = _
+  var FileReader: FileReader = _
+  var filePath: String = _
+  val compressor = new SnappyCompressor
+
+  /**
+   * It is called to load the data map to memory or to initialize it.
+   */
+  override def init(dataMapModel: DataMapModel): Unit = {
+    this.filePath = dataMapModel.getFilePath
+    val size = FileFactory.getCarbonFile(filePath).getSize
+    FileReader = FileFactory.getFileHolder(FileFactory.getFileType(filePath))
+    val footerLen = FileReader.readInt(filePath, size - 4)
+    val bytes = FileReader.readByteArray(filePath, size - footerLen - 4, footerLen)
+    val in = new ByteArrayInputStream(compressor.unCompressByte(bytes))
+    val obj = new ObjectInputStream(in)
+    maxMin = obj.readObject()
+      .asInstanceOf[ArrayBuffer[(String, Int, (Array[Byte], Array[Byte]), Long, Int)]]
+  }
+
+  /**
+   * Prune the datamap with filter expression. It returns the list of
+   * blocklets where these filters can exist.
+   *
+   * @param filterExp
+   * @return
+   */
+  override def prune(
+      filterExp: FilterResolverIntf,
+      segmentProperties: SegmentProperties,
+      partitions: java.util.List[String]): java.util.List[FineGrainBlocklet] = {
+    val buffer: ArrayBuffer[Expression] = new ArrayBuffer[Expression]()
+    val expression = filterExp.getFilterExpression
+    getEqualToExpression(expression, buffer)
+    val value = buffer.map { f =>
+      f.getChildren.get(1).evaluate(null).getString
+    }
+    val meta = findMeta(value(0).getBytes)
+    meta.map { f =>
+      readAndFindData(f, value(0).getBytes())
+    }.filter(_.isDefined).map(_.get).asJava
+  }
+
+  private def readAndFindData(meta: (String, Int, (Array[Byte], Array[Byte]), Long, Int),
+      value: Array[Byte]): Option[FineGrainBlocklet] = {
+    val bytes = FileReader.readByteArray(filePath, meta._4, meta._5)
+    val outputStream = new ByteArrayInputStream(compressor.unCompressByte(bytes))
+    val obj = new ObjectInputStream(outputStream)
+    val blockletsData = obj.readObject()
+      .asInstanceOf[ArrayBuffer[(Array[Byte], Seq[Seq[Int]], Seq[Int])]]
+
+    import scala.collection.Searching._
+    val searching = blockletsData
+      .search[(Array[Byte], Seq[Seq[Int]], Seq[Int])]((value, Seq(Seq(0)), Seq(0)))(new Ordering[
+      (Array[Byte], Seq[Seq[Int]], Seq[Int])] {
+      override def compare(x: (Array[Byte], Seq[Seq[Int]], Seq[Int]),
+          y: (Array[Byte], Seq[Seq[Int]], Seq[Int])) = {
+        ByteUtil.UnsafeComparer.INSTANCE.compareTo(x._1, y._1)
+      }
+    })
+    if (searching.insertionPoint >= 0) {
+      val f = blockletsData(searching.insertionPoint)
+      val pages = f._3.zipWithIndex.map { p =>
+        val pg = new FineGrainBlocklet.Page
+        pg.setPageId(p._1)
+        pg.setRowId(f._2(p._2).toArray)
+        pg
+      }
+      pages
+      Some(new FineGrainBlocklet(meta._1, meta._2.toString, pages.toList.asJava))
+    } else {
+      None
+    }
+
+  }
+
+  private def findMeta(value: Array[Byte]) = {
+    val tuples = maxMin.filter { f =>
+      ByteUtil.UnsafeComparer.INSTANCE.compareTo(value, f._3._1) >= 0 &&
+      ByteUtil.UnsafeComparer.INSTANCE.compareTo(value, f._3._2) <= 0
+    }
+    tuples
+  }
+
+  def getEqualToExpression(expression: Expression, buffer: ArrayBuffer[Expression]): Unit = {
+    if (expression.isInstanceOf[EqualToExpression]) {
+      buffer += expression
+    } else {
+      if (expression.getChildren != null) {
+        expression.getChildren.asScala.map { f =>
+          if (f.isInstanceOf[EqualToExpression]) {
+            buffer += f
+          }
+          getEqualToExpression(f, buffer)
+        }
+      }
+    }
+  }
+
+  /**
+   * Clear complete index table and release memory.
+   */
+  override def clear():Unit = {
+    ???
+  }
+
+  override def isScanRequired(filterExp: FilterResolverIntf): Boolean = ???
+}
+
+class FGDataMapWriter(identifier: AbsoluteTableIdentifier,
+    segmentId: String, dataWriterPath: String, dataMapSchema: DataMapSchema)
+  extends AbstractDataMapWriter(identifier, segmentId, dataWriterPath) {
+
+  var currentBlockId: String = null
+  val fgwritepath = dataWriterPath + "/" + dataMapSchema.getDataMapName + System.nanoTime() +
+                    ".datamap"
+  val stream: DataOutputStream = FileFactory
+    .getDataOutputStream(fgwritepath, FileFactory.getFileType(fgwritepath))
+  val blockletList = new ArrayBuffer[(Array[Byte], Seq[Int], Seq[Int])]()
+  val maxMin = new ArrayBuffer[(String, Int, (Array[Byte], Array[Byte]), Long, Int)]()
+  var position: Long = 0
+  val compressor = new SnappyCompressor
+
+  /**
+   * Start of new block notification.
+   *
+   * @param blockId file name of the carbondata file
+   */
+  override def onBlockStart(blockId: String): Unit = {
+    currentBlockId = blockId
+  }
+
+  /**
+   * End of block notification
+   */
+  override def onBlockEnd(blockId: String): Unit = {
+
+  }
+
+  /**
+   * Start of new blocklet notification.
+   *
+   * @param blockletId sequence number of blocklet in the block
+   */
+  override def onBlockletStart(blockletId: Int): Unit = {
+
+  }
+
+  /**
+   * End of blocklet notification
+   *
+   * @param blockletId sequence number of blocklet in the block
+   */
+  override def onBlockletEnd(blockletId: Int): Unit = {
+    val sorted = blockletList
+      .sortWith((l, r) => ByteUtil.UnsafeComparer.INSTANCE.compareTo(l._1, r._1) <= 0)
+    var oldValue: (Array[Byte], Seq[Seq[Int]], Seq[Int]) = null
+    var addedLast: Boolean = false
+    val blockletListUpdated = new ArrayBuffer[(Array[Byte], Seq[Seq[Int]], Seq[Int])]()
+    // Merge all same column values to single row.
+    sorted.foreach { f =>
+      if (oldValue != null) {
+        if (ByteUtil.UnsafeComparer.INSTANCE.compareTo(f._1, oldValue._1) == 0) {
+          oldValue = (oldValue._1, oldValue._2 ++ Seq(f._2), oldValue._3 ++ f._3)
+          addedLast = false
+        } else {
+          blockletListUpdated += oldValue
+          oldValue = (f._1, Seq(f._2), f._3)
+          addedLast = true
+        }
+      } else {
+        oldValue = (f._1, Seq(f._2), f._3)
+        addedLast = false
+      }
+    }
+    if (!addedLast && oldValue != null) {
+      blockletListUpdated += oldValue
+    }
+
+    val out = new ByteOutputStream()
+    val outStream = new ObjectOutputStream(out)
+    outStream.writeObject(blockletListUpdated)
+    outStream.close()
+    val bytes = compressor.compressByte(out.getBytes)
+    stream.write(bytes)
+    maxMin +=
+    ((currentBlockId + "", blockletId, (blockletListUpdated.head._1, blockletListUpdated.last
+      ._1), position, bytes.length))
+    position += bytes.length
+    blockletList.clear()
+  }
+
+  /**
+   * Add the column pages row to the datamap, order of pages is same as `indexColumns` in
+   * 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.
+   */
+  override def onPageAdded(blockletId: Int,
+      pageId: Int,
+      pages: Array[ColumnPage]): Unit = {
+    val size = pages(0).getPageSize
+    val list = new ArrayBuffer[(Array[Byte], Int)]()
+    var i = 0
+    while (i < size) {
+      val bytes = pages(0).getBytes(i)
+      val newBytes = new Array[Byte](bytes.length - 2)
+      System.arraycopy(bytes, 2, newBytes, 0, newBytes.length)
+      list += ((newBytes, i))
+      i = i + 1
+    }
+    // Sort based on the column data in order to create index.
+    val sorted = list
+      .sortWith((l, r) => ByteUtil.UnsafeComparer.INSTANCE.compareTo(l._1, r._1) <= 0)
+    var oldValue: (Array[Byte], Seq[Int], Seq[Int]) = null
+    var addedLast: Boolean = false
+    // Merge all same column values to single row.
+    sorted.foreach { f =>
+      if (oldValue != null) {
+        if (ByteUtil.UnsafeComparer.INSTANCE.compareTo(f._1, oldValue._1) == 0) {
+          oldValue = (oldValue._1, oldValue._2 ++ Seq(f._2), oldValue._3)
+          addedLast = false
+        } else {
+          blockletList += oldValue
+          oldValue = (f._1, Seq(f._2), Seq(pageId))
+          addedLast = true
+        }
+      } else {
+        oldValue = (f._1, Seq(f._2), Seq(pageId))
+        addedLast = false
+      }
+    }
+    if (!addedLast && oldValue != null) {
+      blockletList += oldValue
+    }
+  }
+
+
+  /**
+   * This is called during closing of writer.So after this call no more data will be sent to this
+   * class.
+   */
+  override def finish(): Unit = {
+    val out = new ByteOutputStream()
+    val outStream = new ObjectOutputStream(out)
+    outStream.writeObject(maxMin)
+    outStream.close()
+    val bytes = compressor.compressByte(out.getBytes)
+    stream.write(bytes)
+    stream.writeInt(bytes.length)
+    stream.close()
+    commitFile(fgwritepath)
+  }
+}
+
+class FGIndexDataMapTestCase extends QueryTest with BeforeAndAfterAll {
+
+  val file2 = resourcesPath + "/compaction/fil2.csv"
+
+  override protected def beforeAll(): Unit = {
+    //n should be about 5000000 of reset if size is default 1024
+    val n = 150000
+    CompactionSupportGlobalSortBigFileTest.createFile(file2, n * 4, n)
+    sql("DROP TABLE IF EXISTS normal_test")
+    sql(
+      """
+        | CREATE TABLE normal_test(id INT, name STRING, city STRING, age INT)
+        | STORED BY 'org.apache.carbondata.format'
+        | TBLPROPERTIES('SORT_COLUMNS'='city,name', 'SORT_SCOPE'='LOCAL_SORT')
+      """.stripMargin)
+    sql(s"LOAD DATA LOCAL INPATH '$file2' INTO TABLE normal_test OPTIONS('header'='false')")
+  }
+
+  test("test fg datamap") {
+    sql("DROP TABLE IF EXISTS datamap_test")
+    sql(
+      """
+        | CREATE TABLE datamap_test(id INT, name STRING, city STRING, age INT)
+        | STORED BY 'org.apache.carbondata.format'
+        | TBLPROPERTIES('SORT_COLUMNS'='city,name', 'SORT_SCOPE'='LOCAL_SORT')
+      """.stripMargin)
+    val table = CarbonMetadata.getInstance().getCarbonTable("default_datamap_test")
+    // register datamap writer
+    sql(
+      s"""
+         | CREATE DATAMAP ggdatamap ON TABLE datamap_test
+         | USING '${classOf[FGIndexDataMapFactory].getName}'
+         | DMPROPERTIES('indexcolumns'='name')
+       """.stripMargin)
+    sql(s"LOAD DATA LOCAL INPATH '$file2' INTO TABLE datamap_test OPTIONS('header'='false')")
+    checkAnswer(sql("select * from datamap_test where name='n502670'"),
+      sql("select * from normal_test where name='n502670'"))
+  }
+
+  test("test fg datamap with 2 datamaps ") {
+    sql("DROP TABLE IF EXISTS datamap_test")
+    sql(
+      """
+        | CREATE TABLE datamap_test(id INT, name STRING, city STRING, age INT)
+        | STORED BY 'org.apache.carbondata.format'
+        | TBLPROPERTIES('SORT_COLUMNS'='city,name', 'SORT_SCOPE'='LOCAL_SORT')
+      """.stripMargin)
+    val table = CarbonMetadata.getInstance().getCarbonTable("default_datamap_test")
+    // register datamap writer
+    sql(
+      s"""
+         | CREATE DATAMAP ggdatamap1 ON TABLE datamap_test
+         | USING '${classOf[FGIndexDataMapFactory].getName}'
+         | DMPROPERTIES('indexcolumns'='name')
+       """.stripMargin)
+    sql(
+      s"""
+         | CREATE DATAMAP ggdatamap2 ON TABLE datamap_test
+         | USING '${classOf[FGIndexDataMapFactory].getName}'
+         | DMPROPERTIES('indexcolumns'='city')
+       """.stripMargin)
+    sql(s"LOAD DATA LOCAL INPATH '$file2' INTO TABLE datamap_test OPTIONS('header'='false')")
+    checkAnswer(sql("select * from datamap_test where name='n502670' and city='c2670'"),
+      sql("select * from normal_test where name='n502670' and city='c2670'"))
+  }
+
+  override protected def afterAll(): Unit = {
+    CompactionSupportGlobalSortBigFileTest.deleteFile(file2)
+    sql("DROP TABLE IF EXISTS normal_test")
+    sql("DROP TABLE IF EXISTS datamap_test")
+  }
+}

http://git-wip-us.apache.org/repos/asf/carbondata/blob/5a625f4c/integration/spark-common-test/src/test/scala/org/apache/carbondata/spark/testsuite/datamap/IndexDataMapWriterSuite.scala
----------------------------------------------------------------------
diff --git a/integration/spark-common-test/src/test/scala/org/apache/carbondata/spark/testsuite/datamap/IndexDataMapWriterSuite.scala b/integration/spark-common-test/src/test/scala/org/apache/carbondata/spark/testsuite/datamap/IndexDataMapWriterSuite.scala
new file mode 100644
index 0000000..5fd8ae9
--- /dev/null
+++ b/integration/spark-common-test/src/test/scala/org/apache/carbondata/spark/testsuite/datamap/IndexDataMapWriterSuite.scala
@@ -0,0 +1,216 @@
+/*
+ * 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.spark.testsuite.datamap
+
+import java.util
+
+import scala.collection.JavaConverters._
+
+import org.apache.spark.sql.test.util.QueryTest
+import org.apache.spark.sql.{DataFrame, SaveMode}
+import org.scalatest.BeforeAndAfterAll
+
+import org.apache.carbondata.core.constants.CarbonCommonConstants
+import org.apache.carbondata.core.datamap.dev.AbstractDataMapWriter
+import org.apache.carbondata.core.datamap.dev.cgdatamap.{AbstractCoarseGrainIndexDataMap, AbstractCoarseGrainIndexDataMapFactory}
+import org.apache.carbondata.core.datamap.{DataMapDistributable, DataMapMeta}
+import org.apache.carbondata.core.datastore.page.ColumnPage
+import org.apache.carbondata.core.metadata.AbsoluteTableIdentifier
+import org.apache.carbondata.core.metadata.schema.table.DataMapSchema
+import org.apache.carbondata.core.scan.filter.intf.ExpressionType
+import org.apache.carbondata.core.metadata.datatype.DataTypes
+import org.apache.carbondata.core.util.CarbonProperties
+import org.apache.carbondata.events.Event
+
+class C2IndexDataMapFactory() extends AbstractCoarseGrainIndexDataMapFactory {
+
+  var identifier: AbsoluteTableIdentifier = _
+
+  override def init(identifier: AbsoluteTableIdentifier,
+      dataMapSchema: DataMapSchema): Unit = {
+    this.identifier = identifier
+  }
+
+  override def fireEvent(event: Event): Unit = ???
+
+  override def clear(segmentId: String): Unit = {}
+
+  override def clear(): Unit = {}
+
+  override def getDataMaps(distributable: DataMapDistributable): java.util.List[AbstractCoarseGrainIndexDataMap] = ???
+
+  override def getDataMaps(segmentId: String): util.List[AbstractCoarseGrainIndexDataMap] = ???
+
+  override def createWriter(segmentId: String, dataWritePath: String): AbstractDataMapWriter =
+    IndexDataMapWriterSuite.dataMapWriterC2Mock(identifier, segmentId, dataWritePath)
+
+  override def getMeta: DataMapMeta = new DataMapMeta(List("c2").asJava, List(ExpressionType.EQUALS).asJava)
+
+  /**
+   * Get all distributable objects of a segmentid
+   *
+   * @return
+   */
+  override def toDistributable(segmentId: String): util.List[DataMapDistributable] = {
+    ???
+  }
+
+}
+
+class IndexDataMapWriterSuite extends QueryTest with BeforeAndAfterAll {
+  def buildTestData(numRows: Int): DataFrame = {
+    import sqlContext.implicits._
+    sqlContext.sparkContext.parallelize(1 to numRows, 1)
+      .map(x => ("a" + x, "b", x))
+      .toDF("c1", "c2", "c3")
+  }
+
+  def dropTable(): Unit = {
+    sql("DROP TABLE IF EXISTS carbon1")
+    sql("DROP TABLE IF EXISTS carbon2")
+  }
+
+  override def beforeAll {
+    dropTable()
+  }
+
+  test("test write datamap 2 pages") {
+    sql(s"CREATE TABLE carbon1(c1 STRING, c2 STRING, c3 INT) STORED BY 'org.apache.carbondata.format'")
+    // register datamap writer
+    sql(s"CREATE DATAMAP test ON TABLE carbon1 USING '${classOf[C2IndexDataMapFactory].getName}'")
+    val df = buildTestData(33000)
+
+    // save dataframe to carbon file
+    df.write
+      .format("carbondata")
+      .option("tableName", "carbon1")
+      .option("tempCSV", "false")
+      .option("sort_columns","c1")
+      .mode(SaveMode.Overwrite)
+      .save()
+
+    assert(IndexDataMapWriterSuite.callbackSeq.head.contains("block start"))
+    assert(IndexDataMapWriterSuite.callbackSeq.last.contains("block end"))
+    assert(
+      IndexDataMapWriterSuite.callbackSeq.slice(1, IndexDataMapWriterSuite.callbackSeq.length - 1) == Seq(
+        "blocklet start 0",
+        "add page data: blocklet 0, page 0",
+        "add page data: blocklet 0, page 1",
+        "blocklet end: 0"
+      ))
+    IndexDataMapWriterSuite.callbackSeq = Seq()
+  }
+
+  test("test write datamap 2 blocklet") {
+    sql(s"CREATE TABLE carbon2(c1 STRING, c2 STRING, c3 INT) STORED BY 'org.apache.carbondata.format'")
+    sql(s"CREATE DATAMAP test ON TABLE carbon2 USING '${classOf[C2IndexDataMapFactory].getName}'")
+
+    CarbonProperties.getInstance()
+      .addProperty("carbon.blockletgroup.size.in.mb", "1")
+    CarbonProperties.getInstance()
+      .addProperty("carbon.number.of.cores.while.loading",
+        CarbonCommonConstants.NUM_CORES_DEFAULT_VAL)
+
+    val df = buildTestData(300000)
+
+    // save dataframe to carbon file
+    df.write
+      .format("carbondata")
+      .option("tableName", "carbon2")
+      .option("tempCSV", "false")
+      .option("sort_columns","c1")
+      .option("SORT_SCOPE","GLOBAL_SORT")
+      .mode(SaveMode.Overwrite)
+      .save()
+
+    assert(IndexDataMapWriterSuite.callbackSeq.head.contains("block start"))
+    assert(IndexDataMapWriterSuite.callbackSeq.last.contains("block end"))
+    // corrected test case the min "carbon.blockletgroup.size.in.mb" size could not be less than
+    // 64 MB
+    assert(
+      IndexDataMapWriterSuite.callbackSeq.slice(1, IndexDataMapWriterSuite.callbackSeq.length - 1) == Seq(
+        "blocklet start 0",
+        "add page data: blocklet 0, page 0",
+        "add page data: blocklet 0, page 1",
+        "add page data: blocklet 0, page 2",
+        "add page data: blocklet 0, page 3",
+        "add page data: blocklet 0, page 4",
+        "add page data: blocklet 0, page 5",
+        "add page data: blocklet 0, page 6",
+        "add page data: blocklet 0, page 7",
+        "add page data: blocklet 0, page 8",
+        "add page data: blocklet 0, page 9",
+        "blocklet end: 0"
+      ))
+    IndexDataMapWriterSuite.callbackSeq = Seq()
+  }
+
+  override def afterAll {
+    dropTable()
+  }
+}
+
+object IndexDataMapWriterSuite {
+
+  var callbackSeq: Seq[String] = Seq[String]()
+
+  def dataMapWriterC2Mock(identifier: AbsoluteTableIdentifier, segmentId: String,
+      dataWritePath: String) =
+    new AbstractDataMapWriter(identifier, segmentId, dataWritePath) {
+
+    override def onPageAdded(
+        blockletId: Int,
+        pageId: Int,
+        pages: Array[ColumnPage]): Unit = {
+      assert(pages.length == 1)
+      assert(pages(0).getDataType == DataTypes.STRING)
+      val bytes: Array[Byte] = pages(0).getByteArrayPage()(0)
+      assert(bytes.sameElements(Seq(0, 1, 'b'.toByte)))
+      callbackSeq :+= s"add page data: blocklet $blockletId, page $pageId"
+    }
+
+    override def onBlockletEnd(blockletId: Int): Unit = {
+      callbackSeq :+= s"blocklet end: $blockletId"
+    }
+
+    override def onBlockEnd(blockId: String): Unit = {
+      callbackSeq :+= s"block end $blockId"
+    }
+
+    override def onBlockletStart(blockletId: Int): Unit = {
+      callbackSeq :+= s"blocklet start $blockletId"
+    }
+
+    /**
+     * Start of new block notification.
+     *
+     * @param blockId file name of the carbondata file
+     */
+    override def onBlockStart(blockId: String) = {
+      callbackSeq :+= s"block start $blockId"
+    }
+
+    /**
+     * This is called during closing of writer.So after this call no more data will be sent to this
+     * class.
+     */
+    override def finish() = {
+
+    }
+  }
+}
\ No newline at end of file


[4/6] carbondata git commit: [CARBONDATA-2189] Add DataMapProvider developer interface

Posted by ja...@apache.org.
http://git-wip-us.apache.org/repos/asf/carbondata/blob/5a625f4c/core/src/main/java/org/apache/carbondata/core/indexstore/blockletindex/BlockletIndexDataMap.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/carbondata/core/indexstore/blockletindex/BlockletIndexDataMap.java b/core/src/main/java/org/apache/carbondata/core/indexstore/blockletindex/BlockletIndexDataMap.java
new file mode 100644
index 0000000..ef169af
--- /dev/null
+++ b/core/src/main/java/org/apache/carbondata/core/indexstore/blockletindex/BlockletIndexDataMap.java
@@ -0,0 +1,981 @@
+/*
+ * 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.indexstore.blockletindex;
+
+import java.io.ByteArrayInputStream;
+import java.io.ByteArrayOutputStream;
+import java.io.DataInputStream;
+import java.io.DataOutput;
+import java.io.DataOutputStream;
+import java.io.IOException;
+import java.io.UnsupportedEncodingException;
+import java.math.BigDecimal;
+import java.nio.ByteBuffer;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.BitSet;
+import java.util.Comparator;
+import java.util.List;
+
+import org.apache.carbondata.common.logging.LogService;
+import org.apache.carbondata.common.logging.LogServiceFactory;
+import org.apache.carbondata.core.cache.Cacheable;
+import org.apache.carbondata.core.constants.CarbonCommonConstants;
+import org.apache.carbondata.core.datamap.dev.DataMapModel;
+import org.apache.carbondata.core.datamap.dev.cgdatamap.AbstractCoarseGrainIndexDataMap;
+import org.apache.carbondata.core.datastore.IndexKey;
+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.Blocklet;
+import org.apache.carbondata.core.indexstore.BlockletDetailInfo;
+import org.apache.carbondata.core.indexstore.ExtendedBlocklet;
+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.schema.CarbonRowSchema;
+import org.apache.carbondata.core.keygenerator.KeyGenException;
+import org.apache.carbondata.core.memory.MemoryException;
+import org.apache.carbondata.core.metadata.blocklet.BlockletInfo;
+import org.apache.carbondata.core.metadata.blocklet.DataFileFooter;
+import org.apache.carbondata.core.metadata.blocklet.index.BlockletIndex;
+import org.apache.carbondata.core.metadata.blocklet.index.BlockletMinMaxIndex;
+import org.apache.carbondata.core.metadata.datatype.DataType;
+import org.apache.carbondata.core.metadata.datatype.DataTypes;
+import org.apache.carbondata.core.metadata.schema.table.column.CarbonMeasure;
+import org.apache.carbondata.core.metadata.schema.table.column.ColumnSchema;
+import org.apache.carbondata.core.scan.filter.FilterExpressionProcessor;
+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.ByteUtil;
+import org.apache.carbondata.core.util.DataFileFooterConverter;
+import org.apache.carbondata.core.util.DataTypeUtil;
+
+import org.apache.commons.lang3.StringUtils;
+import org.xerial.snappy.Snappy;
+
+/**
+ * Datamap implementation for blocklet.
+ */
+public class BlockletIndexDataMap extends AbstractCoarseGrainIndexDataMap implements Cacheable {
+
+  private static final LogService LOGGER =
+      LogServiceFactory.getLogService(BlockletIndexDataMap.class.getName());
+
+  private static int KEY_INDEX = 0;
+
+  private static int MIN_VALUES_INDEX = 1;
+
+  private static int MAX_VALUES_INDEX = 2;
+
+  private static int ROW_COUNT_INDEX = 3;
+
+  private static int FILE_PATH_INDEX = 4;
+
+  private static int PAGE_COUNT_INDEX = 5;
+
+  private static int VERSION_INDEX = 6;
+
+  private static int SCHEMA_UPADATED_TIME_INDEX = 7;
+
+  private static int BLOCK_INFO_INDEX = 8;
+
+  private static int BLOCK_FOOTER_OFFSET = 9;
+
+  private static int LOCATIONS = 10;
+
+  private static int BLOCKLET_ID_INDEX = 11;
+
+  private static int BLOCK_LENGTH = 12;
+
+  private static int TASK_MIN_VALUES_INDEX = 0;
+
+  private static int TASK_MAX_VALUES_INDEX = 1;
+
+  private static int SCHEMA = 2;
+
+  private static int PARTITION_INFO = 3;
+
+  private UnsafeMemoryDMStore unsafeMemoryDMStore;
+
+  private UnsafeMemoryDMStore unsafeMemorySummaryDMStore;
+
+  private SegmentProperties segmentProperties;
+
+  private int[] columnCardinality;
+
+  private boolean isPartitionedSegment;
+
+  @Override
+  public void init(DataMapModel dataMapModel) throws IOException, MemoryException {
+    long startTime = System.currentTimeMillis();
+    assert (dataMapModel instanceof BlockletDataMapModel);
+    BlockletDataMapModel blockletDataMapInfo = (BlockletDataMapModel) dataMapModel;
+    DataFileFooterConverter fileFooterConverter = new DataFileFooterConverter();
+    List<DataFileFooter> indexInfo = fileFooterConverter
+        .getIndexInfo(blockletDataMapInfo.getFilePath(), blockletDataMapInfo.getFileData());
+    isPartitionedSegment = blockletDataMapInfo.isPartitionedSegment();
+    DataMapRowImpl summaryRow = null;
+    byte[] schemaBinary = null;
+    // below 2 variables will be used for fetching the relative blocklet id. Relative blocklet ID
+    // is id assigned to a blocklet within a part file
+    String tempFilePath = null;
+    int relativeBlockletId = 0;
+    for (DataFileFooter fileFooter : indexInfo) {
+      if (segmentProperties == null) {
+        List<ColumnSchema> columnInTable = fileFooter.getColumnInTable();
+        schemaBinary = convertSchemaToBinary(columnInTable);
+        columnCardinality = fileFooter.getSegmentInfo().getColumnCardinality();
+        segmentProperties = new SegmentProperties(columnInTable, columnCardinality);
+        createSchema(segmentProperties);
+        createSummarySchema(segmentProperties, blockletDataMapInfo.getPartitions(), schemaBinary);
+      }
+      TableBlockInfo blockInfo = fileFooter.getBlockInfo().getTableBlockInfo();
+      BlockMetaInfo blockMetaInfo =
+          blockletDataMapInfo.getBlockMetaInfoMap().get(blockInfo.getFilePath());
+      // Here it loads info about all blocklets of index
+      // Only add if the file exists physically. There are scenarios which index file exists inside
+      // merge index but related carbondata files are deleted. In that case we first check whether
+      // the file exists physically or not
+      if (blockMetaInfo != null) {
+        if (fileFooter.getBlockletList() == null) {
+          // This is old store scenario, here blocklet information is not available in index file so
+          // load only block info
+          summaryRow =
+              loadToUnsafeBlock(fileFooter, segmentProperties, blockInfo.getFilePath(), summaryRow,
+                  blockMetaInfo);
+        } else {
+          // blocklet ID will start from 0 again only when part file path is changed
+          if (null == tempFilePath || !tempFilePath.equals(blockInfo.getFilePath())) {
+            tempFilePath = blockInfo.getFilePath();
+            relativeBlockletId = 0;
+          }
+          summaryRow =
+              loadToUnsafe(fileFooter, segmentProperties, blockInfo.getFilePath(), summaryRow,
+                  blockMetaInfo, relativeBlockletId);
+          // this is done because relative blocklet id need to be incremented based on the
+          // total number of blocklets
+          relativeBlockletId += fileFooter.getBlockletList().size();
+        }
+      }
+    }
+    if (unsafeMemoryDMStore != null) {
+      unsafeMemoryDMStore.finishWriting();
+    }
+    if (null != unsafeMemorySummaryDMStore) {
+      addTaskSummaryRowToUnsafeMemoryStore(
+          summaryRow,
+          blockletDataMapInfo.getPartitions(),
+          schemaBinary);
+      unsafeMemorySummaryDMStore.finishWriting();
+    }
+    LOGGER.info(
+        "Time taken to load blocklet datamap from file : " + dataMapModel.getFilePath() + "is " + (
+            System.currentTimeMillis() - startTime));
+  }
+
+  private DataMapRowImpl loadToUnsafe(DataFileFooter fileFooter,
+      SegmentProperties segmentProperties, String filePath, DataMapRowImpl summaryRow,
+      BlockMetaInfo blockMetaInfo, int relativeBlockletId) {
+    int[] minMaxLen = segmentProperties.getColumnsValueSize();
+    List<BlockletInfo> blockletList = fileFooter.getBlockletList();
+    CarbonRowSchema[] schema = unsafeMemoryDMStore.getSchema();
+    // Add one row to maintain task level min max for segment pruning
+    if (!blockletList.isEmpty() && summaryRow == null) {
+      summaryRow = new DataMapRowImpl(unsafeMemorySummaryDMStore.getSchema());
+    }
+    for (int index = 0; index < blockletList.size(); index++) {
+      DataMapRow row = new DataMapRowImpl(schema);
+      int ordinal = 0;
+      int taskMinMaxOrdinal = 0;
+      BlockletInfo blockletInfo = blockletList.get(index);
+
+      // add start key as index key
+      row.setByteArray(blockletInfo.getBlockletIndex().getBtreeIndex().getStartKey(), ordinal++);
+
+      BlockletMinMaxIndex minMaxIndex = blockletInfo.getBlockletIndex().getMinMaxIndex();
+      byte[][] minValues = updateMinValues(minMaxIndex.getMinValues(), minMaxLen);
+      row.setRow(addMinMax(minMaxLen, schema[ordinal], minValues), ordinal);
+      // compute and set task level min values
+      addTaskMinMaxValues(summaryRow, minMaxLen,
+          unsafeMemorySummaryDMStore.getSchema()[taskMinMaxOrdinal], minValues,
+          TASK_MIN_VALUES_INDEX, true);
+      ordinal++;
+      taskMinMaxOrdinal++;
+      byte[][] maxValues = updateMaxValues(minMaxIndex.getMaxValues(), minMaxLen);
+      row.setRow(addMinMax(minMaxLen, schema[ordinal], maxValues), ordinal);
+      // compute and set task level max values
+      addTaskMinMaxValues(summaryRow, minMaxLen,
+          unsafeMemorySummaryDMStore.getSchema()[taskMinMaxOrdinal], maxValues,
+          TASK_MAX_VALUES_INDEX, false);
+      ordinal++;
+
+      row.setInt(blockletInfo.getNumberOfRows(), ordinal++);
+
+      // add file path
+      byte[] filePathBytes = filePath.getBytes(CarbonCommonConstants.DEFAULT_CHARSET_CLASS);
+      row.setByteArray(filePathBytes, ordinal++);
+
+      // add pages
+      row.setShort((short) blockletInfo.getNumberOfPages(), ordinal++);
+
+      // add version number
+      row.setShort(fileFooter.getVersionId().number(), ordinal++);
+
+      // add schema updated time
+      row.setLong(fileFooter.getSchemaUpdatedTimeStamp(), ordinal++);
+
+      // add blocklet info
+      byte[] serializedData;
+      try {
+        ByteArrayOutputStream stream = new ByteArrayOutputStream();
+        DataOutput dataOutput = new DataOutputStream(stream);
+        blockletInfo.write(dataOutput);
+        serializedData = stream.toByteArray();
+        row.setByteArray(serializedData, ordinal++);
+        // Add block footer offset, it is used if we need to read footer of block
+        row.setLong(fileFooter.getBlockInfo().getTableBlockInfo().getBlockOffset(), ordinal++);
+        setLocations(blockMetaInfo.getLocationInfo(), row, ordinal);
+        ordinal++;
+        // for relative blockelt id i.e blocklet id that belongs to a particular part file
+        row.setShort((short) relativeBlockletId++, ordinal++);
+        // Store block size
+        row.setLong(blockMetaInfo.getSize(), ordinal);
+        unsafeMemoryDMStore.addIndexRowToUnsafe(row);
+      } catch (Exception e) {
+        throw new RuntimeException(e);
+      }
+    }
+
+    return summaryRow;
+  }
+
+  private void setLocations(String[] locations, DataMapRow row, int ordinal)
+      throws UnsupportedEncodingException {
+    // Add location info
+    String locationStr = StringUtils.join(locations, ',');
+    row.setByteArray(locationStr.getBytes(CarbonCommonConstants.DEFAULT_CHARSET), ordinal);
+  }
+
+  /**
+   * Load information for the block.It is the case can happen only for old stores
+   * where blocklet information is not available in index file. So load only block information
+   * and read blocklet information in executor.
+   */
+  private DataMapRowImpl loadToUnsafeBlock(DataFileFooter fileFooter,
+      SegmentProperties segmentProperties, String filePath, DataMapRowImpl summaryRow,
+      BlockMetaInfo blockMetaInfo) {
+    int[] minMaxLen = segmentProperties.getColumnsValueSize();
+    BlockletIndex blockletIndex = fileFooter.getBlockletIndex();
+    CarbonRowSchema[] schema = unsafeMemoryDMStore.getSchema();
+    // Add one row to maintain task level min max for segment pruning
+    if (summaryRow == null) {
+      summaryRow = new DataMapRowImpl(unsafeMemorySummaryDMStore.getSchema());
+    }
+    DataMapRow row = new DataMapRowImpl(schema);
+    int ordinal = 0;
+    int taskMinMaxOrdinal = 0;
+    // add start key as index key
+    row.setByteArray(blockletIndex.getBtreeIndex().getStartKey(), ordinal++);
+
+    BlockletMinMaxIndex minMaxIndex = blockletIndex.getMinMaxIndex();
+    byte[][] minValues = updateMinValues(minMaxIndex.getMinValues(), minMaxLen);
+    row.setRow(addMinMax(minMaxLen, schema[ordinal], minValues), ordinal);
+    // compute and set task level min values
+    addTaskMinMaxValues(summaryRow, minMaxLen,
+        unsafeMemorySummaryDMStore.getSchema()[taskMinMaxOrdinal], minValues,
+        TASK_MIN_VALUES_INDEX, true);
+    ordinal++;
+    taskMinMaxOrdinal++;
+    byte[][] maxValues = updateMaxValues(minMaxIndex.getMaxValues(), minMaxLen);
+    row.setRow(addMinMax(minMaxLen, schema[ordinal], maxValues), ordinal);
+    // compute and set task level max values
+    addTaskMinMaxValues(summaryRow, minMaxLen,
+        unsafeMemorySummaryDMStore.getSchema()[taskMinMaxOrdinal], maxValues,
+        TASK_MAX_VALUES_INDEX, false);
+    ordinal++;
+
+    row.setInt((int)fileFooter.getNumberOfRows(), ordinal++);
+
+    // add file path
+    byte[] filePathBytes = filePath.getBytes(CarbonCommonConstants.DEFAULT_CHARSET_CLASS);
+    row.setByteArray(filePathBytes, ordinal++);
+
+    // add pages
+    row.setShort((short) 0, ordinal++);
+
+    // add version number
+    row.setShort(fileFooter.getVersionId().number(), ordinal++);
+
+    // add schema updated time
+    row.setLong(fileFooter.getSchemaUpdatedTimeStamp(), ordinal++);
+
+    // add blocklet info
+    row.setByteArray(new byte[0], ordinal++);
+
+    row.setLong(fileFooter.getBlockInfo().getTableBlockInfo().getBlockOffset(), ordinal++);
+    try {
+      setLocations(blockMetaInfo.getLocationInfo(), row, ordinal);
+      ordinal++;
+      // for relative blocklet id. Value is -1 because in case of old store blocklet info will
+      // not be present in the index file and in that case we will not knwo the total number of
+      // blocklets
+      row.setShort((short) -1, ordinal++);
+
+      // store block size
+      row.setLong(blockMetaInfo.getSize(), ordinal);
+      unsafeMemoryDMStore.addIndexRowToUnsafe(row);
+    } catch (Exception e) {
+      throw new RuntimeException(e);
+    }
+
+    return summaryRow;
+  }
+
+  private void addTaskSummaryRowToUnsafeMemoryStore(DataMapRow summaryRow,
+      List<String> partitions, byte[] schemaBinary) throws IOException {
+    // write the task summary info to unsafe memory store
+    if (null != summaryRow) {
+      // Add column schema , it is useful to generate segment properties in executor.
+      // So we no need to read footer again there.
+      if (schemaBinary != null) {
+        summaryRow.setByteArray(schemaBinary, SCHEMA);
+      }
+      if (partitions != null && partitions.size() > 0) {
+        CarbonRowSchema[] minSchemas =
+            ((CarbonRowSchema.StructCarbonRowSchema) unsafeMemorySummaryDMStore
+                .getSchema()[PARTITION_INFO]).getChildSchemas();
+        DataMapRow partitionRow = new DataMapRowImpl(minSchemas);
+        for (int i = 0; i < partitions.size(); i++) {
+          partitionRow
+              .setByteArray(partitions.get(i).getBytes(CarbonCommonConstants.DEFAULT_CHARSET_CLASS),
+                  i);
+        }
+        summaryRow.setRow(partitionRow, PARTITION_INFO);
+      }
+      try {
+        unsafeMemorySummaryDMStore.addIndexRowToUnsafe(summaryRow);
+      } catch (Exception e) {
+        throw new RuntimeException(e);
+      }
+    }
+  }
+
+  /**
+   * Fill the measures min values with minimum , this is needed for backward version compatability
+   * as older versions don't store min values for measures
+   */
+  private byte[][] updateMinValues(byte[][] minValues, int[] minMaxLen) {
+    byte[][] updatedValues = minValues;
+    if (minValues.length < minMaxLen.length) {
+      updatedValues = new byte[minMaxLen.length][];
+      System.arraycopy(minValues, 0, updatedValues, 0, minValues.length);
+      List<CarbonMeasure> measures = segmentProperties.getMeasures();
+      ByteBuffer buffer = ByteBuffer.allocate(8);
+      for (int i = 0; i < measures.size(); i++) {
+        buffer.rewind();
+        DataType dataType = measures.get(i).getDataType();
+        if (dataType == DataTypes.BYTE) {
+          buffer.putLong(Byte.MIN_VALUE);
+          updatedValues[minValues.length + i] = buffer.array().clone();
+        } else if (dataType == DataTypes.SHORT) {
+          buffer.putLong(Short.MIN_VALUE);
+          updatedValues[minValues.length + i] = buffer.array().clone();
+        } else if (dataType == DataTypes.INT) {
+          buffer.putLong(Integer.MIN_VALUE);
+          updatedValues[minValues.length + i] = buffer.array().clone();
+        } else if (dataType == DataTypes.LONG) {
+          buffer.putLong(Long.MIN_VALUE);
+          updatedValues[minValues.length + i] = buffer.array().clone();
+        } else if (DataTypes.isDecimal(dataType)) {
+          updatedValues[minValues.length + i] =
+              DataTypeUtil.bigDecimalToByte(BigDecimal.valueOf(Long.MIN_VALUE));
+        } else {
+          buffer.putDouble(Double.MIN_VALUE);
+          updatedValues[minValues.length + i] = buffer.array().clone();
+        }
+      }
+    }
+    return updatedValues;
+  }
+
+  /**
+   * Fill the measures max values with maximum , this is needed for backward version compatability
+   * as older versions don't store max values for measures
+   */
+  private byte[][] updateMaxValues(byte[][] maxValues, int[] minMaxLen) {
+    byte[][] updatedValues = maxValues;
+    if (maxValues.length < minMaxLen.length) {
+      updatedValues = new byte[minMaxLen.length][];
+      System.arraycopy(maxValues, 0, updatedValues, 0, maxValues.length);
+      List<CarbonMeasure> measures = segmentProperties.getMeasures();
+      ByteBuffer buffer = ByteBuffer.allocate(8);
+      for (int i = 0; i < measures.size(); i++) {
+        buffer.rewind();
+        DataType dataType = measures.get(i).getDataType();
+        if (dataType == DataTypes.BYTE) {
+          buffer.putLong(Byte.MAX_VALUE);
+          updatedValues[maxValues.length + i] = buffer.array().clone();
+        } else if (dataType == DataTypes.SHORT) {
+          buffer.putLong(Short.MAX_VALUE);
+          updatedValues[maxValues.length + i] = buffer.array().clone();
+        } else if (dataType == DataTypes.INT) {
+          buffer.putLong(Integer.MAX_VALUE);
+          updatedValues[maxValues.length + i] = buffer.array().clone();
+        } else if (dataType == DataTypes.LONG) {
+          buffer.putLong(Long.MAX_VALUE);
+          updatedValues[maxValues.length + i] = buffer.array().clone();
+        } else if (DataTypes.isDecimal(dataType)) {
+          updatedValues[maxValues.length + i] =
+              DataTypeUtil.bigDecimalToByte(BigDecimal.valueOf(Long.MAX_VALUE));
+        } else {
+          buffer.putDouble(Double.MAX_VALUE);
+          updatedValues[maxValues.length + i] = buffer.array().clone();
+        }
+      }
+    }
+    return updatedValues;
+  }
+
+  private DataMapRow addMinMax(int[] minMaxLen, CarbonRowSchema carbonRowSchema,
+      byte[][] minValues) {
+    CarbonRowSchema[] minSchemas =
+        ((CarbonRowSchema.StructCarbonRowSchema) carbonRowSchema).getChildSchemas();
+    DataMapRow minRow = new DataMapRowImpl(minSchemas);
+    int minOrdinal = 0;
+    // min value adding
+    for (int i = 0; i < minMaxLen.length; i++) {
+      minRow.setByteArray(minValues[i], minOrdinal++);
+    }
+    return minRow;
+  }
+
+  /**
+   * This method will compute min/max values at task level
+   *
+   * @param taskMinMaxRow
+   * @param minMaxLen
+   * @param carbonRowSchema
+   * @param minMaxValue
+   * @param ordinal
+   * @param isMinValueComparison
+   */
+  private void addTaskMinMaxValues(DataMapRow taskMinMaxRow, int[] minMaxLen,
+      CarbonRowSchema carbonRowSchema, byte[][] minMaxValue, int ordinal,
+      boolean isMinValueComparison) {
+    DataMapRow row = taskMinMaxRow.getRow(ordinal);
+    byte[][] updatedMinMaxValues = minMaxValue;
+    if (null == row) {
+      CarbonRowSchema[] minSchemas =
+          ((CarbonRowSchema.StructCarbonRowSchema) carbonRowSchema).getChildSchemas();
+      row = new DataMapRowImpl(minSchemas);
+    } else {
+      byte[][] existingMinMaxValues = getMinMaxValue(taskMinMaxRow, ordinal);
+      // Compare and update min max values
+      for (int i = 0; i < minMaxLen.length; i++) {
+        int compare =
+            ByteUtil.UnsafeComparer.INSTANCE.compareTo(existingMinMaxValues[i], minMaxValue[i]);
+        if (isMinValueComparison) {
+          if (compare < 0) {
+            updatedMinMaxValues[i] = existingMinMaxValues[i];
+          }
+        } else if (compare > 0) {
+          updatedMinMaxValues[i] = existingMinMaxValues[i];
+        }
+      }
+    }
+    int minMaxOrdinal = 0;
+    // min/max value adding
+    for (int i = 0; i < minMaxLen.length; i++) {
+      row.setByteArray(updatedMinMaxValues[i], minMaxOrdinal++);
+    }
+    taskMinMaxRow.setRow(row, ordinal);
+  }
+
+  private void createSchema(SegmentProperties segmentProperties) throws MemoryException {
+    List<CarbonRowSchema> indexSchemas = new ArrayList<>();
+
+    // Index key
+    indexSchemas.add(new CarbonRowSchema.VariableCarbonRowSchema(DataTypes.BYTE_ARRAY));
+    getMinMaxSchema(segmentProperties, indexSchemas);
+
+    // for number of rows.
+    indexSchemas.add(new CarbonRowSchema.FixedCarbonRowSchema(DataTypes.INT));
+
+    // for table block path
+    indexSchemas.add(new CarbonRowSchema.VariableCarbonRowSchema(DataTypes.BYTE_ARRAY));
+
+    // for number of pages.
+    indexSchemas.add(new CarbonRowSchema.FixedCarbonRowSchema(DataTypes.SHORT));
+
+    // for version number.
+    indexSchemas.add(new CarbonRowSchema.FixedCarbonRowSchema(DataTypes.SHORT));
+
+    // for schema updated time.
+    indexSchemas.add(new CarbonRowSchema.FixedCarbonRowSchema(DataTypes.LONG));
+
+    //for blocklet info
+    indexSchemas.add(new CarbonRowSchema.VariableCarbonRowSchema(DataTypes.BYTE_ARRAY));
+
+    // for block footer offset.
+    indexSchemas.add(new CarbonRowSchema.FixedCarbonRowSchema(DataTypes.LONG));
+
+    // for locations
+    indexSchemas.add(new CarbonRowSchema.VariableCarbonRowSchema(DataTypes.BYTE_ARRAY));
+
+    // for relative blocklet id i.e. blocklet id that belongs to a particular part file.
+    indexSchemas.add(new CarbonRowSchema.FixedCarbonRowSchema(DataTypes.SHORT));
+
+    // for storing block length.
+    indexSchemas.add(new CarbonRowSchema.FixedCarbonRowSchema(DataTypes.LONG));
+
+    unsafeMemoryDMStore =
+        new UnsafeMemoryDMStore(indexSchemas.toArray(new CarbonRowSchema[indexSchemas.size()]));
+  }
+
+  /**
+   * Creates the schema to store summary information or the information which can be stored only
+   * once per datamap. It stores datamap level max/min of each column and partition information of
+   * datamap
+   * @param segmentProperties
+   * @param partitions
+   * @throws MemoryException
+   */
+  private void createSummarySchema(SegmentProperties segmentProperties, List<String> partitions,
+      byte[] schemaBinary)
+      throws MemoryException {
+    List<CarbonRowSchema> taskMinMaxSchemas = new ArrayList<>();
+    getMinMaxSchema(segmentProperties, taskMinMaxSchemas);
+    // for storing column schema
+    taskMinMaxSchemas.add(
+        new CarbonRowSchema.FixedCarbonRowSchema(DataTypes.BYTE_ARRAY, schemaBinary.length));
+    if (partitions != null && partitions.size() > 0) {
+      CarbonRowSchema[] mapSchemas = new CarbonRowSchema[partitions.size()];
+      for (int i = 0; i < mapSchemas.length; i++) {
+        mapSchemas[i] = new CarbonRowSchema.VariableCarbonRowSchema(DataTypes.BYTE_ARRAY);
+      }
+      CarbonRowSchema mapSchema =
+          new CarbonRowSchema.StructCarbonRowSchema(DataTypes.createDefaultStructType(),
+              mapSchemas);
+      taskMinMaxSchemas.add(mapSchema);
+    }
+    unsafeMemorySummaryDMStore = new UnsafeMemoryDMStore(
+        taskMinMaxSchemas.toArray(new CarbonRowSchema[taskMinMaxSchemas.size()]));
+  }
+
+  private void getMinMaxSchema(SegmentProperties segmentProperties,
+      List<CarbonRowSchema> minMaxSchemas) {
+    // Index key
+    int[] minMaxLen = segmentProperties.getColumnsValueSize();
+    // do it 2 times, one for min and one for max.
+    for (int k = 0; k < 2; k++) {
+      CarbonRowSchema[] mapSchemas = new CarbonRowSchema[minMaxLen.length];
+      for (int i = 0; i < minMaxLen.length; i++) {
+        if (minMaxLen[i] <= 0) {
+          mapSchemas[i] = new CarbonRowSchema.VariableCarbonRowSchema(DataTypes.BYTE_ARRAY);
+        } else {
+          mapSchemas[i] =
+              new CarbonRowSchema.FixedCarbonRowSchema(DataTypes.BYTE_ARRAY, minMaxLen[i]);
+        }
+      }
+      CarbonRowSchema mapSchema =
+          new CarbonRowSchema.StructCarbonRowSchema(DataTypes.createDefaultStructType(),
+              mapSchemas);
+      minMaxSchemas.add(mapSchema);
+    }
+  }
+
+  @Override
+  public boolean isScanRequired(FilterResolverIntf filterExp) {
+    FilterExecuter filterExecuter =
+        FilterUtil.getFilterExecuterTree(filterExp, segmentProperties, null);
+    for (int i = 0; i < unsafeMemorySummaryDMStore.getRowCount(); i++) {
+      DataMapRow unsafeRow = unsafeMemorySummaryDMStore.getUnsafeRow(i);
+      boolean isScanRequired = FilterExpressionProcessor.isScanRequired(
+          filterExecuter, getMinMaxValue(unsafeRow, TASK_MAX_VALUES_INDEX),
+          getMinMaxValue(unsafeRow, TASK_MIN_VALUES_INDEX));
+      if (isScanRequired) {
+        return true;
+      }
+    }
+    return false;
+  }
+
+  private List<Blocklet> prune(FilterResolverIntf filterExp, SegmentProperties segmentProperties) {
+    if (unsafeMemoryDMStore.getRowCount() == 0) {
+      return new ArrayList<>();
+    }
+    // getting the start and end index key based on filter for hitting the
+    // selected block reference nodes based on filter resolver tree.
+    if (LOGGER.isDebugEnabled()) {
+      LOGGER.debug("preparing the start and end key for finding"
+          + "start and end block as per filter resolver");
+    }
+    List<Blocklet> blocklets = new ArrayList<>();
+    Comparator<DataMapRow> comparator =
+        new BlockletDMComparator(segmentProperties.getColumnsValueSize(),
+            segmentProperties.getNumberOfSortColumns(),
+            segmentProperties.getNumberOfNoDictSortColumns());
+    List<IndexKey> listOfStartEndKeys = new ArrayList<IndexKey>(2);
+    FilterUtil
+        .traverseResolverTreeAndGetStartAndEndKey(segmentProperties, filterExp, listOfStartEndKeys);
+    // reading the first value from list which has start key
+    IndexKey searchStartKey = listOfStartEndKeys.get(0);
+    // reading the last value from list which has end key
+    IndexKey searchEndKey = listOfStartEndKeys.get(1);
+    if (null == searchStartKey && null == searchEndKey) {
+      try {
+        // TODO need to handle for no dictionary dimensions
+        searchStartKey = FilterUtil.prepareDefaultStartIndexKey(segmentProperties);
+        // TODO need to handle for no dictionary dimensions
+        searchEndKey = FilterUtil.prepareDefaultEndIndexKey(segmentProperties);
+      } catch (KeyGenException e) {
+        return null;
+      }
+    }
+    if (LOGGER.isDebugEnabled()) {
+      LOGGER.debug(
+          "Successfully retrieved the start and end key" + "Dictionary Start Key: " + Arrays
+              .toString(searchStartKey.getDictionaryKeys()) + "No Dictionary Start Key " + Arrays
+              .toString(searchStartKey.getNoDictionaryKeys()) + "Dictionary End Key: " + Arrays
+              .toString(searchEndKey.getDictionaryKeys()) + "No Dictionary End Key " + Arrays
+              .toString(searchEndKey.getNoDictionaryKeys()));
+    }
+    if (filterExp == null) {
+      int rowCount = unsafeMemoryDMStore.getRowCount();
+      for (int i = 0; i < rowCount; i++) {
+        DataMapRow safeRow = unsafeMemoryDMStore.getUnsafeRow(i).convertToSafeRow();
+        blocklets.add(createBlocklet(safeRow, safeRow.getShort(BLOCKLET_ID_INDEX)));
+      }
+    } else {
+      int startIndex = findStartIndex(convertToRow(searchStartKey), comparator);
+      int endIndex = findEndIndex(convertToRow(searchEndKey), comparator);
+      FilterExecuter filterExecuter =
+          FilterUtil.getFilterExecuterTree(filterExp, segmentProperties, null);
+      while (startIndex <= endIndex) {
+        DataMapRow safeRow = unsafeMemoryDMStore.getUnsafeRow(startIndex).convertToSafeRow();
+        int blockletId = safeRow.getShort(BLOCKLET_ID_INDEX);
+        String filePath = new String(safeRow.getByteArray(FILE_PATH_INDEX),
+            CarbonCommonConstants.DEFAULT_CHARSET_CLASS);
+        boolean isValid =
+            addBlockBasedOnMinMaxValue(filterExecuter, getMinMaxValue(safeRow, MAX_VALUES_INDEX),
+                getMinMaxValue(safeRow, MIN_VALUES_INDEX), filePath, blockletId);
+        if (isValid) {
+          blocklets.add(createBlocklet(safeRow, blockletId));
+        }
+        startIndex++;
+      }
+    }
+    return blocklets;
+  }
+
+  @Override
+  public List<Blocklet> prune(FilterResolverIntf filterExp, SegmentProperties segmentProperties,
+      List<String> partitions) {
+    if (unsafeMemoryDMStore.getRowCount() == 0) {
+      return new ArrayList<>();
+    }
+    // First get the partitions which are stored inside datamap.
+    List<String> storedPartitions = getPartitions();
+    // if it has partitioned datamap but there is no partitioned information stored, it means
+    // partitions are dropped so return empty list.
+    if (isPartitionedSegment && (storedPartitions == null || storedPartitions.size() == 0)) {
+      return new ArrayList<>();
+    }
+    if (storedPartitions != null && storedPartitions.size() > 0) {
+      // Check the exact match of partition information inside the stored partitions.
+      boolean found = false;
+      if (partitions != null && partitions.size() > 0) {
+        found = partitions.containsAll(storedPartitions);
+      }
+      if (!found) {
+        return new ArrayList<>();
+      }
+    }
+    // Prune with filters if the partitions are existed in this datamap
+    return prune(filterExp, segmentProperties);
+  }
+
+  /**
+   * select the blocks based on column min and max value
+   *
+   * @param filterExecuter
+   * @param maxValue
+   * @param minValue
+   * @param filePath
+   * @param blockletId
+   * @return
+   */
+  private boolean addBlockBasedOnMinMaxValue(FilterExecuter filterExecuter, byte[][] maxValue,
+      byte[][] minValue, String filePath, int blockletId) {
+    BitSet bitSet = null;
+    if (filterExecuter instanceof ImplicitColumnFilterExecutor) {
+      String uniqueBlockPath = filePath.substring(filePath.lastIndexOf("/Part") + 1);
+      // this case will come in case of old store where index file does not contain the
+      // blocklet information
+      if (blockletId != -1) {
+        uniqueBlockPath = uniqueBlockPath + CarbonCommonConstants.FILE_SEPARATOR + blockletId;
+      }
+      bitSet = ((ImplicitColumnFilterExecutor) filterExecuter)
+          .isFilterValuesPresentInBlockOrBlocklet(maxValue, minValue, uniqueBlockPath);
+    } else {
+      bitSet = filterExecuter.isScanRequired(maxValue, minValue);
+    }
+    if (!bitSet.isEmpty()) {
+      return true;
+    } else {
+      return false;
+    }
+  }
+
+  public ExtendedBlocklet getDetailedBlocklet(String blockletId) {
+    int index = Integer.parseInt(blockletId);
+    DataMapRow safeRow = unsafeMemoryDMStore.getUnsafeRow(index).convertToSafeRow();
+    return createBlocklet(safeRow, safeRow.getShort(BLOCKLET_ID_INDEX));
+  }
+
+  private byte[][] getMinMaxValue(DataMapRow row, int index) {
+    DataMapRow minMaxRow = row.getRow(index);
+    byte[][] minMax = new byte[minMaxRow.getColumnCount()][];
+    for (int i = 0; i < minMax.length; i++) {
+      minMax[i] = minMaxRow.getByteArray(i);
+    }
+    return minMax;
+  }
+
+  private ExtendedBlocklet createBlocklet(DataMapRow row, int blockletId) {
+    ExtendedBlocklet blocklet = new ExtendedBlocklet(
+        new String(row.getByteArray(FILE_PATH_INDEX), CarbonCommonConstants.DEFAULT_CHARSET_CLASS),
+        blockletId + "");
+    BlockletDetailInfo detailInfo = new BlockletDetailInfo();
+    detailInfo.setRowCount(row.getInt(ROW_COUNT_INDEX));
+    detailInfo.setPagesCount(row.getShort(PAGE_COUNT_INDEX));
+    detailInfo.setVersionNumber(row.getShort(VERSION_INDEX));
+    detailInfo.setBlockletId((short) blockletId);
+    detailInfo.setDimLens(columnCardinality);
+    detailInfo.setSchemaUpdatedTimeStamp(row.getLong(SCHEMA_UPADATED_TIME_INDEX));
+    byte[] byteArray = row.getByteArray(BLOCK_INFO_INDEX);
+    BlockletInfo blockletInfo = null;
+    try {
+      if (byteArray.length > 0) {
+        blockletInfo = new BlockletInfo();
+        ByteArrayInputStream stream = new ByteArrayInputStream(byteArray);
+        DataInputStream inputStream = new DataInputStream(stream);
+        blockletInfo.readFields(inputStream);
+        inputStream.close();
+      }
+      blocklet.setLocation(
+          new String(row.getByteArray(LOCATIONS), CarbonCommonConstants.DEFAULT_CHARSET)
+              .split(","));
+    } catch (IOException e) {
+      throw new RuntimeException(e);
+    }
+    detailInfo.setBlockletInfo(blockletInfo);
+    blocklet.setDetailInfo(detailInfo);
+    detailInfo.setBlockFooterOffset(row.getLong(BLOCK_FOOTER_OFFSET));
+    detailInfo.setColumnSchemaBinary(getColumnSchemaBinary());
+    detailInfo.setBlockSize(row.getLong(BLOCK_LENGTH));
+    return blocklet;
+  }
+
+  /**
+   * Binary search used to get the first tentative index row based on
+   * search key
+   *
+   * @param key search key
+   * @return first tentative block
+   */
+  private int findStartIndex(DataMapRow key, Comparator<DataMapRow> comparator) {
+    int childNodeIndex;
+    int low = 0;
+    int high = unsafeMemoryDMStore.getRowCount() - 1;
+    int mid = 0;
+    int compareRes = -1;
+    //
+    while (low <= high) {
+      mid = (low + high) >>> 1;
+      // compare the entries
+      compareRes = comparator.compare(key, unsafeMemoryDMStore.getUnsafeRow(mid));
+      if (compareRes < 0) {
+        high = mid - 1;
+      } else if (compareRes > 0) {
+        low = mid + 1;
+      } else {
+        // if key is matched then get the first entry
+        int currentPos = mid;
+        while (currentPos - 1 >= 0
+            && comparator.compare(key, unsafeMemoryDMStore.getUnsafeRow(currentPos - 1)) == 0) {
+          currentPos--;
+        }
+        mid = currentPos;
+        break;
+      }
+    }
+    // if compare result is less than zero then we
+    // and mid is more than 0 then we need to previous block as duplicates
+    // record can be present
+    if (compareRes < 0) {
+      if (mid > 0) {
+        mid--;
+      }
+      childNodeIndex = mid;
+    } else {
+      childNodeIndex = mid;
+    }
+    // get the leaf child
+    return childNodeIndex;
+  }
+
+  /**
+   * Binary search used to get the last tentative block  based on
+   * search key
+   *
+   * @param key search key
+   * @return first tentative block
+   */
+  private int findEndIndex(DataMapRow key, Comparator<DataMapRow> comparator) {
+    int childNodeIndex;
+    int low = 0;
+    int high = unsafeMemoryDMStore.getRowCount() - 1;
+    int mid = 0;
+    int compareRes = -1;
+    //
+    while (low <= high) {
+      mid = (low + high) >>> 1;
+      // compare the entries
+      compareRes = comparator.compare(key, unsafeMemoryDMStore.getUnsafeRow(mid));
+      if (compareRes < 0) {
+        high = mid - 1;
+      } else if (compareRes > 0) {
+        low = mid + 1;
+      } else {
+        int currentPos = mid;
+        // if key is matched then get the first entry
+        while (currentPos + 1 < unsafeMemoryDMStore.getRowCount()
+            && comparator.compare(key, unsafeMemoryDMStore.getUnsafeRow(currentPos + 1)) == 0) {
+          currentPos++;
+        }
+        mid = currentPos;
+        break;
+      }
+    }
+    // if compare result is less than zero then we
+    // and mid is more than 0 then we need to previous block as duplicates
+    // record can be present
+    if (compareRes < 0) {
+      if (mid > 0) {
+        mid--;
+      }
+      childNodeIndex = mid;
+    } else {
+      childNodeIndex = mid;
+    }
+    return childNodeIndex;
+  }
+
+  private DataMapRow convertToRow(IndexKey key) {
+    ByteBuffer buffer =
+        ByteBuffer.allocate(key.getDictionaryKeys().length + key.getNoDictionaryKeys().length + 8);
+    buffer.putInt(key.getDictionaryKeys().length);
+    buffer.putInt(key.getNoDictionaryKeys().length);
+    buffer.put(key.getDictionaryKeys());
+    buffer.put(key.getNoDictionaryKeys());
+    DataMapRowImpl dataMapRow = new DataMapRowImpl(unsafeMemoryDMStore.getSchema());
+    dataMapRow.setByteArray(buffer.array(), 0);
+    return dataMapRow;
+  }
+
+  private List<String> getPartitions() {
+    DataMapRow unsafeRow = unsafeMemorySummaryDMStore.getUnsafeRow(0);
+    if (unsafeRow.getColumnCount() > PARTITION_INFO) {
+      List<String> partitions = new ArrayList<>();
+      DataMapRow row = unsafeRow.getRow(PARTITION_INFO);
+      for (int i = 0; i < row.getColumnCount(); i++) {
+        partitions.add(
+            new String(row.getByteArray(i), CarbonCommonConstants.DEFAULT_CHARSET_CLASS));
+      }
+      return partitions;
+    }
+    return null;
+  }
+
+  private byte[] getColumnSchemaBinary() {
+    DataMapRow unsafeRow = unsafeMemorySummaryDMStore.getUnsafeRow(0);
+    return unsafeRow.getByteArray(SCHEMA);
+  }
+
+  /**
+   * Convert schema to binary
+   */
+  private byte[] convertSchemaToBinary(List<ColumnSchema> columnSchemas) throws IOException {
+    ByteArrayOutputStream stream = new ByteArrayOutputStream();
+    DataOutput dataOutput = new DataOutputStream(stream);
+    dataOutput.writeShort(columnSchemas.size());
+    for (ColumnSchema columnSchema : columnSchemas) {
+      if (columnSchema.getColumnReferenceId() == null) {
+        columnSchema.setColumnReferenceId(columnSchema.getColumnUniqueId());
+      }
+      columnSchema.write(dataOutput);
+    }
+    byte[] byteArray = stream.toByteArray();
+    // Compress with snappy to reduce the size of schema
+    return Snappy.rawCompress(byteArray, byteArray.length);
+  }
+
+  @Override
+  public void clear() {
+    if (unsafeMemoryDMStore != null) {
+      unsafeMemoryDMStore.freeMemory();
+      unsafeMemoryDMStore = null;
+      segmentProperties = null;
+    }
+    // clear task min/max unsafe memory
+    if (null != unsafeMemorySummaryDMStore) {
+      unsafeMemorySummaryDMStore.freeMemory();
+      unsafeMemorySummaryDMStore = null;
+    }
+  }
+
+  @Override
+  public long getFileTimeStamp() {
+    return 0;
+  }
+
+  @Override
+  public int getAccessCount() {
+    return 0;
+  }
+
+  @Override
+  public long getMemorySize() {
+    long memoryUsed = 0L;
+    if (unsafeMemoryDMStore != null) {
+      memoryUsed += unsafeMemoryDMStore.getMemoryUsed();
+    }
+    if (null != unsafeMemorySummaryDMStore) {
+      memoryUsed += unsafeMemorySummaryDMStore.getMemoryUsed();
+    }
+    return memoryUsed;
+  }
+
+  public SegmentProperties getSegmentProperties() {
+    return segmentProperties;
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/carbondata/blob/5a625f4c/core/src/main/java/org/apache/carbondata/core/indexstore/blockletindex/BlockletIndexDataMapFactory.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/carbondata/core/indexstore/blockletindex/BlockletIndexDataMapFactory.java b/core/src/main/java/org/apache/carbondata/core/indexstore/blockletindex/BlockletIndexDataMapFactory.java
new file mode 100644
index 0000000..a2c65ba
--- /dev/null
+++ b/core/src/main/java/org/apache/carbondata/core/indexstore/blockletindex/BlockletIndexDataMapFactory.java
@@ -0,0 +1,256 @@
+/*
+ * 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.indexstore.blockletindex;
+
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+
+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.DataMapMeta;
+import org.apache.carbondata.core.datamap.dev.AbstractDataMapWriter;
+import org.apache.carbondata.core.datamap.dev.IndexDataMap;
+import org.apache.carbondata.core.datamap.dev.cgdatamap.AbstractCoarseGrainIndexDataMap;
+import org.apache.carbondata.core.datamap.dev.cgdatamap.AbstractCoarseGrainIndexDataMapFactory;
+import org.apache.carbondata.core.datastore.block.SegmentProperties;
+import org.apache.carbondata.core.datastore.filesystem.CarbonFile;
+import org.apache.carbondata.core.datastore.impl.FileFactory;
+import org.apache.carbondata.core.indexstore.Blocklet;
+import org.apache.carbondata.core.indexstore.BlockletDetailsFetcher;
+import org.apache.carbondata.core.indexstore.ExtendedBlocklet;
+import org.apache.carbondata.core.indexstore.SegmentPropertiesFetcher;
+import org.apache.carbondata.core.indexstore.TableBlockIndexUniqueIdentifier;
+import org.apache.carbondata.core.metadata.AbsoluteTableIdentifier;
+import org.apache.carbondata.core.metadata.schema.table.DataMapSchema;
+import org.apache.carbondata.core.util.path.CarbonTablePath;
+import org.apache.carbondata.events.Event;
+
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.LocatedFileStatus;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.fs.RemoteIterator;
+
+/**
+ * Table map for blocklet
+ */
+public class BlockletIndexDataMapFactory extends AbstractCoarseGrainIndexDataMapFactory
+    implements BlockletDetailsFetcher,
+    SegmentPropertiesFetcher {
+
+  private static final String NAME = "clustered.btree.blocklet";
+
+  public static final DataMapSchema DATA_MAP_SCHEMA =
+      new DataMapSchema(NAME, BlockletIndexDataMapFactory.class.getName());
+
+  private AbsoluteTableIdentifier identifier;
+
+  // segmentId -> list of index file
+  private Map<String, List<TableBlockIndexUniqueIdentifier>> segmentMap = new HashMap<>();
+
+  private Cache<TableBlockIndexUniqueIdentifier, AbstractCoarseGrainIndexDataMap> cache;
+
+  @Override
+  public void init(AbsoluteTableIdentifier identifier, DataMapSchema dataMapSchema) {
+    this.identifier = identifier;
+    cache = CacheProvider.getInstance()
+        .createCache(CacheType.DRIVER_BLOCKLET_DATAMAP);
+  }
+
+  @Override
+  public AbstractDataMapWriter createWriter(String segmentId, String dataWriterPath) {
+    throw new UnsupportedOperationException("not implemented");
+  }
+
+  @Override
+  public List<AbstractCoarseGrainIndexDataMap> getDataMaps(String segmentId) throws IOException {
+    List<TableBlockIndexUniqueIdentifier> tableBlockIndexUniqueIdentifiers =
+        getTableBlockIndexUniqueIdentifiers(segmentId);
+    return cache.getAll(tableBlockIndexUniqueIdentifiers);
+  }
+
+  private List<TableBlockIndexUniqueIdentifier> getTableBlockIndexUniqueIdentifiers(
+      String segmentId) throws IOException {
+    List<TableBlockIndexUniqueIdentifier> tableBlockIndexUniqueIdentifiers =
+        segmentMap.get(segmentId);
+    if (tableBlockIndexUniqueIdentifiers == null) {
+      tableBlockIndexUniqueIdentifiers = new ArrayList<>();
+      String path = CarbonTablePath.getSegmentPath(identifier.getTablePath(), segmentId);
+      List<String> indexFiles = new SegmentIndexFileStore().getIndexFilesFromSegment(path);
+      for (int i = 0; i < indexFiles.size(); i++) {
+        tableBlockIndexUniqueIdentifiers.add(
+            new TableBlockIndexUniqueIdentifier(identifier, segmentId, indexFiles.get(i)));
+      }
+      segmentMap.put(segmentId, tableBlockIndexUniqueIdentifiers);
+    }
+    return tableBlockIndexUniqueIdentifiers;
+  }
+
+  /**
+   * Get the blocklet detail information based on blockletid, blockid and segmentid. This method is
+   * exclusively for BlockletIndexDataMapFactory as detail information is only available in this
+   * default datamap.
+   */
+  @Override
+  public List<ExtendedBlocklet> getExtendedBlocklets(List<Blocklet> blocklets, String segmentId)
+      throws IOException {
+    List<ExtendedBlocklet> detailedBlocklets = new ArrayList<>();
+    // If it is already detailed blocklet then type cast and return same
+    if (blocklets.size() > 0 && blocklets.get(0) instanceof ExtendedBlocklet) {
+      for (Blocklet blocklet : blocklets) {
+        detailedBlocklets.add((ExtendedBlocklet) blocklet);
+      }
+      return detailedBlocklets;
+    }
+    List<TableBlockIndexUniqueIdentifier> identifiers =
+        getTableBlockIndexUniqueIdentifiers(segmentId);
+    // Retrieve each blocklets detail information from blocklet datamap
+    for (Blocklet blocklet : blocklets) {
+      detailedBlocklets.add(getExtendedBlocklet(identifiers, blocklet));
+    }
+    return detailedBlocklets;
+  }
+
+  @Override
+  public ExtendedBlocklet getExtendedBlocklet(Blocklet blocklet, String segmentId)
+      throws IOException {
+    if (blocklet instanceof ExtendedBlocklet) {
+      return (ExtendedBlocklet) blocklet;
+    }
+    List<TableBlockIndexUniqueIdentifier> identifiers =
+        getTableBlockIndexUniqueIdentifiers(segmentId);
+    return getExtendedBlocklet(identifiers, blocklet);
+  }
+
+  private ExtendedBlocklet getExtendedBlocklet(List<TableBlockIndexUniqueIdentifier> identifiers,
+      Blocklet blocklet) throws IOException {
+    String carbonIndexFileName = CarbonTablePath.getCarbonIndexFileName(blocklet.getBlockId());
+    for (TableBlockIndexUniqueIdentifier identifier : identifiers) {
+      if (identifier.getCarbonIndexFileName().equals(carbonIndexFileName)) {
+        IndexDataMap indexDataMap = cache.get(identifier);
+        return ((BlockletIndexDataMap) indexDataMap).getDetailedBlocklet(blocklet.getBlockletId());
+      }
+    }
+    throw new IOException("Blocklet with blockid " + blocklet.getBlockletId() + " not found ");
+  }
+
+
+
+  @Override
+  public List<DataMapDistributable> toDistributable(String segmentId) {
+    CarbonFile[] carbonIndexFiles = SegmentIndexFileStore.getCarbonIndexFiles(segmentId);
+    List<DataMapDistributable> distributables = new ArrayList<>();
+    for (int i = 0; i < carbonIndexFiles.length; i++) {
+      Path path = new Path(carbonIndexFiles[i].getPath());
+      try {
+        FileSystem fs = path.getFileSystem(FileFactory.getConfiguration());
+        RemoteIterator<LocatedFileStatus> iter = fs.listLocatedStatus(path);
+        LocatedFileStatus fileStatus = iter.next();
+        String[] location = fileStatus.getBlockLocations()[0].getHosts();
+        BlockletDataMapDistributable distributable =
+            new BlockletDataMapDistributable(path.getName());
+        distributable.setLocations(location);
+        distributables.add(distributable);
+      } catch (IOException e) {
+        throw new RuntimeException(e);
+      }
+    }
+    return distributables;
+  }
+
+  @Override public void fireEvent(Event event) {
+
+  }
+
+  @Override
+  public void clear(String segmentId) {
+    List<TableBlockIndexUniqueIdentifier> blockIndexes = segmentMap.remove(segmentId);
+    if (blockIndexes != null) {
+      for (TableBlockIndexUniqueIdentifier blockIndex : blockIndexes) {
+        IndexDataMap indexDataMap = cache.getIfPresent(blockIndex);
+        if (indexDataMap != null) {
+          cache.invalidate(blockIndex);
+          indexDataMap.clear();
+        }
+      }
+    }
+  }
+
+  @Override
+  public void clear() {
+    for (String segmentId : segmentMap.keySet().toArray(new String[segmentMap.size()])) {
+      clear(segmentId);
+    }
+  }
+
+  @Override
+  public List<AbstractCoarseGrainIndexDataMap> getDataMaps(DataMapDistributable distributable)
+      throws IOException {
+    BlockletDataMapDistributable mapDistributable = (BlockletDataMapDistributable) distributable;
+    List<TableBlockIndexUniqueIdentifier> identifiers = new ArrayList<>();
+    if (mapDistributable.getFilePath().endsWith(CarbonTablePath.INDEX_FILE_EXT)) {
+      identifiers.add(new TableBlockIndexUniqueIdentifier(identifier, distributable.getSegmentId(),
+          mapDistributable.getFilePath()));
+    } else if (mapDistributable.getFilePath().endsWith(CarbonTablePath.MERGE_INDEX_FILE_EXT)) {
+      SegmentIndexFileStore fileStore = new SegmentIndexFileStore();
+      List<String> indexFiles = fileStore.getIndexFilesFromMergeFile(
+          CarbonTablePath.getSegmentPath(identifier.getTablePath(), mapDistributable.getSegmentId())
+              + "/" + mapDistributable.getFilePath());
+      for (String indexFile : indexFiles) {
+        identifiers.add(
+            new TableBlockIndexUniqueIdentifier(identifier, distributable.getSegmentId(),
+                indexFile));
+      }
+    }
+    List<AbstractCoarseGrainIndexDataMap> dataMaps;
+    try {
+      dataMaps = cache.getAll(identifiers);
+    } catch (IOException e) {
+      throw new RuntimeException(e);
+    }
+    return dataMaps;
+  }
+
+  @Override
+  public DataMapMeta getMeta() {
+    // TODO: pass SORT_COLUMNS into this class
+    return null;
+  }
+
+  @Override public SegmentProperties getSegmentProperties(String segmentId) throws IOException {
+    List<AbstractCoarseGrainIndexDataMap> dataMaps = getDataMaps(segmentId);
+    assert (dataMaps.size() > 0);
+    AbstractCoarseGrainIndexDataMap coarseGrainDataMap = dataMaps.get(0);
+    assert (coarseGrainDataMap instanceof BlockletIndexDataMap);
+    BlockletIndexDataMap dataMap = (BlockletIndexDataMap) coarseGrainDataMap;
+    return dataMap.getSegmentProperties();
+  }
+
+  @Override public List<Blocklet> getAllBlocklets(String segmentId, List<String> partitions)
+      throws IOException {
+    List<Blocklet> blocklets = new ArrayList<>();
+    List<AbstractCoarseGrainIndexDataMap> dataMaps = getDataMaps(segmentId);
+    for (AbstractCoarseGrainIndexDataMap dataMap : dataMaps) {
+      blocklets.addAll(dataMap.prune(null, getSegmentProperties(segmentId), partitions));
+    }
+    return blocklets;
+  }
+}

http://git-wip-us.apache.org/repos/asf/carbondata/blob/5a625f4c/core/src/main/java/org/apache/carbondata/core/metadata/schema/table/CarbonTable.java
----------------------------------------------------------------------
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 5b919d0..2393c54 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
@@ -768,7 +768,15 @@ public class CarbonTable implements Serializable {
   }
 
   /**
-   * whether this table has aggregation DataMap or not
+   * Return true if 'autoRefreshDataMap' is enabled, by default it is enabled
+   */
+  public boolean isAutoRefreshDataMap() {
+    String refresh = getTableInfo().getFactTable().getTableProperties().get("autoRefreshDataMap");
+    return refresh == null || refresh.equalsIgnoreCase("true");
+  }
+
+  /**
+   * whether this table has aggregation IndexDataMap or not
    */
   public boolean hasAggregationDataMap() {
     List<DataMapSchema> dataMapSchemaList = tableInfo.getDataMapSchemaList();

http://git-wip-us.apache.org/repos/asf/carbondata/blob/5a625f4c/core/src/main/java/org/apache/carbondata/core/metadata/schema/table/DataMapSchema.java
----------------------------------------------------------------------
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 5a9017b..ae49467 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
@@ -32,6 +32,7 @@ public class DataMapSchema implements Serializable, Writable {
 
   protected String dataMapName;
 
+  // this name can be class name of the DataMapProvider implementation or short name of it
   private String className;
 
   protected RelationIdentifier relationIdentifier;

http://git-wip-us.apache.org/repos/asf/carbondata/blob/5a625f4c/core/src/main/java/org/apache/carbondata/core/metadata/schema/table/DataMapSchemaFactory.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/carbondata/core/metadata/schema/table/DataMapSchemaFactory.java b/core/src/main/java/org/apache/carbondata/core/metadata/schema/table/DataMapSchemaFactory.java
index 5729959..1c6ebad 100644
--- a/core/src/main/java/org/apache/carbondata/core/metadata/schema/table/DataMapSchemaFactory.java
+++ b/core/src/main/java/org/apache/carbondata/core/metadata/schema/table/DataMapSchemaFactory.java
@@ -16,7 +16,7 @@
  */
 package org.apache.carbondata.core.metadata.schema.table;
 
-import static org.apache.carbondata.core.constants.CarbonCommonConstants.AGGREGATIONDATAMAPSCHEMA;
+import org.apache.carbondata.core.metadata.schema.datamap.DataMapProvider;
 
 public class DataMapSchemaFactory {
   public static final DataMapSchemaFactory INSTANCE = new DataMapSchemaFactory();
@@ -24,15 +24,16 @@ public class DataMapSchemaFactory {
   /**
    * Below class will be used to get data map schema object
    * based on class name
-   * @param className
+   * @param providerName
    * @return data map schema
    */
-  public DataMapSchema getDataMapSchema(String dataMapName, String className) {
-    switch (className) {
-      case AGGREGATIONDATAMAPSCHEMA:
-        return new AggregationDataMapSchema(dataMapName, className);
-      default:
-        return new DataMapSchema(dataMapName, className);
+  public DataMapSchema getDataMapSchema(String dataMapName, String providerName) {
+    if (providerName.equalsIgnoreCase(DataMapProvider.PREAGGREGATE.toString())) {
+      return new AggregationDataMapSchema(dataMapName, providerName);
+    } else if (providerName.equalsIgnoreCase(DataMapProvider.TIMESERIES.toString())) {
+      return new AggregationDataMapSchema(dataMapName, providerName);
+    } else {
+      return new DataMapSchema(dataMapName, providerName);
     }
   }
 }

http://git-wip-us.apache.org/repos/asf/carbondata/blob/5a625f4c/core/src/main/java/org/apache/carbondata/core/metadata/schema/table/TableSchema.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/carbondata/core/metadata/schema/table/TableSchema.java b/core/src/main/java/org/apache/carbondata/core/metadata/schema/table/TableSchema.java
index fff1a74..5d79abc 100644
--- a/core/src/main/java/org/apache/carbondata/core/metadata/schema/table/TableSchema.java
+++ b/core/src/main/java/org/apache/carbondata/core/metadata/schema/table/TableSchema.java
@@ -285,8 +285,7 @@ public class TableSchema implements Serializable, Writable {
                 // only = is allowed as special character , so replace with &
                 CarbonCommonConstants.DEFAULT_CHARSET)).replace("=","&"));
     properties.put("QUERYTYPE", queryType);
-    DataMapSchema dataMapSchema =
-        new DataMapSchema(dataMapName, className);
+    DataMapSchema dataMapSchema = new DataMapSchema(dataMapName, className);
     dataMapSchema.setProperties(properties);
 
     dataMapSchema.setChildSchema(this);

http://git-wip-us.apache.org/repos/asf/carbondata/blob/5a625f4c/core/src/test/java/org/apache/carbondata/core/indexstore/blockletindex/TestBlockletDataMap.java
----------------------------------------------------------------------
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/TestBlockletDataMap.java
deleted file mode 100644
index 88ac3ed..0000000
--- a/core/src/test/java/org/apache/carbondata/core/indexstore/blockletindex/TestBlockletDataMap.java
+++ /dev/null
@@ -1,66 +0,0 @@
-package org.apache.carbondata.core.indexstore.blockletindex;
-
-import java.lang.reflect.Method;
-import java.util.ArrayList;
-import java.util.BitSet;
-import java.util.List;
-import java.util.UUID;
-
-import org.apache.carbondata.core.cache.dictionary.AbstractDictionaryCacheTest;
-import org.apache.carbondata.core.constants.CarbonCommonConstants;
-import org.apache.carbondata.core.metadata.CarbonTableIdentifier;
-import org.apache.carbondata.core.metadata.datatype.DataTypes;
-import org.apache.carbondata.core.metadata.encoder.Encoding;
-import org.apache.carbondata.core.metadata.schema.table.column.CarbonImplicitDimension;
-import org.apache.carbondata.core.metadata.schema.table.column.ColumnSchema;
-import org.apache.carbondata.core.scan.filter.executer.FilterExecuter;
-import org.apache.carbondata.core.scan.filter.executer.ImplicitIncludeFilterExecutorImpl;
-import org.apache.carbondata.core.scan.filter.resolver.resolverinfo.DimColumnResolvedFilterInfo;
-import org.apache.carbondata.core.util.ByteUtil;
-
-import mockit.Mock;
-import mockit.MockUp;
-import org.junit.Before;
-import org.junit.Test;
-
-public class TestBlockletDataMap extends AbstractDictionaryCacheTest {
-
-  ImplicitIncludeFilterExecutorImpl implicitIncludeFilterExecutor;
-  @Before public void setUp() throws Exception {
-    CarbonImplicitDimension carbonImplicitDimension =
-        new CarbonImplicitDimension(0, CarbonCommonConstants.CARBON_IMPLICIT_COLUMN_POSITIONID);
-    DimColumnResolvedFilterInfo dimColumnEvaluatorInfo = new DimColumnResolvedFilterInfo();
-    dimColumnEvaluatorInfo.setColumnIndex(0);
-    dimColumnEvaluatorInfo.setRowIndex(0);
-    dimColumnEvaluatorInfo.setDimension(carbonImplicitDimension);
-    dimColumnEvaluatorInfo.setDimensionExistsInCurrentSilce(false);
-    implicitIncludeFilterExecutor =
-        new ImplicitIncludeFilterExecutorImpl(dimColumnEvaluatorInfo);
-  }
-
-  @Test public void testaddBlockBasedOnMinMaxValue() throws Exception {
-
-    new MockUp<ImplicitIncludeFilterExecutorImpl>() {
-      @Mock BitSet isFilterValuesPresentInBlockOrBlocklet(byte[][] maxValue, byte[][] minValue,
-          String uniqueBlockPath) {
-        BitSet bitSet = new BitSet(1);
-        bitSet.set(8);
-        return bitSet;
-      }
-    };
-
-    BlockletDataMap blockletDataMap = new BlockletDataMap();
-    Method method = BlockletDataMap.class
-        .getDeclaredMethod("addBlockBasedOnMinMaxValue", FilterExecuter.class, byte[][].class,
-            byte[][].class, String.class, int.class);
-    method.setAccessible(true);
-
-    byte[][] minValue = { ByteUtil.toBytes("sfds") };
-    byte[][] maxValue = { ByteUtil.toBytes("resa") };
-    Object result = method
-        .invoke(blockletDataMap, implicitIncludeFilterExecutor, minValue, maxValue,
-            "/opt/store/default/carbon_table/Fact/Part0/Segment_0/part-0-0_batchno0-0-1514989110586.carbondata",
-            0);
-    assert ((boolean) result);
-  }
-}

http://git-wip-us.apache.org/repos/asf/carbondata/blob/5a625f4c/core/src/test/java/org/apache/carbondata/core/indexstore/blockletindex/TestBlockletIndexDataMap.java
----------------------------------------------------------------------
diff --git a/core/src/test/java/org/apache/carbondata/core/indexstore/blockletindex/TestBlockletIndexDataMap.java b/core/src/test/java/org/apache/carbondata/core/indexstore/blockletindex/TestBlockletIndexDataMap.java
new file mode 100644
index 0000000..16048db
--- /dev/null
+++ b/core/src/test/java/org/apache/carbondata/core/indexstore/blockletindex/TestBlockletIndexDataMap.java
@@ -0,0 +1,59 @@
+package org.apache.carbondata.core.indexstore.blockletindex;
+
+import java.lang.reflect.Method;
+import java.util.BitSet;
+
+import org.apache.carbondata.core.cache.dictionary.AbstractDictionaryCacheTest;
+import org.apache.carbondata.core.constants.CarbonCommonConstants;
+import org.apache.carbondata.core.metadata.schema.table.column.CarbonImplicitDimension;
+import org.apache.carbondata.core.scan.filter.executer.FilterExecuter;
+import org.apache.carbondata.core.scan.filter.executer.ImplicitIncludeFilterExecutorImpl;
+import org.apache.carbondata.core.scan.filter.resolver.resolverinfo.DimColumnResolvedFilterInfo;
+import org.apache.carbondata.core.util.ByteUtil;
+
+import mockit.Mock;
+import mockit.MockUp;
+import org.junit.Before;
+import org.junit.Test;
+
+public class TestBlockletIndexDataMap extends AbstractDictionaryCacheTest {
+
+  ImplicitIncludeFilterExecutorImpl implicitIncludeFilterExecutor;
+  @Before public void setUp() throws Exception {
+    CarbonImplicitDimension carbonImplicitDimension =
+        new CarbonImplicitDimension(0, CarbonCommonConstants.CARBON_IMPLICIT_COLUMN_POSITIONID);
+    DimColumnResolvedFilterInfo dimColumnEvaluatorInfo = new DimColumnResolvedFilterInfo();
+    dimColumnEvaluatorInfo.setColumnIndex(0);
+    dimColumnEvaluatorInfo.setRowIndex(0);
+    dimColumnEvaluatorInfo.setDimension(carbonImplicitDimension);
+    dimColumnEvaluatorInfo.setDimensionExistsInCurrentSilce(false);
+    implicitIncludeFilterExecutor =
+        new ImplicitIncludeFilterExecutorImpl(dimColumnEvaluatorInfo);
+  }
+
+  @Test public void testaddBlockBasedOnMinMaxValue() throws Exception {
+
+    new MockUp<ImplicitIncludeFilterExecutorImpl>() {
+      @Mock BitSet isFilterValuesPresentInBlockOrBlocklet(byte[][] maxValue, byte[][] minValue,
+          String uniqueBlockPath) {
+        BitSet bitSet = new BitSet(1);
+        bitSet.set(8);
+        return bitSet;
+      }
+    };
+
+    BlockletIndexDataMap blockletDataMap = new BlockletIndexDataMap();
+    Method method = BlockletIndexDataMap.class
+        .getDeclaredMethod("addBlockBasedOnMinMaxValue", FilterExecuter.class, byte[][].class,
+            byte[][].class, String.class, int.class);
+    method.setAccessible(true);
+
+    byte[][] minValue = { ByteUtil.toBytes("sfds") };
+    byte[][] maxValue = { ByteUtil.toBytes("resa") };
+    Object result = method
+        .invoke(blockletDataMap, implicitIncludeFilterExecutor, minValue, maxValue,
+            "/opt/store/default/carbon_table/Fact/Part0/Segment_0/part-0-0_batchno0-0-1514989110586.carbondata",
+            0);
+    assert ((boolean) result);
+  }
+}

http://git-wip-us.apache.org/repos/asf/carbondata/blob/5a625f4c/datamap/examples/src/minmaxdatamap/main/java/org/apache/carbondata/datamap/examples/MinMaxDataMap.java
----------------------------------------------------------------------
diff --git a/datamap/examples/src/minmaxdatamap/main/java/org/apache/carbondata/datamap/examples/MinMaxDataMap.java b/datamap/examples/src/minmaxdatamap/main/java/org/apache/carbondata/datamap/examples/MinMaxDataMap.java
deleted file mode 100644
index 8002e57..0000000
--- a/datamap/examples/src/minmaxdatamap/main/java/org/apache/carbondata/datamap/examples/MinMaxDataMap.java
+++ /dev/null
@@ -1,151 +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.datamap.examples;
-
-import java.io.BufferedReader;
-import java.io.DataInputStream;
-import java.io.IOException;
-import java.io.InputStreamReader;
-import java.util.ArrayList;
-import java.util.BitSet;
-import java.util.List;
-
-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.DataMapModel;
-import org.apache.carbondata.core.datamap.dev.cgdatamap.AbstractCoarseGrainDataMap;
-import org.apache.carbondata.core.datastore.block.SegmentProperties;
-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.AtomicFileOperations;
-import org.apache.carbondata.core.fileoperations.AtomicFileOperationsImpl;
-import org.apache.carbondata.core.indexstore.Blocklet;
-import org.apache.carbondata.core.indexstore.row.DataMapRow;
-import org.apache.carbondata.core.memory.MemoryException;
-import org.apache.carbondata.core.scan.filter.FilterExpressionProcessor;
-import org.apache.carbondata.core.scan.filter.FilterUtil;
-import org.apache.carbondata.core.scan.filter.executer.FilterExecuter;
-import org.apache.carbondata.core.scan.filter.resolver.FilterResolverIntf;
-import org.apache.carbondata.core.util.CarbonUtil;
-
-import com.google.gson.Gson;
-
-/**
- * Datamap implementation for min max blocklet.
- */
-public class MinMaxDataMap extends AbstractCoarseGrainDataMap {
-
-  private static final LogService LOGGER =
-      LogServiceFactory.getLogService(MinMaxDataMap.class.getName());
-
-  public static final String NAME = "clustered.minmax.btree.blocklet";
-
-  private String filePath;
-
-  private MinMaxIndexBlockDetails[] readMinMaxDataMap;
-
-  @Override
-  public void init(DataMapModel model) throws MemoryException, IOException {
-    this.filePath = model.getFilePath();
-    CarbonFile[] listFiles = getCarbonMinMaxIndexFiles(filePath, "0");
-    for (int i = 0; i < listFiles.length; i++) {
-      readMinMaxDataMap = readJson(listFiles[i].getPath());
-    }
-  }
-
-  private CarbonFile[] getCarbonMinMaxIndexFiles(String filePath, String segmentId) {
-    String path = filePath.substring(0, filePath.lastIndexOf("/") + 1);
-    CarbonFile carbonFile = FileFactory.getCarbonFile(path);
-    return carbonFile.listFiles(new CarbonFileFilter() {
-      @Override public boolean accept(CarbonFile file) {
-        return file.getName().endsWith(".minmaxindex");
-      }
-    });
-  }
-
-  private MinMaxIndexBlockDetails[] readJson(String filePath) {
-    Gson gsonObjectToRead = new Gson();
-    DataInputStream dataInputStream = null;
-    BufferedReader buffReader = null;
-    InputStreamReader inStream = null;
-    MinMaxIndexBlockDetails[] readMinMax = null;
-    AtomicFileOperations fileOperation =
-        new AtomicFileOperationsImpl(filePath, FileFactory.getFileType(filePath));
-
-    try {
-      if (!FileFactory.isFileExist(filePath, FileFactory.getFileType(filePath))) {
-        return null;
-      }
-      dataInputStream = fileOperation.openForRead();
-      inStream = new InputStreamReader(dataInputStream, "UTF-8");
-      buffReader = new BufferedReader(inStream);
-      readMinMax = gsonObjectToRead.fromJson(buffReader, MinMaxIndexBlockDetails[].class);
-    } catch (IOException e) {
-      return null;
-    } finally {
-      CarbonUtil.closeStreams(buffReader, inStream, dataInputStream);
-    }
-    return readMinMax;
-  }
-
-  /**
-   * Block Prunning logic for Min Max DataMap.
-   *
-   * @param filterExp
-   * @param segmentProperties
-   * @return
-   */
-  @Override
-  public List<Blocklet> prune(FilterResolverIntf filterExp,
-      SegmentProperties segmentProperties, List<String> partitions) {
-    List<Blocklet> blocklets = new ArrayList<>();
-
-    if (filterExp == null) {
-      for (int i = 0; i < readMinMaxDataMap.length; i++) {
-        blocklets.add(new Blocklet(filePath, String.valueOf(readMinMaxDataMap[i].getBlockletId())));
-      }
-    } else {
-      FilterExecuter filterExecuter =
-          FilterUtil.getFilterExecuterTree(filterExp, segmentProperties, null);
-      int startIndex = 0;
-      while (startIndex < readMinMaxDataMap.length) {
-        BitSet bitSet = filterExecuter.isScanRequired(readMinMaxDataMap[startIndex].getMaxValues(),
-            readMinMaxDataMap[startIndex].getMinValues());
-        if (!bitSet.isEmpty()) {
-          blocklets.add(new Blocklet(filePath,
-              String.valueOf(readMinMaxDataMap[startIndex].getBlockletId())));
-        }
-        startIndex++;
-      }
-    }
-    return blocklets;
-  }
-
-  @Override
-  public boolean isScanRequired(FilterResolverIntf filterExp) {
-    throw new UnsupportedOperationException();
-  }
-
-  @Override
-  public void clear() {
-    readMinMaxDataMap = null;
-  }
-
-}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/carbondata/blob/5a625f4c/datamap/examples/src/minmaxdatamap/main/java/org/apache/carbondata/datamap/examples/MinMaxDataMapFactory.java
----------------------------------------------------------------------
diff --git a/datamap/examples/src/minmaxdatamap/main/java/org/apache/carbondata/datamap/examples/MinMaxDataMapFactory.java b/datamap/examples/src/minmaxdatamap/main/java/org/apache/carbondata/datamap/examples/MinMaxDataMapFactory.java
deleted file mode 100644
index 925731a..0000000
--- a/datamap/examples/src/minmaxdatamap/main/java/org/apache/carbondata/datamap/examples/MinMaxDataMapFactory.java
+++ /dev/null
@@ -1,116 +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.datamap.examples;
-
-import java.io.File;
-import java.io.IOException;
-import java.util.ArrayList;
-import java.util.Arrays;
-import java.util.List;
-
-import org.apache.carbondata.core.datamap.DataMapDistributable;
-import org.apache.carbondata.core.datamap.DataMapMeta;
-import org.apache.carbondata.core.datamap.dev.AbstractDataMapWriter;
-import org.apache.carbondata.core.datamap.dev.DataMapModel;
-import org.apache.carbondata.core.datamap.dev.cgdatamap.AbstractCoarseGrainDataMap;
-import org.apache.carbondata.core.datamap.dev.cgdatamap.AbstractCoarseGrainDataMapFactory;
-import org.apache.carbondata.core.memory.MemoryException;
-import org.apache.carbondata.core.metadata.AbsoluteTableIdentifier;
-import org.apache.carbondata.core.metadata.schema.table.DataMapSchema;
-import org.apache.carbondata.core.scan.filter.intf.ExpressionType;
-import org.apache.carbondata.events.Event;
-
-/**
- * Min Max DataMap Factory
- */
-public class MinMaxDataMapFactory extends AbstractCoarseGrainDataMapFactory {
-
-  private AbsoluteTableIdentifier identifier;
-
-  @Override public void init(AbsoluteTableIdentifier identifier, DataMapSchema dataMapSchema) {
-    this.identifier = identifier;
-  }
-
-  /**
-   * createWriter will return the MinMaxDataWriter.
-   *
-   * @param segmentId
-   * @return
-   */
-  @Override public AbstractDataMapWriter createWriter(String segmentId, String dataWritePath) {
-    return new MinMaxDataWriter(identifier, segmentId, dataWritePath);
-  }
-
-  /**
-   * getDataMaps Factory method Initializes the Min Max Data Map and returns.
-   *
-   * @param segmentId
-   * @return
-   * @throws IOException
-   */
-  @Override public List<AbstractCoarseGrainDataMap> getDataMaps(String segmentId)
-      throws IOException {
-    List<AbstractCoarseGrainDataMap> dataMapList = new ArrayList<>();
-    // Form a dataMap of Type MinMaxDataMap.
-    MinMaxDataMap dataMap = new MinMaxDataMap();
-    try {
-      dataMap.init(new DataMapModel(
-          identifier.getTablePath() + "/Fact/Part0/Segment_" + segmentId + File.separator));
-    } catch (MemoryException ex) {
-
-    }
-    dataMapList.add(dataMap);
-    return dataMapList;
-  }
-
-  /**
-   * @param segmentId
-   * @return
-   */
-  @Override public List<DataMapDistributable> toDistributable(String segmentId) {
-    return null;
-  }
-
-  /**
-   * Clear the DataMap.
-   *
-   * @param segmentId
-   */
-  @Override public void clear(String segmentId) {
-  }
-
-  /**
-   * Clearing the data map.
-   */
-  @Override public void clear() {
-  }
-
-  @Override public List<AbstractCoarseGrainDataMap> getDataMaps(DataMapDistributable distributable)
-      throws IOException {
-    return null;
-  }
-
-  @Override public void fireEvent(Event event) {
-
-  }
-
-  @Override public DataMapMeta getMeta() {
-    return new DataMapMeta(new ArrayList<String>(Arrays.asList("c2")),
-        new ArrayList<ExpressionType>());
-  }
-}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/carbondata/blob/5a625f4c/datamap/examples/src/minmaxdatamap/main/java/org/apache/carbondata/datamap/examples/MinMaxIndexDataMap.java
----------------------------------------------------------------------
diff --git a/datamap/examples/src/minmaxdatamap/main/java/org/apache/carbondata/datamap/examples/MinMaxIndexDataMap.java b/datamap/examples/src/minmaxdatamap/main/java/org/apache/carbondata/datamap/examples/MinMaxIndexDataMap.java
new file mode 100644
index 0000000..216000b
--- /dev/null
+++ b/datamap/examples/src/minmaxdatamap/main/java/org/apache/carbondata/datamap/examples/MinMaxIndexDataMap.java
@@ -0,0 +1,148 @@
+/*
+ * 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.datamap.examples;
+
+import java.io.BufferedReader;
+import java.io.DataInputStream;
+import java.io.IOException;
+import java.io.InputStreamReader;
+import java.util.ArrayList;
+import java.util.BitSet;
+import java.util.List;
+
+import org.apache.carbondata.common.logging.LogService;
+import org.apache.carbondata.common.logging.LogServiceFactory;
+import org.apache.carbondata.core.datamap.dev.DataMapModel;
+import org.apache.carbondata.core.datamap.dev.cgdatamap.AbstractCoarseGrainIndexDataMap;
+import org.apache.carbondata.core.datastore.block.SegmentProperties;
+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.AtomicFileOperations;
+import org.apache.carbondata.core.fileoperations.AtomicFileOperationsImpl;
+import org.apache.carbondata.core.indexstore.Blocklet;
+import org.apache.carbondata.core.memory.MemoryException;
+import org.apache.carbondata.core.scan.filter.FilterUtil;
+import org.apache.carbondata.core.scan.filter.executer.FilterExecuter;
+import org.apache.carbondata.core.scan.filter.resolver.FilterResolverIntf;
+import org.apache.carbondata.core.util.CarbonUtil;
+
+import com.google.gson.Gson;
+
+/**
+ * Datamap implementation for min max blocklet.
+ */
+public class MinMaxIndexDataMap extends AbstractCoarseGrainIndexDataMap {
+
+  private static final LogService LOGGER =
+      LogServiceFactory.getLogService(MinMaxIndexDataMap.class.getName());
+
+  public static final String NAME = "clustered.minmax.btree.blocklet";
+
+  private String filePath;
+
+  private MinMaxIndexBlockDetails[] readMinMaxDataMap;
+
+  @Override
+  public void init(DataMapModel model) throws MemoryException, IOException {
+    this.filePath = model.getFilePath();
+    CarbonFile[] listFiles = getCarbonMinMaxIndexFiles(filePath, "0");
+    for (int i = 0; i < listFiles.length; i++) {
+      readMinMaxDataMap = readJson(listFiles[i].getPath());
+    }
+  }
+
+  private CarbonFile[] getCarbonMinMaxIndexFiles(String filePath, String segmentId) {
+    String path = filePath.substring(0, filePath.lastIndexOf("/") + 1);
+    CarbonFile carbonFile = FileFactory.getCarbonFile(path);
+    return carbonFile.listFiles(new CarbonFileFilter() {
+      @Override public boolean accept(CarbonFile file) {
+        return file.getName().endsWith(".minmaxindex");
+      }
+    });
+  }
+
+  private MinMaxIndexBlockDetails[] readJson(String filePath) {
+    Gson gsonObjectToRead = new Gson();
+    DataInputStream dataInputStream = null;
+    BufferedReader buffReader = null;
+    InputStreamReader inStream = null;
+    MinMaxIndexBlockDetails[] readMinMax = null;
+    AtomicFileOperations fileOperation =
+        new AtomicFileOperationsImpl(filePath, FileFactory.getFileType(filePath));
+
+    try {
+      if (!FileFactory.isFileExist(filePath, FileFactory.getFileType(filePath))) {
+        return null;
+      }
+      dataInputStream = fileOperation.openForRead();
+      inStream = new InputStreamReader(dataInputStream, "UTF-8");
+      buffReader = new BufferedReader(inStream);
+      readMinMax = gsonObjectToRead.fromJson(buffReader, MinMaxIndexBlockDetails[].class);
+    } catch (IOException e) {
+      return null;
+    } finally {
+      CarbonUtil.closeStreams(buffReader, inStream, dataInputStream);
+    }
+    return readMinMax;
+  }
+
+  /**
+   * Block Prunning logic for Min Max DataMap.
+   *
+   * @param filterExp
+   * @param segmentProperties
+   * @return
+   */
+  @Override
+  public List<Blocklet> prune(FilterResolverIntf filterExp,
+      SegmentProperties segmentProperties, List<String> partitions) {
+    List<Blocklet> blocklets = new ArrayList<>();
+
+    if (filterExp == null) {
+      for (int i = 0; i < readMinMaxDataMap.length; i++) {
+        blocklets.add(new Blocklet(filePath, String.valueOf(readMinMaxDataMap[i].getBlockletId())));
+      }
+    } else {
+      FilterExecuter filterExecuter =
+          FilterUtil.getFilterExecuterTree(filterExp, segmentProperties, null);
+      int startIndex = 0;
+      while (startIndex < readMinMaxDataMap.length) {
+        BitSet bitSet = filterExecuter.isScanRequired(readMinMaxDataMap[startIndex].getMaxValues(),
+            readMinMaxDataMap[startIndex].getMinValues());
+        if (!bitSet.isEmpty()) {
+          blocklets.add(new Blocklet(filePath,
+              String.valueOf(readMinMaxDataMap[startIndex].getBlockletId())));
+        }
+        startIndex++;
+      }
+    }
+    return blocklets;
+  }
+
+  @Override
+  public boolean isScanRequired(FilterResolverIntf filterExp) {
+    throw new UnsupportedOperationException();
+  }
+
+  @Override
+  public void clear() {
+    readMinMaxDataMap = null;
+  }
+
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/carbondata/blob/5a625f4c/datamap/examples/src/minmaxdatamap/main/java/org/apache/carbondata/datamap/examples/MinMaxIndexDataMapFactory.java
----------------------------------------------------------------------
diff --git a/datamap/examples/src/minmaxdatamap/main/java/org/apache/carbondata/datamap/examples/MinMaxIndexDataMapFactory.java b/datamap/examples/src/minmaxdatamap/main/java/org/apache/carbondata/datamap/examples/MinMaxIndexDataMapFactory.java
new file mode 100644
index 0000000..5f714a1
--- /dev/null
+++ b/datamap/examples/src/minmaxdatamap/main/java/org/apache/carbondata/datamap/examples/MinMaxIndexDataMapFactory.java
@@ -0,0 +1,116 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.carbondata.datamap.examples;
+
+import java.io.File;
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.List;
+
+import org.apache.carbondata.core.datamap.DataMapDistributable;
+import org.apache.carbondata.core.datamap.DataMapMeta;
+import org.apache.carbondata.core.datamap.dev.AbstractDataMapWriter;
+import org.apache.carbondata.core.datamap.dev.DataMapModel;
+import org.apache.carbondata.core.datamap.dev.cgdatamap.AbstractCoarseGrainIndexDataMap;
+import org.apache.carbondata.core.datamap.dev.cgdatamap.AbstractCoarseGrainIndexDataMapFactory;
+import org.apache.carbondata.core.memory.MemoryException;
+import org.apache.carbondata.core.metadata.AbsoluteTableIdentifier;
+import org.apache.carbondata.core.metadata.schema.table.DataMapSchema;
+import org.apache.carbondata.core.scan.filter.intf.ExpressionType;
+import org.apache.carbondata.events.Event;
+
+/**
+ * Min Max DataMap Factory
+ */
+public class MinMaxIndexDataMapFactory extends AbstractCoarseGrainIndexDataMapFactory {
+
+  private AbsoluteTableIdentifier identifier;
+
+  @Override public void init(AbsoluteTableIdentifier identifier, DataMapSchema dataMapSchema) {
+    this.identifier = identifier;
+  }
+
+  /**
+   * createWriter will return the MinMaxDataWriter.
+   *
+   * @param segmentId
+   * @return
+   */
+  @Override public AbstractDataMapWriter createWriter(String segmentId, String dataWritePath) {
+    return new MinMaxDataWriter(identifier, segmentId, dataWritePath);
+  }
+
+  /**
+   * getDataMaps Factory method Initializes the Min Max Data Map and returns.
+   *
+   * @param segmentId
+   * @return
+   * @throws IOException
+   */
+  @Override public List<AbstractCoarseGrainIndexDataMap> getDataMaps(String segmentId)
+      throws IOException {
+    List<AbstractCoarseGrainIndexDataMap> dataMapList = new ArrayList<>();
+    // Form a dataMap of Type MinMaxIndexDataMap.
+    MinMaxIndexDataMap dataMap = new MinMaxIndexDataMap();
+    try {
+      dataMap.init(new DataMapModel(
+          identifier.getTablePath() + "/Fact/Part0/Segment_" + segmentId + File.separator));
+    } catch (MemoryException ex) {
+
+    }
+    dataMapList.add(dataMap);
+    return dataMapList;
+  }
+
+  /**
+   * @param segmentId
+   * @return
+   */
+  @Override public List<DataMapDistributable> toDistributable(String segmentId) {
+    return null;
+  }
+
+  /**
+   * Clear the DataMap.
+   *
+   * @param segmentId
+   */
+  @Override public void clear(String segmentId) {
+  }
+
+  /**
+   * Clearing the data map.
+   */
+  @Override public void clear() {
+  }
+
+  @Override public List<AbstractCoarseGrainIndexDataMap> getDataMaps(DataMapDistributable distributable)
+      throws IOException {
+    return null;
+  }
+
+  @Override public void fireEvent(Event event) {
+
+  }
+
+  @Override public DataMapMeta getMeta() {
+    return new DataMapMeta(new ArrayList<String>(Arrays.asList("c2")),
+        new ArrayList<ExpressionType>());
+  }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/carbondata/blob/5a625f4c/datamap/examples/src/minmaxdatamap/test/scala/minmaxdatamaptestcase/MinMaxDataMapExample.scala
----------------------------------------------------------------------
diff --git a/datamap/examples/src/minmaxdatamap/test/scala/minmaxdatamaptestcase/MinMaxDataMapExample.scala b/datamap/examples/src/minmaxdatamap/test/scala/minmaxdatamaptestcase/MinMaxDataMapExample.scala
index 0cfe410..59872aa 100644
--- a/datamap/examples/src/minmaxdatamap/test/scala/minmaxdatamaptestcase/MinMaxDataMapExample.scala
+++ b/datamap/examples/src/minmaxdatamap/test/scala/minmaxdatamaptestcase/MinMaxDataMapExample.scala
@@ -52,8 +52,8 @@ object MinMaxDataMapExample {
     // register datamap writer
     DataMapStoreManager.getInstance().createAndRegisterDataMap(
       AbsoluteTableIdentifier.from(storeLocation, "default", "carbonminmax"),
-      classOf[MinMaxDataMapFactory].getName,
-      MinMaxDataMap.NAME)
+      classOf[MinMaxIndexDataMapFactory].getName,
+      MinMaxIndexDataMap.NAME)
 
     spark.sql("DROP TABLE IF EXISTS carbonminmax")
 

http://git-wip-us.apache.org/repos/asf/carbondata/blob/5a625f4c/docs/datamap-developer-guide.md
----------------------------------------------------------------------
diff --git a/docs/datamap-developer-guide.md b/docs/datamap-developer-guide.md
new file mode 100644
index 0000000..31afd34
--- /dev/null
+++ b/docs/datamap-developer-guide.md
@@ -0,0 +1,16 @@
+# DataMap Developer Guide
+
+### Introduction
+DataMap is a data structure that can be used to accelerate certain query of the table. Different DataMap can be implemented by developers. 
+Currently, there are two 2 types of DataMap supported:
+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)
+
+### DataMap provider
+When user issues `CREATE DATAMAP dm ON TABLE main USING 'provider'`, the corresponding DataMapProvider implementation will be created and initialized. 
+Currently, the provider string can be:
+1. preaggregate: one type of MVDataMap that do pre-aggregate of single table
+2. timeseries: one type of MVDataMap that do pre-aggregate based on time dimension of the table
+3. 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.
\ No newline at end of file


[2/6] carbondata git commit: [CARBONDATA-2189] Add DataMapProvider developer interface

Posted by ja...@apache.org.
http://git-wip-us.apache.org/repos/asf/carbondata/blob/5a625f4c/integration/spark-common-test/src/test/scala/org/apache/carbondata/spark/testsuite/datamap/TestDataMapCommand.scala
----------------------------------------------------------------------
diff --git a/integration/spark-common-test/src/test/scala/org/apache/carbondata/spark/testsuite/datamap/TestDataMapCommand.scala b/integration/spark-common-test/src/test/scala/org/apache/carbondata/spark/testsuite/datamap/TestDataMapCommand.scala
deleted file mode 100644
index b2ab977..0000000
--- a/integration/spark-common-test/src/test/scala/org/apache/carbondata/spark/testsuite/datamap/TestDataMapCommand.scala
+++ /dev/null
@@ -1,282 +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.spark.testsuite.datamap
-
-import java.io.{File, FilenameFilter}
-
-import org.apache.spark.sql.Row
-import org.apache.spark.sql.test.util.QueryTest
-import org.scalatest.BeforeAndAfterAll
-
-import org.apache.carbondata.common.exceptions.MetadataProcessException
-import org.apache.carbondata.common.exceptions.sql.{MalformedDataMapCommandException, NoSuchDataMapException}
-import org.apache.carbondata.core.constants.CarbonCommonConstants
-import org.apache.carbondata.core.metadata.CarbonMetadata
-import org.apache.carbondata.core.util.CarbonProperties
-import org.apache.carbondata.core.util.path.CarbonTablePath
-
-class TestDataMapCommand extends QueryTest with BeforeAndAfterAll {
-
-  val testData = s"$resourcesPath/sample.csv"
-
-  override def beforeAll {
-    sql("drop table if exists datamaptest")
-    sql("drop table if exists datamapshowtest")
-    sql("drop table if exists uniqdata")
-    sql("create table datamaptest (a string, b string, c string) stored by 'carbondata'")
-  }
-
-  val newClass = "org.apache.spark.sql.CarbonSource"
-
-  test("test datamap create: don't support using non-exist class") {
-    intercept[MetadataProcessException] {
-      sql(s"CREATE DATAMAP datamap1 ON TABLE datamaptest USING '$newClass'")
-    }
-  }
-
-  test("test datamap create with dmproperties: don't support using non-exist class") {
-    intercept[MetadataProcessException] {
-      sql(s"CREATE DATAMAP datamap2 ON TABLE datamaptest USING '$newClass' DMPROPERTIES('key'='value')")
-    }
-  }
-
-  test("test datamap create with existing name: don't support using non-exist class") {
-    intercept[MetadataProcessException] {
-      sql(
-        s"CREATE DATAMAP datamap2 ON TABLE datamaptest USING '$newClass' DMPROPERTIES('key'='value')")
-    }
-  }
-
-  test("test datamap create with preagg") {
-    sql("drop datamap if exists datamap3 on table datamaptest")
-    sql(
-      "create datamap datamap3 on table datamaptest using 'preaggregate' dmproperties('key'='value') as select count(a) from datamaptest")
-    val table = CarbonMetadata.getInstance().getCarbonTable("default", "datamaptest")
-    assert(table != null)
-    val dataMapSchemaList = table.getTableInfo.getDataMapSchemaList
-    assert(dataMapSchemaList.size() == 1)
-    assert(dataMapSchemaList.get(0).getDataMapName.equals("datamap3"))
-    assert(dataMapSchemaList.get(0).getProperties.get("key").equals("value"))
-    assert(dataMapSchemaList.get(0).getChildSchema.getTableName.equals("datamaptest_datamap3"))
-  }
-
-  test("check hivemetastore after drop datamap") {
-    try {
-      CarbonProperties.getInstance()
-        .addProperty(CarbonCommonConstants.ENABLE_HIVE_SCHEMA_META_STORE,
-          "true")
-      sql("drop table if exists hiveMetaStoreTable")
-      sql("create table hiveMetaStoreTable (a string, b string, c string) stored by 'carbondata'")
-
-      sql(
-        "create datamap datamap_hiveMetaStoreTable on table hiveMetaStoreTable using 'preaggregate' dmproperties('key'='value') as select count(a) from hiveMetaStoreTable")
-      checkExistence(sql("show datamap on table hiveMetaStoreTable"), true, "datamap_hiveMetaStoreTable")
-
-      sql("drop datamap datamap_hiveMetaStoreTable on table hiveMetaStoreTable")
-      checkExistence(sql("show datamap on table hiveMetaStoreTable"), false, "datamap_hiveMetaStoreTable")
-
-    } finally {
-      sql("drop table hiveMetaStoreTable")
-      CarbonProperties.getInstance()
-        .addProperty(CarbonCommonConstants.ENABLE_HIVE_SCHEMA_META_STORE,
-          CarbonCommonConstants.ENABLE_HIVE_SCHEMA_META_STORE_DEFAULT)
-    }
-  }
-
-  test("drop the table having pre-aggregate") {
-    try {
-      CarbonProperties.getInstance()
-        .addProperty(CarbonCommonConstants.ENABLE_HIVE_SCHEMA_META_STORE,
-          "true")
-      sql("drop table if exists hiveMetaStoreTable_1")
-      sql("create table hiveMetaStoreTable_1 (a string, b string, c string) stored by 'carbondata'")
-
-      sql(
-        "create datamap datamap_hiveMetaStoreTable_1 on table hiveMetaStoreTable_1 using 'preaggregate' dmproperties('key'='value') as select count(a) from hiveMetaStoreTable_1")
-
-      checkExistence(sql("show datamap on table hiveMetaStoreTable_1"),
-        true,
-        "datamap_hiveMetaStoreTable_1")
-
-      sql("drop table hiveMetaStoreTable_1")
-
-      checkExistence(sql("show tables"), false, "datamap_hiveMetaStoreTable_1")
-    } finally {
-      CarbonProperties.getInstance()
-        .addProperty(CarbonCommonConstants.ENABLE_HIVE_SCHEMA_META_STORE,
-          CarbonCommonConstants.ENABLE_HIVE_SCHEMA_META_STORE_DEFAULT)
-    }
-  }
-
-  test("test datamap create with preagg with duplicate name") {
-    sql(
-      s"""
-         | CREATE DATAMAP datamap10 ON TABLE datamaptest
-         | USING 'preaggregate'
-         | DMPROPERTIES('key'='value')
-         | AS SELECT COUNT(a) FROM datamaptest
-         """.stripMargin)
-    intercept[MalformedDataMapCommandException] {
-      sql(
-        s"""
-           | CREATE DATAMAP datamap10 ON TABLE datamaptest
-           | USING 'preaggregate'
-           | DMPROPERTIES('key'='value')
-           | AS SELECT COUNT(a) FROM datamaptest
-         """.stripMargin)
-    }
-    val table = CarbonMetadata.getInstance().getCarbonTable("default", "datamaptest")
-    assert(table != null)
-    val dataMapSchemaList = table.getTableInfo.getDataMapSchemaList
-    assert(dataMapSchemaList.size() == 2)
-  }
-
-  test("test drop non-exist datamap") {
-    intercept[NoSuchDataMapException] {
-      sql("drop datamap nonexist on table datamaptest")
-    }
-    val table = CarbonMetadata.getInstance().getCarbonTable("default", "datamaptest")
-    assert(table != null)
-    val dataMapSchemaList = table.getTableInfo.getDataMapSchemaList
-    assert(dataMapSchemaList.size() == 2)
-  }
-
-  test("test show datamap without preaggregate: don't support using non-exist class") {
-    intercept[MetadataProcessException] {
-      sql("drop table if exists datamapshowtest")
-      sql("create table datamapshowtest (a string, b string, c string) stored by 'carbondata'")
-      sql(s"CREATE DATAMAP datamap1 ON TABLE datamapshowtest USING '$newClass' DMPROPERTIES('key'='value')")
-      sql(s"CREATE DATAMAP datamap2 ON TABLE datamapshowtest USING '$newClass' DMPROPERTIES('key'='value')")
-      checkExistence(sql("SHOW DATAMAP ON TABLE datamapshowtest"), true, "datamap1", "datamap2", "(NA)", newClass)
-    }
-  }
-
-  test("test show datamap with preaggregate: don't support using non-exist class") {
-    intercept[MetadataProcessException] {
-      sql("drop table if exists datamapshowtest")
-      sql("create table datamapshowtest (a string, b string, c string) stored by 'carbondata'")
-      sql("create datamap datamap1 on table datamapshowtest using 'preaggregate' as select count(a) from datamapshowtest")
-      sql(s"CREATE DATAMAP datamap2 ON TABLE datamapshowtest USING '$newClass' DMPROPERTIES('key'='value')")
-      val frame = sql("show datamap on table datamapshowtest")
-      assert(frame.collect().length == 2)
-      checkExistence(frame, true, "datamap1", "datamap2", "(NA)", newClass, "default.datamapshowtest_datamap1")
-    }
-  }
-
-  test("test show datamap with no datamap") {
-    sql("drop table if exists datamapshowtest")
-    sql("create table datamapshowtest (a string, b string, c string) stored by 'carbondata'")
-    assert(sql("show datamap on table datamapshowtest").collect().length == 0)
-  }
-
-  test("test show datamap after dropping datamap: don't support using non-exist class") {
-    intercept[MetadataProcessException] {
-      sql("drop table if exists datamapshowtest")
-      sql("create table datamapshowtest (a string, b string, c string) stored by 'carbondata'")
-      sql("create datamap datamap1 on table datamapshowtest using 'preaggregate' as select count(a) from datamapshowtest")
-      sql(s"CREATE DATAMAP datamap2 ON TABLE datamapshowtest USING '$newClass' DMPROPERTIES('key'='value')")
-      sql("drop datamap datamap1 on table datamapshowtest")
-      val frame = sql("show datamap on table datamapshowtest")
-      assert(frame.collect().length == 1)
-      checkExistence(frame, true, "datamap2", "(NA)", newClass)
-    }
-  }
-
-  test("test if preaggregate load is successfull for hivemetastore") {
-    try {
-      CarbonProperties.getInstance()
-        .addProperty(CarbonCommonConstants.ENABLE_HIVE_SCHEMA_META_STORE, "true")
-      sql("DROP TABLE IF EXISTS maintable")
-      sql(
-        """
-          | CREATE TABLE maintable(id int, name string, city string, age int)
-          | STORED BY 'org.apache.carbondata.format'
-        """.stripMargin)
-      sql(
-        s"""create datamap preagg_sum on table maintable using 'preaggregate' as select id,sum(age) from maintable group by id"""
-
-          .stripMargin)
-      sql(s"LOAD DATA LOCAL INPATH '$testData' into table maintable")
-      checkAnswer(sql(s"select * from maintable_preagg_sum"),
-        Seq(Row(1, 31), Row(2, 27), Row(3, 70), Row(4, 55)))
-    } finally {
-      CarbonProperties.getInstance()
-        .addProperty(CarbonCommonConstants.ENABLE_HIVE_SCHEMA_META_STORE,
-          CarbonCommonConstants.ENABLE_HIVE_SCHEMA_META_STORE_DEFAULT)
-    }
-  }
-
-  test("test preaggregate load for decimal column for hivemetastore") {
-    CarbonProperties.getInstance().addProperty(CarbonCommonConstants.ENABLE_HIVE_SCHEMA_META_STORE, "true")
-    sql("CREATE TABLE uniqdata(CUST_ID int,CUST_NAME String,ACTIVE_EMUI_VERSION string,DOB timestamp,DOJ timestamp, BIGINT_COLUMN1 bigint,BIGINT_COLUMN2 bigint,DECIMAL_COLUMN1 decimal(30,10),DECIMAL_COLUMN2 decimal(36,10),Double_COLUMN1 double, Double_COLUMN2 double,INTEGER_COLUMN1 int) STORED BY 'org.apache.carbondata.format'")
-    sql("insert into uniqdata select 9000,'CUST_NAME_00000','ACTIVE_EMUI_VERSION_00000','1970-01-01 01:00:03','1970-01-01 02:00:03',123372036854,-223372036854,12345678901.1234000000,22345678901.1234000000,11234567489.7976000000,-11234567489.7976000000,1")
-    sql("create datamap uniqdata_agg on table uniqdata using 'preaggregate' as select min(DECIMAL_COLUMN1) from uniqdata group by DECIMAL_COLUMN1")
-    checkAnswer(sql("select * from uniqdata_uniqdata_agg"), Seq(Row(12345678901.1234000000, 12345678901.1234000000)))
-    sql("drop datamap if exists uniqdata_agg on table uniqdata")
-  }
-
-  test("create pre-agg table with path") {
-    sql("drop table if exists main_preagg")
-    sql("drop table if exists main ")
-    val warehouse = s"$metastoredb/warehouse"
-    val path = warehouse + "/" + System.nanoTime + "_preAggTestPath"
-    sql(
-      s"""
-         | create table main(
-         |     year int,
-         |     month int,
-         |     name string,
-         |     salary int)
-         | stored by 'carbondata'
-         | tblproperties('sort_columns'='month,year,name')
-      """.stripMargin)
-    sql("insert into main select 10,11,'amy',12")
-    sql("insert into main select 10,11,'amy',14")
-    sql(
-      s"""
-         | create datamap preagg
-         | on table main
-         | using 'preaggregate'
-         | dmproperties ('path'='$path')
-         | as select name,avg(salary)
-         |    from main
-         |    group by name
-       """.stripMargin)
-    assertResult(true)(new File(path).exists())
-    assertResult(true)(new File(s"${CarbonTablePath.getSegmentPath(path, "0")}")
-      .list(new FilenameFilter {
-        override def accept(dir: File, name: String): Boolean = {
-          name.contains(CarbonCommonConstants.FACT_FILE_EXT)
-        }
-      }).length > 0)
-    checkAnswer(sql("select name,avg(salary) from main group by name"), Row("amy", 13.0))
-    checkAnswer(sql("select * from main_preagg"), Row("amy", 26, 2))
-    sql("drop datamap preagg on table main")
-    assertResult(false)(new File(path).exists())
-    sql("drop table main")
-  }
-
-  override def afterAll {
-    sql("DROP TABLE IF EXISTS maintable")
-    sql("drop table if exists uniqdata")
-    CarbonProperties.getInstance().addProperty(CarbonCommonConstants.ENABLE_HIVE_SCHEMA_META_STORE,
-      CarbonCommonConstants.ENABLE_HIVE_SCHEMA_META_STORE_DEFAULT)
-    sql("drop table if exists datamaptest")
-    sql("drop table if exists datamapshowtest")
-  }
-}

http://git-wip-us.apache.org/repos/asf/carbondata/blob/5a625f4c/integration/spark-common-test/src/test/scala/org/apache/carbondata/spark/testsuite/datamap/TestIndexDataMapCommand.scala
----------------------------------------------------------------------
diff --git a/integration/spark-common-test/src/test/scala/org/apache/carbondata/spark/testsuite/datamap/TestIndexDataMapCommand.scala b/integration/spark-common-test/src/test/scala/org/apache/carbondata/spark/testsuite/datamap/TestIndexDataMapCommand.scala
new file mode 100644
index 0000000..a6ffa9a
--- /dev/null
+++ b/integration/spark-common-test/src/test/scala/org/apache/carbondata/spark/testsuite/datamap/TestIndexDataMapCommand.scala
@@ -0,0 +1,279 @@
+/*
+ * 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.spark.testsuite.datamap
+
+import java.io.{File, FilenameFilter}
+
+import org.apache.spark.sql.Row
+import org.apache.spark.sql.test.util.QueryTest
+import org.scalatest.BeforeAndAfterAll
+
+import org.apache.carbondata.common.exceptions.MetadataProcessException
+import org.apache.carbondata.common.exceptions.sql.{MalformedDataMapCommandException, NoSuchDataMapException}
+import org.apache.carbondata.core.constants.CarbonCommonConstants
+import org.apache.carbondata.core.metadata.CarbonMetadata
+import org.apache.carbondata.core.util.CarbonProperties
+import org.apache.carbondata.core.util.path.CarbonTablePath
+
+class TestIndexDataMapCommand extends QueryTest with BeforeAndAfterAll {
+
+  val testData = s"$resourcesPath/sample.csv"
+
+  override def beforeAll {
+    sql("drop table if exists datamaptest")
+    sql("drop table if exists datamapshowtest")
+    sql("drop table if exists uniqdata")
+    sql("create table datamaptest (a string, b string, c string) stored by 'carbondata'")
+  }
+
+  val newClass = "org.apache.spark.sql.CarbonSource"
+
+  test("test datamap create: don't support using non-exist class") {
+    intercept[MetadataProcessException] {
+      sql(s"CREATE DATAMAP datamap1 ON TABLE datamaptest USING '$newClass'")
+    }
+  }
+
+  test("test datamap create with dmproperties: don't support using non-exist class") {
+    intercept[MetadataProcessException] {
+      sql(s"CREATE DATAMAP datamap2 ON TABLE datamaptest USING '$newClass' DMPROPERTIES('key'='value')")
+    }
+  }
+
+  test("test datamap create with existing name: don't support using non-exist class") {
+    intercept[MetadataProcessException] {
+      sql(
+        s"CREATE DATAMAP datamap2 ON TABLE datamaptest USING '$newClass' DMPROPERTIES('key'='value')")
+    }
+  }
+
+  test("test datamap create with preagg") {
+    sql("drop datamap if exists datamap3 on table datamaptest")
+    sql(
+      "create datamap datamap3 on table datamaptest using 'preaggregate' as select count(a) from datamaptest")
+    val table = CarbonMetadata.getInstance().getCarbonTable("default", "datamaptest")
+    assert(table != null)
+    val dataMapSchemaList = table.getTableInfo.getDataMapSchemaList
+    assert(dataMapSchemaList.size() == 1)
+    assert(dataMapSchemaList.get(0).getDataMapName.equals("datamap3"))
+    assert(dataMapSchemaList.get(0).getChildSchema.getTableName.equals("datamaptest_datamap3"))
+  }
+
+  test("check hivemetastore after drop datamap") {
+    try {
+      CarbonProperties.getInstance()
+        .addProperty(CarbonCommonConstants.ENABLE_HIVE_SCHEMA_META_STORE,
+          "true")
+      sql("drop table if exists hiveMetaStoreTable")
+      sql("create table hiveMetaStoreTable (a string, b string, c string) stored by 'carbondata'")
+
+      sql(
+        "create datamap datamap_hiveMetaStoreTable on table hiveMetaStoreTable using 'preaggregate' as select count(a) from hiveMetaStoreTable")
+      checkExistence(sql("show datamap on table hiveMetaStoreTable"), true, "datamap_hiveMetaStoreTable")
+
+      sql("drop datamap datamap_hiveMetaStoreTable on table hiveMetaStoreTable")
+      checkExistence(sql("show datamap on table hiveMetaStoreTable"), false, "datamap_hiveMetaStoreTable")
+
+    } finally {
+      sql("drop table hiveMetaStoreTable")
+      CarbonProperties.getInstance()
+        .addProperty(CarbonCommonConstants.ENABLE_HIVE_SCHEMA_META_STORE,
+          CarbonCommonConstants.ENABLE_HIVE_SCHEMA_META_STORE_DEFAULT)
+    }
+  }
+
+  test("drop the table having pre-aggregate") {
+    try {
+      CarbonProperties.getInstance()
+        .addProperty(CarbonCommonConstants.ENABLE_HIVE_SCHEMA_META_STORE,
+          "true")
+      sql("drop table if exists hiveMetaStoreTable_1")
+      sql("create table hiveMetaStoreTable_1 (a string, b string, c string) stored by 'carbondata'")
+
+      sql(
+        "create datamap datamap_hiveMetaStoreTable_1 on table hiveMetaStoreTable_1 using 'preaggregate' as select count(a) from hiveMetaStoreTable_1")
+
+      checkExistence(sql("show datamap on table hiveMetaStoreTable_1"),
+        true,
+        "datamap_hiveMetaStoreTable_1")
+
+      sql("drop table hiveMetaStoreTable_1")
+
+      checkExistence(sql("show tables"), false, "datamap_hiveMetaStoreTable_1")
+    } finally {
+      CarbonProperties.getInstance()
+        .addProperty(CarbonCommonConstants.ENABLE_HIVE_SCHEMA_META_STORE,
+          CarbonCommonConstants.ENABLE_HIVE_SCHEMA_META_STORE_DEFAULT)
+    }
+  }
+
+  test("test datamap create with preagg with duplicate name") {
+    sql(
+      s"""
+         | CREATE DATAMAP datamap10 ON TABLE datamaptest
+         | USING 'preaggregate'
+         | AS SELECT COUNT(a) FROM datamaptest
+         """.stripMargin)
+    intercept[MalformedDataMapCommandException] {
+      sql(
+        s"""
+           | CREATE DATAMAP datamap10 ON TABLE datamaptest
+           | USING 'preaggregate'
+           | AS SELECT COUNT(a) FROM datamaptest
+         """.stripMargin)
+    }
+    val table = CarbonMetadata.getInstance().getCarbonTable("default", "datamaptest")
+    assert(table != null)
+    val dataMapSchemaList = table.getTableInfo.getDataMapSchemaList
+    assert(dataMapSchemaList.size() == 2)
+  }
+
+  test("test drop non-exist datamap") {
+    intercept[NoSuchDataMapException] {
+      sql("drop datamap nonexist on table datamaptest")
+    }
+    val table = CarbonMetadata.getInstance().getCarbonTable("default", "datamaptest")
+    assert(table != null)
+    val dataMapSchemaList = table.getTableInfo.getDataMapSchemaList
+    assert(dataMapSchemaList.size() == 2)
+  }
+
+  test("test show datamap without preaggregate: don't support using non-exist class") {
+    intercept[MetadataProcessException] {
+      sql("drop table if exists datamapshowtest")
+      sql("create table datamapshowtest (a string, b string, c string) stored by 'carbondata'")
+      sql(s"CREATE DATAMAP datamap1 ON TABLE datamapshowtest USING '$newClass' ")
+      sql(s"CREATE DATAMAP datamap2 ON TABLE datamapshowtest USING '$newClass' ")
+      checkExistence(sql("SHOW DATAMAP ON TABLE datamapshowtest"), true, "datamap1", "datamap2", "(NA)", newClass)
+    }
+  }
+
+  test("test show datamap with preaggregate: don't support using non-exist class") {
+    intercept[MetadataProcessException] {
+      sql("drop table if exists datamapshowtest")
+      sql("create table datamapshowtest (a string, b string, c string) stored by 'carbondata'")
+      sql("create datamap datamap1 on table datamapshowtest using 'preaggregate' as select count(a) from datamapshowtest")
+      sql(s"CREATE DATAMAP datamap2 ON TABLE datamapshowtest USING '$newClass' ")
+      val frame = sql("show datamap on table datamapshowtest")
+      assert(frame.collect().length == 2)
+      checkExistence(frame, true, "datamap1", "datamap2", "(NA)", newClass, "default.datamapshowtest_datamap1")
+    }
+  }
+
+  test("test show datamap with no datamap") {
+    sql("drop table if exists datamapshowtest")
+    sql("create table datamapshowtest (a string, b string, c string) stored by 'carbondata'")
+    assert(sql("show datamap on table datamapshowtest").collect().length == 0)
+  }
+
+  test("test show datamap after dropping datamap: don't support using non-exist class") {
+    intercept[MetadataProcessException] {
+      sql("drop table if exists datamapshowtest")
+      sql("create table datamapshowtest (a string, b string, c string) stored by 'carbondata'")
+      sql("create datamap datamap1 on table datamapshowtest using 'preaggregate' as select count(a) from datamapshowtest")
+      sql(s"CREATE DATAMAP datamap2 ON TABLE datamapshowtest USING '$newClass' ")
+      sql("drop datamap datamap1 on table datamapshowtest")
+      val frame = sql("show datamap on table datamapshowtest")
+      assert(frame.collect().length == 1)
+      checkExistence(frame, true, "datamap2", "(NA)", newClass)
+    }
+  }
+
+  test("test if preaggregate load is successfull for hivemetastore") {
+    try {
+      CarbonProperties.getInstance()
+        .addProperty(CarbonCommonConstants.ENABLE_HIVE_SCHEMA_META_STORE, "true")
+      sql("DROP TABLE IF EXISTS maintable")
+      sql(
+        """
+          | CREATE TABLE maintable(id int, name string, city string, age int)
+          | STORED BY 'org.apache.carbondata.format'
+        """.stripMargin)
+      sql(
+        s"""create datamap preagg_sum on table maintable using 'preaggregate' as select id,sum(age) from maintable group by id"""
+
+          .stripMargin)
+      sql(s"LOAD DATA LOCAL INPATH '$testData' into table maintable")
+      checkAnswer(sql(s"select * from maintable_preagg_sum"),
+        Seq(Row(1, 31), Row(2, 27), Row(3, 70), Row(4, 55)))
+    } finally {
+      CarbonProperties.getInstance()
+        .addProperty(CarbonCommonConstants.ENABLE_HIVE_SCHEMA_META_STORE,
+          CarbonCommonConstants.ENABLE_HIVE_SCHEMA_META_STORE_DEFAULT)
+    }
+  }
+
+  test("test preaggregate load for decimal column for hivemetastore") {
+    CarbonProperties.getInstance().addProperty(CarbonCommonConstants.ENABLE_HIVE_SCHEMA_META_STORE, "true")
+    sql("CREATE TABLE uniqdata(CUST_ID int,CUST_NAME String,ACTIVE_EMUI_VERSION string,DOB timestamp,DOJ timestamp, BIGINT_COLUMN1 bigint,BIGINT_COLUMN2 bigint,DECIMAL_COLUMN1 decimal(30,10),DECIMAL_COLUMN2 decimal(36,10),Double_COLUMN1 double, Double_COLUMN2 double,INTEGER_COLUMN1 int) STORED BY 'org.apache.carbondata.format'")
+    sql("insert into uniqdata select 9000,'CUST_NAME_00000','ACTIVE_EMUI_VERSION_00000','1970-01-01 01:00:03','1970-01-01 02:00:03',123372036854,-223372036854,12345678901.1234000000,22345678901.1234000000,11234567489.7976000000,-11234567489.7976000000,1")
+    sql("create datamap uniqdata_agg on table uniqdata using 'preaggregate' as select min(DECIMAL_COLUMN1) from uniqdata group by DECIMAL_COLUMN1")
+    checkAnswer(sql("select * from uniqdata_uniqdata_agg"), Seq(Row(12345678901.1234000000, 12345678901.1234000000)))
+    sql("drop datamap if exists uniqdata_agg on table uniqdata")
+  }
+
+  test("create pre-agg table with path") {
+    sql("drop table if exists main_preagg")
+    sql("drop table if exists main ")
+    val warehouse = s"$metastoredb/warehouse"
+    val path = warehouse + "/" + System.nanoTime + "_preAggTestPath"
+    sql(
+      s"""
+         | create table main(
+         |     year int,
+         |     month int,
+         |     name string,
+         |     salary int)
+         | stored by 'carbondata'
+         | tblproperties('sort_columns'='month,year,name')
+      """.stripMargin)
+    sql("insert into main select 10,11,'amy',12")
+    sql("insert into main select 10,11,'amy',14")
+    sql(
+      s"""
+         | create datamap preagg
+         | on table main
+         | using 'preaggregate'
+         | dmproperties ('path'='$path')
+         | as select name,avg(salary)
+         |    from main
+         |    group by name
+       """.stripMargin)
+    assertResult(true)(new File(path).exists())
+    assertResult(true)(new File(s"${CarbonTablePath.getSegmentPath(path, "0")}")
+      .list(new FilenameFilter {
+        override def accept(dir: File, name: String): Boolean = {
+          name.contains(CarbonCommonConstants.FACT_FILE_EXT)
+        }
+      }).length > 0)
+    checkAnswer(sql("select name,avg(salary) from main group by name"), Row("amy", 13.0))
+    checkAnswer(sql("select * from main_preagg"), Row("amy", 26, 2))
+    sql("drop datamap preagg on table main")
+    assertResult(false)(new File(path).exists())
+    sql("drop table main")
+  }
+
+  override def afterAll {
+    sql("DROP TABLE IF EXISTS maintable")
+    sql("drop table if exists uniqdata")
+    CarbonProperties.getInstance().addProperty(CarbonCommonConstants.ENABLE_HIVE_SCHEMA_META_STORE,
+      CarbonCommonConstants.ENABLE_HIVE_SCHEMA_META_STORE_DEFAULT)
+    sql("drop table if exists datamaptest")
+    sql("drop table if exists datamapshowtest")
+  }
+}

http://git-wip-us.apache.org/repos/asf/carbondata/blob/5a625f4c/integration/spark-common-test/src/test/scala/org/apache/carbondata/spark/testsuite/iud/InsertOverwriteConcurrentTest.scala
----------------------------------------------------------------------
diff --git a/integration/spark-common-test/src/test/scala/org/apache/carbondata/spark/testsuite/iud/InsertOverwriteConcurrentTest.scala b/integration/spark-common-test/src/test/scala/org/apache/carbondata/spark/testsuite/iud/InsertOverwriteConcurrentTest.scala
index 84b59e6..248441f 100644
--- a/integration/spark-common-test/src/test/scala/org/apache/carbondata/spark/testsuite/iud/InsertOverwriteConcurrentTest.scala
+++ b/integration/spark-common-test/src/test/scala/org/apache/carbondata/spark/testsuite/iud/InsertOverwriteConcurrentTest.scala
@@ -28,8 +28,8 @@ import org.apache.spark.sql.{DataFrame, SaveMode}
 import org.scalatest.{BeforeAndAfterAll, BeforeAndAfterEach}
 
 import org.apache.carbondata.core.constants.CarbonCommonConstants
-import org.apache.carbondata.core.datamap.dev.cgdatamap.{AbstractCoarseGrainDataMap, AbstractCoarseGrainDataMapFactory}
-import org.apache.carbondata.core.datamap.dev.{AbstractDataMapWriter, DataMap}
+import org.apache.carbondata.core.datamap.dev.cgdatamap.{AbstractCoarseGrainIndexDataMapFactory, AbstractCoarseGrainIndexDataMap}
+import org.apache.carbondata.core.datamap.dev.AbstractDataMapWriter
 import org.apache.carbondata.core.datamap.{DataMapDistributable, DataMapMeta, DataMapStoreManager}
 import org.apache.carbondata.core.datastore.page.ColumnPage
 import org.apache.carbondata.core.metadata.AbsoluteTableIdentifier
@@ -47,7 +47,12 @@ class InsertOverwriteConcurrentTest extends QueryTest with BeforeAndAfterAll wit
     buildTestData()
 
     // register hook to the table to sleep, thus the other command will be executed
-    sql(s"create datamap test on table orders using '${classOf[WaitingDataMap].getName}' as select count(a) from hiveMetaStoreTable_1")
+    sql(
+      s"""
+         | create datamap test on table orders
+         | using '${classOf[WaitingIndexDataMap].getName}'
+         | as select count(a) from hiveMetaStoreTable_1")
+       """.stripMargin)
   }
 
   private def buildTestData(): Unit = {
@@ -101,7 +106,7 @@ class InsertOverwriteConcurrentTest extends QueryTest with BeforeAndAfterAll wit
     )
     while (!Global.overwriteRunning && count < 1000) {
       Thread.sleep(10)
-      // to avoid dead loop in case WaitingDataMap is not invoked
+      // to avoid dead loop in case WaitingIndexDataMap is not invoked
       count += 1
     }
     future
@@ -162,7 +167,7 @@ object Global {
   var overwriteRunning = false
 }
 
-class WaitingDataMap() extends AbstractCoarseGrainDataMapFactory {
+class WaitingIndexDataMap() extends AbstractCoarseGrainIndexDataMapFactory {
 
   override def init(identifier: AbsoluteTableIdentifier, dataMapSchema: DataMapSchema): Unit = { }
 
@@ -172,9 +177,9 @@ class WaitingDataMap() extends AbstractCoarseGrainDataMapFactory {
 
   override def clear(): Unit = {}
 
-  override def getDataMaps(distributable: DataMapDistributable): java.util.List[AbstractCoarseGrainDataMap] = ???
+  override def getDataMaps(distributable: DataMapDistributable): java.util.List[AbstractCoarseGrainIndexDataMap] = ???
 
-  override def getDataMaps(segmentId: String): util.List[AbstractCoarseGrainDataMap] = ???
+  override def getDataMaps(segmentId: String): util.List[AbstractCoarseGrainIndexDataMap] = ???
 
   override def createWriter(segmentId: String, writerPath: String): AbstractDataMapWriter = {
     new AbstractDataMapWriter(null, segmentId, writerPath) {

http://git-wip-us.apache.org/repos/asf/carbondata/blob/5a625f4c/integration/spark2/src/main/java/org/apache/carbondata/datamap/DataMapManager.java
----------------------------------------------------------------------
diff --git a/integration/spark2/src/main/java/org/apache/carbondata/datamap/DataMapManager.java b/integration/spark2/src/main/java/org/apache/carbondata/datamap/DataMapManager.java
new file mode 100644
index 0000000..2b3a306
--- /dev/null
+++ b/integration/spark2/src/main/java/org/apache/carbondata/datamap/DataMapManager.java
@@ -0,0 +1,53 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.carbondata.datamap;
+
+import org.apache.carbondata.core.metadata.schema.table.DataMapSchema;
+
+import static org.apache.carbondata.core.metadata.schema.datamap.DataMapProvider.PREAGGREGATE;
+import static org.apache.carbondata.core.metadata.schema.datamap.DataMapProvider.TIMESERIES;
+
+public class DataMapManager {
+
+  private static DataMapManager INSTANCE;
+
+  private DataMapManager() { }
+
+  public static synchronized DataMapManager get() {
+    if (INSTANCE == null) {
+      INSTANCE = new DataMapManager();
+    }
+    return INSTANCE;
+  }
+
+  /**
+   * Return a DataMapProvider instance for specified dataMapSchema.
+   */
+  public DataMapProvider getDataMapProvider(DataMapSchema dataMapSchema) {
+    DataMapProvider provider;
+    if (dataMapSchema.getClassName().equalsIgnoreCase(PREAGGREGATE.toString())) {
+      provider = new PreAggregateDataMapProvider();
+    } else if (dataMapSchema.getClassName().equalsIgnoreCase(TIMESERIES.toString())) {
+      provider = new TimeseriesDataMapProvider();
+    } else {
+      provider = new IndexDataMapProvider();
+    }
+    return provider;
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/carbondata/blob/5a625f4c/integration/spark2/src/main/java/org/apache/carbondata/datamap/DataMapProperty.java
----------------------------------------------------------------------
diff --git a/integration/spark2/src/main/java/org/apache/carbondata/datamap/DataMapProperty.java b/integration/spark2/src/main/java/org/apache/carbondata/datamap/DataMapProperty.java
new file mode 100644
index 0000000..0cf0d04
--- /dev/null
+++ b/integration/spark2/src/main/java/org/apache/carbondata/datamap/DataMapProperty.java
@@ -0,0 +1,32 @@
+/*
+ * 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.datamap;
+
+import org.apache.carbondata.common.annotations.InterfaceAudience;
+
+/**
+ * Property that can be specified when creating DataMap
+ */
+@InterfaceAudience.Internal
+public class DataMapProperty {
+
+  /**
+   * Used to specify the store location of the datamap
+   */
+  public static final String PATH = "path";
+}

http://git-wip-us.apache.org/repos/asf/carbondata/blob/5a625f4c/integration/spark2/src/main/java/org/apache/carbondata/datamap/DataMapProvider.java
----------------------------------------------------------------------
diff --git a/integration/spark2/src/main/java/org/apache/carbondata/datamap/DataMapProvider.java b/integration/spark2/src/main/java/org/apache/carbondata/datamap/DataMapProvider.java
new file mode 100644
index 0000000..a71e0d8
--- /dev/null
+++ b/integration/spark2/src/main/java/org/apache/carbondata/datamap/DataMapProvider.java
@@ -0,0 +1,105 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.carbondata.datamap;
+
+import org.apache.carbondata.common.annotations.InterfaceAudience;
+import org.apache.carbondata.common.annotations.InterfaceStability;
+import org.apache.carbondata.common.exceptions.sql.MalformedDataMapCommandException;
+import org.apache.carbondata.core.metadata.schema.table.CarbonTable;
+import org.apache.carbondata.core.metadata.schema.table.DataMapSchema;
+import org.apache.carbondata.processing.exception.DataLoadingException;
+
+import org.apache.spark.sql.SparkSession;
+
+/**
+ * DataMap is a accelerator for certain type of query. Developer can add new DataMap
+ * implementation to improve query performance.
+ *
+ * Currently two types of DataMap are supported
+ * <ol>
+ *   <li> MVDataMap: materialized view type of DataMap to accelerate olap style query,
+ * like SPJG query (select, predicate, join, groupby) </li>
+ *   <li> IndexDataMap: index type of DataMap 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.
+ *
+ * <br>Currently CarbonData supports following provider:
+ * <ol>
+ *   <li> preaggregate: one type of MVDataMap that do pre-aggregate of single table </li>
+ *   <li> timeseries: one type of MVDataMap that do pre-aggregate based on time dimension
+ *     of the table </li>
+ *   <li> class name of {@link org.apache.carbondata.core.datamap.dev.IndexDataMapFactory}
+ * implementation: Developer can implement new type of IndexDataMap by extending
+ * {@link org.apache.carbondata.core.datamap.dev.IndexDataMapFactory} </li>
+ * </ol>
+ *
+ * @since 1.4.0
+ */
+@InterfaceAudience.Developer("DataMap")
+@InterfaceStability.Unstable
+public interface DataMapProvider {
+
+  /**
+   * Initialize a datamap's metadata.
+   * This is called when user creates datamap, for example "CREATE DATAMAP dm ON TABLE mainTable"
+   * Implementation should initialize metadata for datamap, like creating table
+   */
+  void initMeta(CarbonTable mainTable, DataMapSchema dataMapSchema, String ctasSqlStatement,
+      SparkSession sparkSession) throws MalformedDataMapCommandException;
+
+  /**
+   * Initialize a datamap's data.
+   * This is called when user creates datamap, for example "CREATE DATAMAP dm ON TABLE mainTable"
+   * Implementation should initialize data for datamap, like creating data folders
+   */
+  void initData(CarbonTable mainTable, SparkSession sparkSession);
+
+  /**
+   * Opposite operation of {@link #initMeta(CarbonTable, DataMapSchema, String, SparkSession)}.
+   * This is called when user drops datamap, for example "DROP DATAMAP dm ON TABLE mainTable"
+   * Implementation should clean all meta for the datamap
+   */
+  void freeMeta(CarbonTable mainTable, DataMapSchema dataMapSchema, SparkSession sparkSession);
+
+  /**
+   * Opposite operation of {@link #initData(CarbonTable, SparkSession)}.
+   * This is called when user drops datamap, for example "DROP DATAMAP dm ON TABLE mainTable"
+   * Implementation should clean all data for the datamap
+   */
+  void freeData(CarbonTable mainTable, DataMapSchema dataMapSchema, SparkSession sparkSession);
+
+  /**
+   * Rebuild the datamap by loading all existing data from mainTable
+   * This is called when refreshing the datamap when
+   * 1. after datamap creation and if `autoRefreshDataMap` is set to true
+   * 2. user manually trigger refresh datamap command
+   */
+  void rebuild(CarbonTable mainTable, SparkSession sparkSession) throws DataLoadingException;
+
+  /**
+   * Build the datamap incrementally by loading specified segment data
+   * This is called when user manually trigger refresh datamap
+   */
+  void incrementalBuild(CarbonTable mainTable, String[] segmentIds, SparkSession sparkSession)
+    throws DataLoadingException;
+
+}

http://git-wip-us.apache.org/repos/asf/carbondata/blob/5a625f4c/integration/spark2/src/main/java/org/apache/carbondata/datamap/IndexDataMapProvider.java
----------------------------------------------------------------------
diff --git a/integration/spark2/src/main/java/org/apache/carbondata/datamap/IndexDataMapProvider.java b/integration/spark2/src/main/java/org/apache/carbondata/datamap/IndexDataMapProvider.java
new file mode 100644
index 0000000..e11e522
--- /dev/null
+++ b/integration/spark2/src/main/java/org/apache/carbondata/datamap/IndexDataMapProvider.java
@@ -0,0 +1,116 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.carbondata.datamap;
+
+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.core.datamap.DataMapRegistry;
+import org.apache.carbondata.core.datamap.DataMapStoreManager;
+import org.apache.carbondata.core.datamap.dev.IndexDataMapFactory;
+import org.apache.carbondata.core.metadata.schema.table.CarbonTable;
+import org.apache.carbondata.core.metadata.schema.table.DataMapSchema;
+import org.apache.carbondata.format.TableInfo;
+
+import org.apache.spark.sql.SparkSession;
+import org.apache.spark.sql.execution.command.preaaggregate.PreAggregateUtil;
+
+@InterfaceAudience.Internal
+public class IndexDataMapProvider implements DataMapProvider {
+
+  private TableInfo originalTableInfo;
+
+  @Override
+  public void initMeta(CarbonTable mainTable, DataMapSchema dataMapSchema, String ctasSqlStatement,
+      SparkSession sparkSession) throws MalformedDataMapCommandException {
+    IndexDataMapFactory dataMapFactory = createIndexDataMapFactory(dataMapSchema);
+    DataMapStoreManager.getInstance().registerDataMap(
+        mainTable.getAbsoluteTableIdentifier(), dataMapSchema, dataMapFactory);
+    originalTableInfo = PreAggregateUtil.updateMainTable(mainTable, dataMapSchema, sparkSession);
+  }
+
+  @Override
+  public void initData(CarbonTable mainTable, SparkSession sparkSession) {
+    // Nothing is needed to do by default
+  }
+
+  @Override
+  public void freeMeta(CarbonTable mainTable, DataMapSchema dataMapSchema,
+      SparkSession sparkSession) {
+    PreAggregateUtil.updateSchemaInfo(mainTable, originalTableInfo, sparkSession);
+  }
+
+  @Override
+  public void freeData(CarbonTable mainTable, DataMapSchema dataMapSchema,
+      SparkSession sparkSession) {
+    DataMapStoreManager.getInstance().clearDataMap(
+        mainTable.getAbsoluteTableIdentifier(), dataMapSchema.getDataMapName());
+  }
+
+  @Override
+  public void rebuild(CarbonTable mainTable, SparkSession sparkSession) {
+    // Nothing is needed to do by default
+  }
+
+  @Override
+  public void incrementalBuild(CarbonTable mainTable, String[] segmentIds,
+      SparkSession sparkSession) {
+    throw new UnsupportedOperationException();
+  }
+
+  private IndexDataMapFactory createIndexDataMapFactory(DataMapSchema dataMapSchema)
+      throws MalformedDataMapCommandException {
+    IndexDataMapFactory dataMapFactory;
+    try {
+      // try to create DataMapProvider instance by taking providerName as class name
+      Class<? extends IndexDataMapFactory> providerClass =
+          (Class<? extends IndexDataMapFactory>) Class.forName(dataMapSchema.getClassName());
+      dataMapFactory = providerClass.newInstance();
+    } catch (ClassNotFoundException e) {
+      // try to create DataMapProvider instance by taking providerName as short name
+      dataMapFactory = getDataMapFactoryByShortName(dataMapSchema.getClassName());
+    } catch (Throwable e) {
+      throw new MetadataProcessException(
+          "failed to create DataMapProvider '" + dataMapSchema.getClassName() + "'", e);
+    }
+    return dataMapFactory;
+  }
+
+  private IndexDataMapFactory getDataMapFactoryByShortName(String providerName)
+      throws MalformedDataMapCommandException {
+    IndexDataMapFactory dataMapFactory;
+    String className = DataMapRegistry.getDataMapClassName(providerName);
+    if (className != null) {
+      try {
+        Class<? extends IndexDataMapFactory> datamapClass =
+            (Class<? extends IndexDataMapFactory>) Class.forName(providerName);
+        dataMapFactory = datamapClass.newInstance();
+      } catch (ClassNotFoundException ex) {
+        throw new MalformedDataMapCommandException(
+            "DataMap '" + providerName + "' not found", ex);
+      } catch (Throwable ex) {
+        throw new MetadataProcessException(
+            "failed to create DataMap '" + providerName + "'", ex);
+      }
+    } else {
+      throw new MalformedDataMapCommandException(
+          "DataMap '" + providerName + "' not found");
+    }
+    return dataMapFactory;
+  }
+}

http://git-wip-us.apache.org/repos/asf/carbondata/blob/5a625f4c/integration/spark2/src/main/java/org/apache/carbondata/datamap/PreAggregateDataMapProvider.java
----------------------------------------------------------------------
diff --git a/integration/spark2/src/main/java/org/apache/carbondata/datamap/PreAggregateDataMapProvider.java b/integration/spark2/src/main/java/org/apache/carbondata/datamap/PreAggregateDataMapProvider.java
new file mode 100644
index 0000000..ac38347
--- /dev/null
+++ b/integration/spark2/src/main/java/org/apache/carbondata/datamap/PreAggregateDataMapProvider.java
@@ -0,0 +1,92 @@
+/*
+ * 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.datamap;
+
+import org.apache.carbondata.common.annotations.InterfaceAudience;
+import org.apache.carbondata.common.exceptions.sql.MalformedDataMapCommandException;
+import org.apache.carbondata.core.metadata.schema.table.CarbonTable;
+import org.apache.carbondata.core.metadata.schema.table.DataMapSchema;
+
+import org.apache.spark.sql.SparkSession;
+import org.apache.spark.sql.execution.command.preaaggregate.PreAggregateTableHelper;
+import org.apache.spark.sql.execution.command.table.CarbonDropTableCommand;
+import scala.Some;
+
+@InterfaceAudience.Internal
+public class PreAggregateDataMapProvider implements DataMapProvider {
+  protected PreAggregateTableHelper helper;
+  protected CarbonDropTableCommand dropTableCommand;
+
+  @Override
+  public void initMeta(CarbonTable mainTable, DataMapSchema dataMapSchema, String ctasSqlStatement,
+      SparkSession sparkSession) throws MalformedDataMapCommandException {
+    validateDmProperty(dataMapSchema);
+    helper = new PreAggregateTableHelper(
+        mainTable, dataMapSchema.getDataMapName(), dataMapSchema.getClassName(),
+        dataMapSchema.getProperties(), ctasSqlStatement, null);
+    helper.initMeta(sparkSession);
+  }
+
+  private void validateDmProperty(DataMapSchema dataMapSchema)
+      throws MalformedDataMapCommandException {
+    if (!dataMapSchema.getProperties().isEmpty()) {
+      if (dataMapSchema.getProperties().size() > 1 ||
+          !dataMapSchema.getProperties().containsKey(DataMapProperty.PATH)) {
+        throw new MalformedDataMapCommandException(
+            "Only 'path' dmproperty is allowed for this datamap");
+      }
+    }
+  }
+
+  @Override
+  public void initData(CarbonTable mainTable, SparkSession sparkSession) {
+    // Nothing is needed to do by default
+  }
+
+  @Override
+  public void freeMeta(CarbonTable mainTable, DataMapSchema dataMapSchema,
+      SparkSession sparkSession) {
+    dropTableCommand = new CarbonDropTableCommand(
+        true,
+        new Some<>(dataMapSchema.getRelationIdentifier().getDatabaseName()),
+        dataMapSchema.getRelationIdentifier().getTableName(),
+        true);
+    dropTableCommand.processMetadata(sparkSession);
+  }
+
+  @Override
+  public void freeData(CarbonTable mainTable, DataMapSchema dataMapSchema,
+      SparkSession sparkSession) {
+    if (dropTableCommand != null) {
+      dropTableCommand.processData(sparkSession);
+    }
+  }
+
+  @Override
+  public void rebuild(CarbonTable mainTable, SparkSession sparkSession) {
+    if (helper != null) {
+      helper.initData(sparkSession);
+    }
+  }
+
+  @Override
+  public void incrementalBuild(CarbonTable mainTable, String[] segmentIds,
+      SparkSession sparkSession) {
+    throw new UnsupportedOperationException();
+  }
+}

http://git-wip-us.apache.org/repos/asf/carbondata/blob/5a625f4c/integration/spark2/src/main/java/org/apache/carbondata/datamap/TimeseriesDataMapProvider.java
----------------------------------------------------------------------
diff --git a/integration/spark2/src/main/java/org/apache/carbondata/datamap/TimeseriesDataMapProvider.java b/integration/spark2/src/main/java/org/apache/carbondata/datamap/TimeseriesDataMapProvider.java
new file mode 100644
index 0000000..510839d
--- /dev/null
+++ b/integration/spark2/src/main/java/org/apache/carbondata/datamap/TimeseriesDataMapProvider.java
@@ -0,0 +1,49 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.carbondata.datamap;
+
+import java.util.Map;
+
+import org.apache.carbondata.common.annotations.InterfaceAudience;
+import org.apache.carbondata.core.metadata.schema.table.CarbonTable;
+import org.apache.carbondata.core.metadata.schema.table.DataMapSchema;
+
+import org.apache.spark.sql.SparkSession;
+import org.apache.spark.sql.execution.command.preaaggregate.PreAggregateTableHelper;
+import org.apache.spark.sql.execution.command.timeseries.TimeSeriesUtil;
+import scala.Tuple2;
+
+@InterfaceAudience.Internal
+public class TimeseriesDataMapProvider extends PreAggregateDataMapProvider {
+
+  @Override
+  public void initMeta(CarbonTable mainTable, DataMapSchema dataMapSchema, String ctasSqlStatement,
+      SparkSession sparkSession) {
+    Map<String, String> dmProperties = dataMapSchema.getProperties();
+    String dmProviderName = dataMapSchema.getClassName();
+    TimeSeriesUtil.validateTimeSeriesGranularity(dmProperties, dmProviderName);
+    Tuple2<String, String> details =
+        TimeSeriesUtil.getTimeSeriesGranularityDetails(dmProperties, dmProviderName);
+    dmProperties.remove(details._1());
+    helper = new PreAggregateTableHelper(
+        mainTable, dataMapSchema.getDataMapName(), dataMapSchema.getClassName(),
+        dmProperties, ctasSqlStatement, details._1());
+    helper.initMeta(sparkSession);
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/carbondata/blob/5a625f4c/integration/spark2/src/main/scala/org/apache/spark/sql/CarbonEnv.scala
----------------------------------------------------------------------
diff --git a/integration/spark2/src/main/scala/org/apache/spark/sql/CarbonEnv.scala b/integration/spark2/src/main/scala/org/apache/spark/sql/CarbonEnv.scala
index 870b1f3..5fb3d56 100644
--- a/integration/spark2/src/main/scala/org/apache/spark/sql/CarbonEnv.scala
+++ b/integration/spark2/src/main/scala/org/apache/spark/sql/CarbonEnv.scala
@@ -93,7 +93,7 @@ class CarbonEnv {
             properties.addProperty(CarbonCommonConstants.STORE_LOCATION, storePath)
           }
           LOGGER.info(s"carbon env initial: $storePath")
-          // trigger event for CarbonEnv init
+          // trigger event for CarbonEnv create
           val operationContext = new OperationContext
           val carbonEnvInitPreEvent: CarbonEnvInitPreEvent =
             CarbonEnvInitPreEvent(sparkSession, storePath)

http://git-wip-us.apache.org/repos/asf/carbondata/blob/5a625f4c/integration/spark2/src/main/scala/org/apache/spark/sql/execution/command/datamap/CarbonCreateDataMapCommand.scala
----------------------------------------------------------------------
diff --git a/integration/spark2/src/main/scala/org/apache/spark/sql/execution/command/datamap/CarbonCreateDataMapCommand.scala b/integration/spark2/src/main/scala/org/apache/spark/sql/execution/command/datamap/CarbonCreateDataMapCommand.scala
index 3d3f83b..3f22955 100644
--- a/integration/spark2/src/main/scala/org/apache/spark/sql/execution/command/datamap/CarbonCreateDataMapCommand.scala
+++ b/integration/spark2/src/main/scala/org/apache/spark/sql/execution/command/datamap/CarbonCreateDataMapCommand.scala
@@ -21,16 +21,11 @@ import scala.collection.JavaConverters._
 import org.apache.spark.sql._
 import org.apache.spark.sql.catalyst.TableIdentifier
 import org.apache.spark.sql.execution.command._
-import org.apache.spark.sql.execution.command.preaaggregate.{CreatePreAggregateTableCommand, PreAggregateUtil}
-import org.apache.spark.sql.execution.command.timeseries.TimeSeriesUtil
-import org.apache.spark.sql.hive.CarbonRelation
 
-import org.apache.carbondata.common.exceptions.MetadataProcessException
 import org.apache.carbondata.common.exceptions.sql.{MalformedCarbonCommandException, MalformedDataMapCommandException}
 import org.apache.carbondata.common.logging.LogServiceFactory
-import org.apache.carbondata.core.datamap.DataMapStoreManager
-import org.apache.carbondata.core.metadata.schema.datamap.DataMapProvider._
 import org.apache.carbondata.core.metadata.schema.table.{CarbonTable, DataMapSchema}
+import org.apache.carbondata.datamap.{DataMapManager, DataMapProvider}
 
 /**
  * Below command class will be used to create datamap on table
@@ -44,64 +39,30 @@ case class CarbonCreateDataMapCommand(
     queryString: Option[String])
   extends AtomicRunnableCommand {
 
-  var createPreAggregateTableCommands: CreatePreAggregateTableCommand = _
+  private var dataMapProvider: DataMapProvider = _
+  private var mainTable: CarbonTable = _
+  private var dataMapSchema: DataMapSchema = _
 
   override def processMetadata(sparkSession: SparkSession): Seq[Row] = {
     // since streaming segment does not support building index and pre-aggregate yet,
     // so streaming table does not support create datamap
-    val carbonTable =
-    CarbonEnv.getCarbonTable(tableIdentifier.database, tableIdentifier.table)(sparkSession)
-    if (carbonTable.isStreamingTable) {
+    mainTable =
+      CarbonEnv.getCarbonTable(tableIdentifier.database, tableIdentifier.table)(sparkSession)
+    if (mainTable.isStreamingTable) {
       throw new MalformedCarbonCommandException("Streaming table does not support creating datamap")
     }
-    validateDataMapName(carbonTable)
+    validateDataMapName(mainTable)
+    dataMapSchema = new DataMapSchema(dataMapName, dmClassName)
+    dataMapSchema.setProperties(new java.util.HashMap[String, String](dmproperties.asJava))
+    dataMapProvider = DataMapManager.get().getDataMapProvider(dataMapSchema)
+    dataMapProvider.initMeta(mainTable, dataMapSchema, queryString.orNull, sparkSession)
 
-    if (dmClassName.equalsIgnoreCase(PREAGGREGATE.toString) ||
-      dmClassName.equalsIgnoreCase(TIMESERIES.toString)) {
-      TimeSeriesUtil.validateTimeSeriesGranularity(dmproperties, dmClassName)
-      createPreAggregateTableCommands = if (dmClassName.equalsIgnoreCase(TIMESERIES.toString)) {
-        val details = TimeSeriesUtil
-          .getTimeSeriesGranularityDetails(dmproperties, dmClassName)
-        val updatedDmProperties = dmproperties - details._1
-        CreatePreAggregateTableCommand(dataMapName,
-          tableIdentifier,
-          dmClassName,
-          updatedDmProperties,
-          queryString.get,
-          Some(details._1))
-      } else {
-        CreatePreAggregateTableCommand(
-          dataMapName,
-          tableIdentifier,
-          dmClassName,
-          dmproperties,
-          queryString.get
-        )
-      }
-      try {
-        createPreAggregateTableCommands.processMetadata(sparkSession)
-      } catch {
-        case e: Throwable => throw new MetadataProcessException(s"Failed to create datamap " +
-                                                                s"'$dataMapName'", e)
-      }
-    } else {
-      val dataMapSchema = new DataMapSchema(dataMapName, dmClassName)
-      dataMapSchema.setProperties(new java.util.HashMap[String, String](dmproperties.asJava))
-      val dbName = CarbonEnv.getDatabaseName(tableIdentifier.database)(sparkSession)
-      val carbonTable = CarbonEnv.getInstance(sparkSession).carbonMetastore.lookupRelation(
-        Some(dbName),
-        tableIdentifier.table)(sparkSession).asInstanceOf[CarbonRelation].carbonTable
-      DataMapStoreManager.getInstance().createAndRegisterDataMap(
-        carbonTable.getAbsoluteTableIdentifier, dataMapSchema)
-      // Save DataMapSchema in the  schema file of main table
-      PreAggregateUtil.updateMainTable(carbonTable, dataMapSchema, sparkSession)
-    }
     val LOGGER = LogServiceFactory.getLogService(this.getClass.getCanonicalName)
     LOGGER.audit(s"DataMap $dataMapName successfully added to Table ${tableIdentifier.table}")
     Seq.empty
   }
 
-  private def validateDataMapName(carbonTable: CarbonTable) = {
+  private def validateDataMapName(carbonTable: CarbonTable): Unit = {
     val existingDataMaps = carbonTable.getTableInfo.getDataMapSchemaList
     existingDataMaps.asScala.foreach { dataMapSchema =>
       if (dataMapSchema.getDataMapName.equalsIgnoreCase(dataMapName)) {
@@ -111,18 +72,19 @@ case class CarbonCreateDataMapCommand(
   }
 
   override def processData(sparkSession: SparkSession): Seq[Row] = {
-    if (dmClassName.equalsIgnoreCase(PREAGGREGATE.toString) ||
-      dmClassName.equalsIgnoreCase(TIMESERIES.toString)) {
-      createPreAggregateTableCommands.processData(sparkSession)
-    } else {
-      Seq.empty
+    if (dataMapProvider != null) {
+      dataMapProvider.initData(mainTable, sparkSession)
+      if (mainTable.isAutoRefreshDataMap) {
+        dataMapProvider.rebuild(mainTable, sparkSession)
+      }
     }
+    Seq.empty
   }
 
   override def undoMetadata(sparkSession: SparkSession, exception: Exception): Seq[Row] = {
-    if (dmClassName.equalsIgnoreCase(PREAGGREGATE.toString) ||
-      dmClassName.equalsIgnoreCase(TIMESERIES.toString)) {
-      createPreAggregateTableCommands.undoMetadata(sparkSession, exception)
+    if (dataMapProvider != null) {
+      dataMapProvider.freeMeta(mainTable, dataMapSchema, sparkSession)
+      Seq.empty
     } else {
       throw new MalformedDataMapCommandException("Unknown data map type " + dmClassName)
     }

http://git-wip-us.apache.org/repos/asf/carbondata/blob/5a625f4c/integration/spark2/src/main/scala/org/apache/spark/sql/execution/command/datamap/CarbonDropDataMapCommand.scala
----------------------------------------------------------------------
diff --git a/integration/spark2/src/main/scala/org/apache/spark/sql/execution/command/datamap/CarbonDropDataMapCommand.scala b/integration/spark2/src/main/scala/org/apache/spark/sql/execution/command/datamap/CarbonDropDataMapCommand.scala
index f410f52..4cfc6b4 100644
--- a/integration/spark2/src/main/scala/org/apache/spark/sql/execution/command/datamap/CarbonDropDataMapCommand.scala
+++ b/integration/spark2/src/main/scala/org/apache/spark/sql/execution/command/datamap/CarbonDropDataMapCommand.scala
@@ -25,16 +25,15 @@ import org.apache.spark.sql.catalyst.TableIdentifier
 import org.apache.spark.sql.catalyst.analysis.NoSuchTableException
 import org.apache.spark.sql.execution.command.AtomicRunnableCommand
 import org.apache.spark.sql.execution.command.preaaggregate.PreAggregateUtil
-import org.apache.spark.sql.execution.command.table.CarbonDropTableCommand
 
 import org.apache.carbondata.common.exceptions.MetadataProcessException
 import org.apache.carbondata.common.exceptions.sql.{MalformedCarbonCommandException, NoSuchDataMapException}
 import org.apache.carbondata.common.logging.{LogService, LogServiceFactory}
-import org.apache.carbondata.core.datamap.DataMapStoreManager
 import org.apache.carbondata.core.locks.{CarbonLockUtil, ICarbonLock, LockUsage}
 import org.apache.carbondata.core.metadata.AbsoluteTableIdentifier
 import org.apache.carbondata.core.metadata.converter.ThriftWrapperSchemaConverterImpl
-import org.apache.carbondata.core.metadata.schema.table.CarbonTable
+import org.apache.carbondata.core.metadata.schema.table.{CarbonTable, DataMapSchema}
+import org.apache.carbondata.datamap.{DataMapManager, DataMapProvider}
 import org.apache.carbondata.events._
 
 /**
@@ -51,7 +50,9 @@ case class CarbonDropDataMapCommand(
     tableName: String)
   extends AtomicRunnableCommand {
 
-  var commandToRun: CarbonDropTableCommand = _
+  private var dataMapProvider: DataMapProvider = _
+  private var mainTable: CarbonTable = _
+  private var dataMapSchema: DataMapSchema = _
 
   override def processMetadata(sparkSession: SparkSession): Seq[Row] = {
     val LOGGER: LogService = LogServiceFactory.getLogService(this.getClass.getCanonicalName)
@@ -76,44 +77,42 @@ case class CarbonDropDataMapCommand(
           throw new MetadataProcessException(s"Dropping datamap $dataMapName failed", ex)
       }
       if (carbonTable.isDefined && carbonTable.get.getTableInfo.getDataMapSchemaList.size() > 0) {
-        val dataMapSchema = carbonTable.get.getTableInfo.getDataMapSchemaList.asScala.zipWithIndex.
+        mainTable = carbonTable.get
+        val dataMapSchemaOp = mainTable.getTableInfo.getDataMapSchemaList.asScala.zipWithIndex.
           find(_._1.getDataMapName.equalsIgnoreCase(dataMapName))
-        if (dataMapSchema.isDefined) {
+        if (dataMapSchemaOp.isDefined) {
+          dataMapSchema = dataMapSchemaOp.get._1
           val operationContext = new OperationContext
           val dropDataMapPreEvent =
             DropDataMapPreEvent(
-              Some(dataMapSchema.get._1),
+              Some(dataMapSchema),
               ifExistsSet,
               sparkSession)
           OperationListenerBus.getInstance.fireEvent(dropDataMapPreEvent, operationContext)
 
-          carbonTable.get.getTableInfo.getDataMapSchemaList.remove(dataMapSchema.get._2)
+          mainTable.getTableInfo.getDataMapSchemaList.remove(dataMapSchemaOp.get._2)
           val schemaConverter = new ThriftWrapperSchemaConverterImpl
           PreAggregateUtil.updateSchemaInfo(
-            carbonTable.get,
+            mainTable,
             schemaConverter.fromWrapperToExternalTableInfo(
-              carbonTable.get.getTableInfo,
+              mainTable.getTableInfo,
               dbName,
               tableName))(sparkSession)
-          commandToRun = CarbonDropTableCommand(
-            ifExistsSet = true,
-            Some(dataMapSchema.get._1.getRelationIdentifier.getDatabaseName),
-            dataMapSchema.get._1.getRelationIdentifier.getTableName,
-            dropChildTable = true
-          )
-          commandToRun.processMetadata(sparkSession)
+          dataMapProvider = DataMapManager.get.getDataMapProvider(dataMapSchema)
+          dataMapProvider.freeMeta(mainTable, dataMapSchema, sparkSession)
+
           // fires the event after dropping datamap from main table schema
           val dropDataMapPostEvent =
             DropDataMapPostEvent(
-              Some(dataMapSchema.get._1),
+              Some(dataMapSchema),
               ifExistsSet,
               sparkSession)
           OperationListenerBus.getInstance.fireEvent(dropDataMapPostEvent, operationContext)
         } else if (!ifExistsSet) {
           throw new NoSuchDataMapException(dataMapName, tableName)
         }
-      } else if ((carbonTable.isDefined &&
-        carbonTable.get.getTableInfo.getDataMapSchemaList.size() == 0)) {
+      } else if (carbonTable.isDefined &&
+                 carbonTable.get.getTableInfo.getDataMapSchemaList.size() == 0) {
         if (!ifExistsSet) {
           throw new NoSuchDataMapException(dataMapName, tableName)
         }
@@ -140,10 +139,8 @@ case class CarbonDropDataMapCommand(
 
   override def processData(sparkSession: SparkSession): Seq[Row] = {
     // delete the table folder
-    if (commandToRun != null) {
-      DataMapStoreManager.getInstance().clearDataMap(
-        commandToRun.carbonTable.getAbsoluteTableIdentifier, dataMapName)
-      commandToRun.processData(sparkSession)
+    if (dataMapProvider != null) {
+      dataMapProvider.freeData(mainTable, dataMapSchema, sparkSession)
     }
     Seq.empty
   }

http://git-wip-us.apache.org/repos/asf/carbondata/blob/5a625f4c/integration/spark2/src/main/scala/org/apache/spark/sql/execution/command/partition/CarbonAlterTableDropPartitionCommand.scala
----------------------------------------------------------------------
diff --git a/integration/spark2/src/main/scala/org/apache/spark/sql/execution/command/partition/CarbonAlterTableDropPartitionCommand.scala b/integration/spark2/src/main/scala/org/apache/spark/sql/execution/command/partition/CarbonAlterTableDropPartitionCommand.scala
index b53c609..ecf6f99 100644
--- a/integration/spark2/src/main/scala/org/apache/spark/sql/execution/command/partition/CarbonAlterTableDropPartitionCommand.scala
+++ b/integration/spark2/src/main/scala/org/apache/spark/sql/execution/command/partition/CarbonAlterTableDropPartitionCommand.scala
@@ -97,7 +97,7 @@ case class CarbonAlterTableDropPartitionCommand(
     partitionInfo.dropPartition(partitionIndex)
 
     // read TableInfo
-    val tableInfo = carbonMetaStore.getThriftTableInfo(carbonTable)(sparkSession)
+    val tableInfo = carbonMetaStore.getThriftTableInfo(carbonTable)
     val schemaConverter = new ThriftWrapperSchemaConverterImpl()
     val wrapperTableInfo = schemaConverter.fromExternalToWrapperTableInfo(tableInfo,
       dbName, tableName, tablePath)

http://git-wip-us.apache.org/repos/asf/carbondata/blob/5a625f4c/integration/spark2/src/main/scala/org/apache/spark/sql/execution/command/partition/CarbonAlterTableSplitPartitionCommand.scala
----------------------------------------------------------------------
diff --git a/integration/spark2/src/main/scala/org/apache/spark/sql/execution/command/partition/CarbonAlterTableSplitPartitionCommand.scala b/integration/spark2/src/main/scala/org/apache/spark/sql/execution/command/partition/CarbonAlterTableSplitPartitionCommand.scala
index 84779cc..732178c 100644
--- a/integration/spark2/src/main/scala/org/apache/spark/sql/execution/command/partition/CarbonAlterTableSplitPartitionCommand.scala
+++ b/integration/spark2/src/main/scala/org/apache/spark/sql/execution/command/partition/CarbonAlterTableSplitPartitionCommand.scala
@@ -89,7 +89,7 @@ case class CarbonAlterTableSplitPartitionCommand(
     updatePartitionInfo(partitionInfo, partitionIds)
 
     // read TableInfo
-    val tableInfo = carbonMetaStore.getThriftTableInfo(carbonTable)(sparkSession)
+    val tableInfo = carbonMetaStore.getThriftTableInfo(carbonTable)
     val schemaConverter = new ThriftWrapperSchemaConverterImpl()
     val wrapperTableInfo = schemaConverter.fromExternalToWrapperTableInfo(tableInfo,
       dbName, tableName, tablePath)

http://git-wip-us.apache.org/repos/asf/carbondata/blob/5a625f4c/integration/spark2/src/main/scala/org/apache/spark/sql/execution/command/preaaggregate/CreatePreAggregateTableCommand.scala
----------------------------------------------------------------------
diff --git a/integration/spark2/src/main/scala/org/apache/spark/sql/execution/command/preaaggregate/CreatePreAggregateTableCommand.scala b/integration/spark2/src/main/scala/org/apache/spark/sql/execution/command/preaaggregate/CreatePreAggregateTableCommand.scala
deleted file mode 100644
index 6d4822b..0000000
--- a/integration/spark2/src/main/scala/org/apache/spark/sql/execution/command/preaaggregate/CreatePreAggregateTableCommand.scala
+++ /dev/null
@@ -1,218 +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.spark.sql.execution.command.preaaggregate
-
-import scala.collection.JavaConverters._
-import scala.collection.mutable
-
-import org.apache.spark.sql._
-import org.apache.spark.sql.catalyst.TableIdentifier
-import org.apache.spark.sql.execution.command._
-import org.apache.spark.sql.execution.command.datamap.CarbonDropDataMapCommand
-import org.apache.spark.sql.execution.command.management.CarbonLoadDataCommand
-import org.apache.spark.sql.execution.command.table.CarbonCreateTableCommand
-import org.apache.spark.sql.execution.command.timeseries.TimeSeriesUtil
-import org.apache.spark.sql.parser.CarbonSpark2SqlParser
-
-import org.apache.carbondata.common.exceptions.sql.MalformedDataMapCommandException
-import org.apache.carbondata.core.constants.CarbonCommonConstants
-import org.apache.carbondata.core.metadata.schema.table.AggregationDataMapSchema
-import org.apache.carbondata.core.metadata.schema.table.CarbonTable
-import org.apache.carbondata.core.statusmanager.{SegmentStatus, SegmentStatusManager}
-
-/**
- * Below command class will be used to create pre-aggregate table
- * and updating the parent table about the child table information
- * It will be either success or nothing happen in case of failure:
- * 1. failed to create pre aggregate table.
- * 2. failed to update main table
- *
- */
-case class CreatePreAggregateTableCommand(
-    dataMapName: String,
-    parentTableIdentifier: TableIdentifier,
-    dmClassName: String,
-    dmProperties: Map[String, String],
-    queryString: String,
-    timeSeriesFunction: Option[String] = None)
-  extends AtomicRunnableCommand {
-
-  var parentTable: CarbonTable = _
-  var loadCommand: CarbonLoadDataCommand = _
-
-  override def processMetadata(sparkSession: SparkSession): Seq[Row] = {
-    val updatedQuery = new CarbonSpark2SqlParser().addPreAggFunction(queryString)
-    val df = sparkSession.sql(updatedQuery)
-    val fieldRelationMap = PreAggregateUtil.validateActualSelectPlanAndGetAttributes(
-      df.logicalPlan, queryString)
-    val fields = fieldRelationMap.keySet.toSeq
-    val tableProperties = mutable.Map[String, String]()
-    dmProperties.foreach(t => tableProperties.put(t._1, t._2))
-
-    parentTable = PreAggregateUtil.getParentCarbonTable(df.logicalPlan)
-    if (!parentTable.getTableName.equalsIgnoreCase(parentTableIdentifier.table)) {
-      throw new MalformedDataMapCommandException(
-        "Parent table name is different in select and create")
-    }
-    var neworder = Seq[String]()
-    val parentOrder = parentTable.getSortColumns(parentTable.getTableName).asScala
-    parentOrder.foreach(parentcol =>
-      fields.filter(col => (fieldRelationMap.get(col).get.aggregateFunction.isEmpty) &&
-                           (parentcol.equals(fieldRelationMap.get(col).get.
-                             columnTableRelationList.get(0).parentColumnName)))
-        .map(cols => neworder :+= cols.column)
-    )
-    tableProperties.put(CarbonCommonConstants.SORT_COLUMNS, neworder.mkString(","))
-    tableProperties.put("sort_scope", parentTable.getTableInfo.getFactTable.
-      getTableProperties.getOrDefault("sort_scope", CarbonCommonConstants
-      .LOAD_SORT_SCOPE_DEFAULT))
-    tableProperties
-      .put(CarbonCommonConstants.TABLE_BLOCKSIZE, parentTable.getBlockSizeInMB.toString)
-    val tableIdentifier =
-      TableIdentifier(parentTableIdentifier.table + "_" + dataMapName,
-        parentTableIdentifier.database)
-    // prepare table model of the collected tokens
-    val tableModel: TableModel = new CarbonSpark2SqlParser().prepareTableModel(
-      ifNotExistPresent = false,
-      new CarbonSpark2SqlParser().convertDbNameToLowerCase(tableIdentifier.database),
-      tableIdentifier.table.toLowerCase,
-      fields,
-      Seq(),
-      tableProperties,
-      None,
-      isAlterFlow = false,
-      None)
-
-
-    // updating the relation identifier, this will be stored in child table
-    // which can be used during dropping of pre-aggreate table as parent table will
-    // also get updated
-    if(timeSeriesFunction.isDefined) {
-      TimeSeriesUtil.validateTimeSeriesEventTime(dmProperties, parentTable)
-      TimeSeriesUtil.validateEventTimeColumnExitsInSelect(
-        fieldRelationMap,
-        dmProperties.get(TimeSeriesUtil.TIMESERIES_EVENTTIME).get)
-      TimeSeriesUtil.updateTimeColumnSelect(fieldRelationMap,
-        dmProperties.get(TimeSeriesUtil.TIMESERIES_EVENTTIME).get,
-      timeSeriesFunction.get)
-    }
-    tableModel.parentTable = Some(parentTable)
-    tableModel.dataMapRelation = Some(fieldRelationMap)
-    val tablePath = if (dmProperties.contains("path")) {
-      dmProperties("path")
-    } else {
-      CarbonEnv.getTablePath(tableModel.databaseNameOp, tableModel.tableName)(sparkSession)
-    }
-    CarbonCreateTableCommand(TableNewProcessor(tableModel),
-      tableModel.ifNotExistsSet, Some(tablePath)).run(sparkSession)
-
-    val table = CarbonEnv.getCarbonTable(tableIdentifier)(sparkSession)
-    val tableInfo = table.getTableInfo
-    // child schema object which will be updated on parent table about the
-    val childSchema = tableInfo.getFactTable.buildChildSchema(
-      dataMapName,
-      CarbonCommonConstants.AGGREGATIONDATAMAPSCHEMA,
-      tableInfo.getDatabaseName,
-      queryString,
-      "AGGREGATION")
-    dmProperties.foreach(f => childSchema.getProperties.put(f._1, f._2))
-
-    // updating the parent table about child table
-    PreAggregateUtil.updateMainTable(
-      parentTable,
-      childSchema,
-      sparkSession)
-    // After updating the parent carbon table with data map entry extract the latest table object
-    // to be used in further create process.
-    parentTable = CarbonEnv.getCarbonTable(parentTableIdentifier.database,
-      parentTableIdentifier.table)(sparkSession)
-    val updatedLoadQuery = if (timeSeriesFunction.isDefined) {
-      val dataMap = parentTable.getTableInfo.getDataMapSchemaList.asScala
-        .filter(p => p.getDataMapName
-          .equalsIgnoreCase(dataMapName)).head
-        .asInstanceOf[AggregationDataMapSchema]
-      PreAggregateUtil.createTimeSeriesSelectQueryFromMain(dataMap.getChildSchema,
-        parentTable.getTableName,
-        parentTable.getDatabaseName)
-    } else {
-      queryString
-    }
-    val dataFrame = sparkSession.sql(new CarbonSpark2SqlParser().addPreAggLoadFunction(
-      updatedLoadQuery)).drop("preAggLoad")
-    val dataMap = parentTable.getTableInfo.getDataMapSchemaList.asScala
-      .filter(dataMap => dataMap.getDataMapName.equalsIgnoreCase(dataMapName)).head
-      .asInstanceOf[AggregationDataMapSchema]
-    loadCommand = PreAggregateUtil.createLoadCommandForChild(
-      dataMap.getChildSchema.getListOfColumns,
-      tableIdentifier,
-      dataFrame,
-      false,
-      sparkSession = sparkSession)
-    loadCommand.processMetadata(sparkSession)
-    Seq.empty
-  }
-
-  override def undoMetadata(sparkSession: SparkSession, exception: Exception): Seq[Row] = {
-    // drop child table and undo the change in table info of main table
-    CarbonDropDataMapCommand(
-      dataMapName,
-      ifExistsSet = true,
-      parentTableIdentifier.database,
-      parentTableIdentifier.table).run(sparkSession)
-    Seq.empty
-  }
-
-  override def processData(sparkSession: SparkSession): Seq[Row] = {
-    // load child table if parent table has existing segments
-    // This will be used to check if the parent table has any segments or not. If not then no
-    // need to fire load for pre-aggregate table. Therefore reading the load details for PARENT
-    // table.
-    SegmentStatusManager.deleteLoadsAndUpdateMetadata(parentTable, false)
-    val loadAvailable = SegmentStatusManager.readLoadMetadata(parentTable.getMetadataPath)
-    if (loadAvailable.exists(load => load.getSegmentStatus == SegmentStatus.INSERT_IN_PROGRESS ||
-      load.getSegmentStatus == SegmentStatus.INSERT_OVERWRITE_IN_PROGRESS)) {
-      throw new UnsupportedOperationException(
-        "Cannot create pre-aggregate table when insert is in progress on main table")
-    } else if (loadAvailable.nonEmpty) {
-      val updatedQuery = if (timeSeriesFunction.isDefined) {
-        val dataMap = parentTable.getTableInfo.getDataMapSchemaList.asScala
-          .filter(p => p.getDataMapName
-            .equalsIgnoreCase(dataMapName)).head
-          .asInstanceOf[AggregationDataMapSchema]
-        PreAggregateUtil.createTimeSeriesSelectQueryFromMain(dataMap.getChildSchema,
-          parentTable.getTableName,
-          parentTable.getDatabaseName)
-      } else {
-        queryString
-      }
-      // Passing segmentToLoad as * because we want to load all the segments into the
-      // pre-aggregate table even if the user has set some segments on the parent table.
-      loadCommand.dataFrame = Some(PreAggregateUtil
-        .getDataFrame(sparkSession, loadCommand.logicalPlan.get))
-      PreAggregateUtil.startDataLoadForDataMap(
-        parentTable,
-        segmentToLoad = "*",
-        validateSegments = true,
-        sparkSession,
-        loadCommand)
-    }
-    Seq.empty
-  }
-}
-
-


[6/6] carbondata git commit: [CARBONDATA-2189] Add DataMapProvider developer interface

Posted by ja...@apache.org.
[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/5a625f4c
Tree: http://git-wip-us.apache.org/repos/asf/carbondata/tree/5a625f4c
Diff: http://git-wip-us.apache.org/repos/asf/carbondata/diff/5a625f4c

Branch: refs/heads/datamap
Commit: 5a625f4ce53225de50aae53057a03885f3b1d3a7
Parents: e616162
Author: Jacky Li <ja...@qq.com>
Authored: Thu Feb 22 20:59:59 2018 +0800
Committer: Jacky Li <ja...@qq.com>
Committed: Sat Feb 24 10:16:42 2018 +0800

----------------------------------------------------------------------
 .../sql/MalformedDataMapCommandException.java   |   4 +
 .../core/constants/CarbonCommonConstants.java   |   1 -
 .../carbondata/core/datamap/DataMapChooser.java |   5 +-
 .../core/datamap/DataMapRegistry.java           |  37 +
 .../core/datamap/DataMapStoreManager.java       |  83 +-
 .../carbondata/core/datamap/TableDataMap.java   |  46 +-
 .../core/datamap/dev/AbstractDataMapWriter.java |   2 +-
 .../carbondata/core/datamap/dev/DataMap.java    |  61 --
 .../core/datamap/dev/DataMapFactory.java        |  85 --
 .../core/datamap/dev/IndexDataMap.java          |  61 ++
 .../core/datamap/dev/IndexDataMapFactory.java   |  85 ++
 .../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          | 981 -------------------
 .../blockletindex/BlockletDataMapFactory.java   | 256 -----
 .../blockletindex/BlockletDataMapModel.java     |   2 +-
 .../blockletindex/BlockletIndexDataMap.java     | 981 +++++++++++++++++++
 .../BlockletIndexDataMapFactory.java            | 256 +++++
 .../core/metadata/schema/table/CarbonTable.java |  10 +-
 .../metadata/schema/table/DataMapSchema.java    |   1 +
 .../schema/table/DataMapSchemaFactory.java      |  17 +-
 .../core/metadata/schema/table/TableSchema.java |   3 +-
 .../blockletindex/TestBlockletDataMap.java      |  66 --
 .../blockletindex/TestBlockletIndexDataMap.java |  59 ++
 .../datamap/examples/MinMaxDataMap.java         | 151 ---
 .../datamap/examples/MinMaxDataMapFactory.java  | 116 ---
 .../datamap/examples/MinMaxIndexDataMap.java    | 148 +++
 .../examples/MinMaxIndexDataMapFactory.java     | 116 +++
 .../MinMaxDataMapExample.scala                  |   4 +-
 docs/datamap-developer-guide.md                 |  16 +
 .../hadoop/api/CarbonTableInputFormat.java      |   4 +-
 .../preaggregate/TestPreAggCreateCommand.scala  |   4 +-
 .../TestPreAggregateTableSelection.scala        |   4 +-
 .../timeseries/TestTimeSeriesCreateTable.scala  |   4 +-
 .../testsuite/datamap/CGDataMapTestCase.scala   | 379 -------
 .../datamap/CGIndexDataMapTestCase.scala        | 379 +++++++
 .../testsuite/datamap/DataMapWriterSuite.scala  | 216 ----
 .../testsuite/datamap/FGDataMapTestCase.scala   | 476 ---------
 .../datamap/FGIndexDataMapTestCase.scala        | 474 +++++++++
 .../datamap/IndexDataMapWriterSuite.scala       | 216 ++++
 .../testsuite/datamap/TestDataMapCommand.scala  | 282 ------
 .../datamap/TestIndexDataMapCommand.scala       | 279 ++++++
 .../iud/InsertOverwriteConcurrentTest.scala     |  19 +-
 .../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      |  49 +
 .../scala/org/apache/spark/sql/CarbonEnv.scala  |   2 +-
 .../datamap/CarbonCreateDataMapCommand.scala    |  82 +-
 .../datamap/CarbonDropDataMapCommand.scala      |  45 +-
 .../CarbonAlterTableDropPartitionCommand.scala  |   2 +-
 .../CarbonAlterTableSplitPartitionCommand.scala |   2 +-
 .../CreatePreAggregateTableCommand.scala        | 218 -----
 .../preaaggregate/PreAggregateTableHelper.scala | 202 ++++
 .../preaaggregate/PreAggregateUtil.scala        |  21 +-
 .../CarbonAlterTableAddColumnCommand.scala      |   2 +-
 .../CarbonAlterTableDataTypeChangeCommand.scala |   2 +-
 .../CarbonAlterTableDropColumnCommand.scala     |   2 +-
 .../schema/CarbonAlterTableRenameCommand.scala  |   2 +-
 .../command/timeseries/TimeSeriesUtil.scala     |  12 +-
 .../spark/sql/hive/CarbonFileMetastore.scala    |   3 +-
 .../spark/sql/hive/CarbonHiveMetaStore.scala    |   3 +-
 .../apache/spark/sql/hive/CarbonMetaStore.scala |   2 +-
 .../org/apache/spark/util/AlterTableUtil.scala  |  10 +-
 .../datamap/DataMapWriterListener.java          |   8 +-
 75 files changed, 4087 insertions(+), 3644 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/carbondata/blob/5a625f4c/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/5a625f4c/core/src/main/java/org/apache/carbondata/core/constants/CarbonCommonConstants.java
----------------------------------------------------------------------
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 4d838a8..f7c4d2c 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
@@ -1507,7 +1507,6 @@ public final class CarbonCommonConstants {
 
   public static final String CARBON_MERGE_INDEX_IN_SEGMENT_DEFAULT = "true";
 
-  public static final String AGGREGATIONDATAMAPSCHEMA = "AggregateDataMapHandler";
   /*
    * The total size of carbon data
    */

http://git-wip-us.apache.org/repos/asf/carbondata/blob/5a625f4c/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/5a625f4c/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/5a625f4c/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 ca5da1e..ab31393 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,12 +26,12 @@ 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;
 import org.apache.carbondata.core.metadata.schema.table.DataMapSchema;
 import org.apache.carbondata.core.mutate.SegmentUpdateDetails;
@@ -68,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);
         }
       }
@@ -86,8 +86,8 @@ public final class DataMapStoreManager {
     List<TableDataMap> dataMaps = new ArrayList<>();
     if (dataMapSchemaList != null) {
       for (DataMapSchema dataMapSchema : dataMapSchemaList) {
-        if (!dataMapSchema.getClassName()
-            .equalsIgnoreCase(CarbonCommonConstants.AGGREGATIONDATAMAPSCHEMA)) {
+        if (!dataMapSchema.getClassName().equalsIgnoreCase(
+            DataMapProvider.PREAGGREGATE.toString())) {
           dataMaps.add(getDataMap(carbonTable.getAbsoluteTableIdentifier(), dataMapSchema));
         }
       }
@@ -96,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);
   }
 
   /**
@@ -141,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);
@@ -151,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;
@@ -261,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/5a625f4c/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 d4a8a22..7f0f3f2 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;
@@ -39,7 +39,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 {
@@ -48,7 +48,7 @@ public final class TableDataMap extends OperationEventListener {
 
   private DataMapSchema dataMapSchema;
 
-  private DataMapFactory dataMapFactory;
+  private IndexDataMapFactory indexDataMapFactory;
 
   private BlockletDetailsFetcher blockletDetailsFetcher;
 
@@ -58,11 +58,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;
   }
@@ -84,10 +84,10 @@ public final class TableDataMap extends OperationEventListener {
       if (filterExp == null) {
         pruneBlocklets = blockletDetailsFetcher.getAllBlocklets(segmentId, partitions);
       } else {
-        List<DataMap> dataMaps = dataMapFactory.getDataMaps(segmentId);
+        List<IndexDataMap> indexDataMaps = indexDataMapFactory.getDataMaps(segmentId);
         segmentProperties = segmentPropertiesFetcher.getSegmentProperties(segmentId);
-        for (DataMap dataMap : dataMaps) {
-          pruneBlocklets.addAll(dataMap.prune(filterExp, segmentProperties, partitions));
+        for (IndexDataMap indexDataMap : indexDataMaps) {
+          pruneBlocklets.addAll(indexDataMap.prune(filterExp, segmentProperties, partitions));
         }
       }
       blocklets.addAll(addSegmentId(blockletDetailsFetcher
@@ -114,7 +114,7 @@ public final class TableDataMap extends OperationEventListener {
   public List<DataMapDistributable> toDistributable(List<String> segmentIds) throws IOException {
     List<DataMapDistributable> distributables = new ArrayList<>();
     for (String segmentsId : segmentIds) {
-      List<DataMapDistributable> list = dataMapFactory.toDistributable(segmentsId);
+      List<DataMapDistributable> list = indexDataMapFactory.toDistributable(segmentsId);
       for (DataMapDistributable distributable: list) {
         distributable.setDataMapSchema(dataMapSchema);
         distributable.setSegmentId(segmentsId);
@@ -137,10 +137,10 @@ public final class TableDataMap extends OperationEventListener {
       FilterResolverIntf filterExp, List<String> 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.getSegmentId()),
               partitions));
@@ -149,13 +149,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.getSegmentId());
-      if (dataMapFactory.getDataMapType() == DataMapType.FG) {
+      if (indexDataMapFactory.getDataMapType() == DataMapType.FG) {
         String blockletwritePath =
             writePath + CarbonCommonConstants.FILE_SEPARATOR + System.nanoTime();
         detailedBlocklet.setDataMapWriterPath(blockletwritePath);
@@ -173,7 +173,7 @@ public final class TableDataMap extends OperationEventListener {
    */
   public void clear(List<String> segmentIds) {
     for (String segmentId: segmentIds) {
-      dataMapFactory.clear(segmentId);
+      indexDataMapFactory.clear(segmentId);
     }
   }
 
@@ -181,19 +181,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);
   }
 
   /**
@@ -208,9 +208,9 @@ public final class TableDataMap extends OperationEventListener {
       throws IOException {
     List<String> prunedSegments = new ArrayList<>(CarbonCommonConstants.DEFAULT_COLLECTION_SIZE);
     for (String segmentId : segmentIds) {
-      List<DataMap> dataMaps = dataMapFactory.getDataMaps(segmentId);
-      for (DataMap dataMap : dataMaps) {
-        if (dataMap.isScanRequired(filterExp)) {
+      List<IndexDataMap> indexDataMaps = indexDataMapFactory.getDataMaps(segmentId);
+      for (IndexDataMap indexDataMap : indexDataMaps) {
+        if (indexDataMap.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(segmentId);

http://git-wip-us.apache.org/repos/asf/carbondata/blob/5a625f4c/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 bcc9bad..1af7dde 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
@@ -70,7 +70,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/5a625f4c/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 3fa6d75..0000000
--- a/core/src/main/java/org/apache/carbondata/core/datamap/dev/DataMap.java
+++ /dev/null
@@ -1,61 +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.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.
-   *
-   * @param filterExp
-   * @return
-   */
-  List<T> prune(FilterResolverIntf filterExp, SegmentProperties segmentProperties,
-      List<String> partitions);
-
-  // TODO Move this method to Abstract class
-  /**
-   * Validate whether the current segment needs to be fetching the required data
-   *
-   * @param filterExp
-   * @return
-   */
-  boolean isScanRequired(FilterResolverIntf filterExp);
-
-  /**
-   * Clear complete index table and release memory.
-   */
-  void clear();
-
-}

http://git-wip-us.apache.org/repos/asf/carbondata/blob/5a625f4c/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 0aebe9b..0000000
--- a/core/src/main/java/org/apache/carbondata/core/datamap/dev/DataMapFactory.java
+++ /dev/null
@@ -1,85 +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.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(String segmentId, String writeDirectoryPath);
-
-  /**
-   * Get the datamap for segmentid
-   */
-  List<T> getDataMaps(String segmentId) 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(String segmentId);
-
-  /**
-   *
-   * @param event
-   */
-  void fireEvent(Event event);
-
-  /**
-   * Clears datamap of the segment
-   */
-  void clear(String segmentId);
-
-  /**
-   * 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/5a625f4c/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..f86bc0a
--- /dev/null
+++ b/core/src/main/java/org/apache/carbondata/core/datamap/dev/IndexDataMap.java
@@ -0,0 +1,61 @@
+/*
+ * 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.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.
+   *
+   * @param filterExp
+   * @return
+   */
+  List<T> prune(FilterResolverIntf filterExp, SegmentProperties segmentProperties,
+      List<String> partitions);
+
+  // TODO Move this method to Abstract class
+  /**
+   * Validate whether the current segment needs to be fetching the required data
+   *
+   * @param filterExp
+   * @return
+   */
+  boolean isScanRequired(FilterResolverIntf filterExp);
+
+  /**
+   * Clear complete index table and release memory.
+   */
+  void clear();
+
+}

http://git-wip-us.apache.org/repos/asf/carbondata/blob/5a625f4c/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..0bab104
--- /dev/null
+++ b/core/src/main/java/org/apache/carbondata/core/datamap/dev/IndexDataMapFactory.java
@@ -0,0 +1,85 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.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.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(String segmentId, String writeDirectoryPath);
+
+  /**
+   * Get the datamap for segmentid
+   */
+  List<T> getDataMaps(String segmentId) 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(String segmentId);
+
+  /**
+   *
+   * @param event
+   */
+  void fireEvent(Event event);
+
+  /**
+   * Clears datamap of the segment
+   */
+  void clear(String segmentId);
+
+  /**
+   * 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/5a625f4c/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/5a625f4c/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/5a625f4c/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/5a625f4c/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/5a625f4c/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 a66d31b..695f653 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
@@ -81,6 +81,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/5a625f4c/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/5a625f4c/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/5a625f4c/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/5a625f4c/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/5a625f4c/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 7598961..c175378 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
@@ -29,8 +29,8 @@ import org.apache.carbondata.core.cache.Cache;
 import org.apache.carbondata.core.cache.CarbonLRUCache;
 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.PartitionMapFileStore;
@@ -41,7 +41,7 @@ import org.apache.carbondata.core.util.path.CarbonTablePath;
  * blocks
  */
 public class BlockletDataMapIndexStore
-    implements Cache<TableBlockIndexUniqueIdentifier, BlockletDataMap> {
+    implements Cache<TableBlockIndexUniqueIdentifier, BlockletIndexDataMap> {
   private static final LogService LOGGER =
       LogServiceFactory.getLogService(BlockletDataMapIndexStore.class.getName());
   /**
@@ -68,10 +68,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 {
         String segmentPath = CarbonTablePath.getSegmentPath(
@@ -99,14 +99,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 {
@@ -145,7 +146,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);
@@ -160,9 +161,9 @@ public class BlockletDataMapIndexStore
    * @return
    */
   @Override
-  public BlockletDataMap getIfPresent(
+  public BlockletIndexDataMap getIfPresent(
       TableBlockIndexUniqueIdentifier tableSegmentUniqueIdentifier) {
-    return (BlockletDataMap) lruCache.get(
+    return (BlockletIndexDataMap) lruCache.get(
         tableSegmentUniqueIdentifier.getUniqueTableSegmentIdentifier());
   }
 
@@ -185,7 +186,7 @@ public class BlockletDataMapIndexStore
    * @return map of taks id to segment mapping
    * @throws IOException
    */
-  private BlockletDataMap loadAndGetDataMap(
+  private BlockletIndexDataMap loadAndGetDataMap(
       TableBlockIndexUniqueIdentifier identifier,
       SegmentIndexFileStore indexFileStore,
       PartitionMapFileStore partitionFileStore,
@@ -197,9 +198,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.getFilePath(),
           indexFileStore.getFileData(identifier.getCarbonIndexFileName()),
           partitionFileStore.getPartitions(identifier.getCarbonIndexFileName()),
@@ -235,9 +236,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();
     }
   }