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/03/04 14:57:43 UTC

[01/20] carbondata git commit: [CARBONDATA-1543] Supported DataMap chooser and expression for supporting multiple datamaps in single query

Repository: carbondata
Updated Branches:
  refs/heads/datamap-rebase1 [created] f9139930e


http://git-wip-us.apache.org/repos/asf/carbondata/blob/c7a9f15e/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 ccb9b68..b1962c1 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
@@ -19,15 +19,15 @@ package org.apache.carbondata.integration.spark.testsuite.preaggregate
 
 import scala.collection.JavaConverters._
 
-import org.apache.spark.sql.{AnalysisException, CarbonDatasourceHadoopRelation, Row}
+import org.apache.spark.sql.{CarbonDatasourceHadoopRelation, Row}
 import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan
 import org.apache.spark.sql.execution.datasources.LogicalRelation
 import org.apache.spark.sql.hive.CarbonRelation
 import org.apache.spark.sql.test.util.QueryTest
 import org.scalatest.BeforeAndAfterAll
 
-import org.apache.carbondata.common.exceptions.sql.MalformedCarbonCommandException
 import org.apache.carbondata.core.constants.CarbonCommonConstants
+import org.apache.carbondata.common.exceptions.sql.{MalformedCarbonCommandException, MalformedDataMapCommandException}
 import org.apache.carbondata.core.metadata.encoder.Encoding
 import org.apache.carbondata.core.metadata.schema.table.CarbonTable
 import org.apache.carbondata.core.metadata.schema.datamap.DataMapProvider.TIMESERIES
@@ -293,7 +293,7 @@ class TestPreAggCreateCommand extends QueryTest with BeforeAndAfterAll {
   test("test pre agg create table 22: using invalid datamap provider") {
     sql("DROP DATAMAP IF EXISTS agg0 ON TABLE maintable")
 
-    val e: Exception = intercept[MalformedDataMapCommandException] {
+    val e = intercept[MalformedDataMapCommandException] {
       sql(
         """
           | CREATE DATAMAP agg0 ON TABLE mainTable

http://git-wip-us.apache.org/repos/asf/carbondata/blob/c7a9f15e/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 43316b3..ec76b37 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
@@ -201,7 +201,7 @@ class TestTimeSeriesCreateTable extends QueryTest with BeforeAndAfterAll {
           | GROUP BY dataTime
         """.stripMargin)
     }
-    assert(e.getMessage.equals("Unknown datamap provider/class abc"))
+    assert(e.getMessage.equals("DataMap class 'abc' not found"))
   }
 
   test("test timeseries create table 12: USING and catch MalformedCarbonCommandException") {
@@ -216,7 +216,7 @@ class TestTimeSeriesCreateTable extends QueryTest with BeforeAndAfterAll {
           | GROUP BY dataTime
         """.stripMargin)
     }
-    assert(e.getMessage.equals("Unknown datamap provider/class abc"))
+    assert(e.getMessage.equals("DataMap class 'abc' not found"))
   }
 
   test("test timeseries create table 13: Only one granularity level can be defined 1") {
@@ -237,6 +237,7 @@ class TestTimeSeriesCreateTable extends QueryTest with BeforeAndAfterAll {
            | GROUP BY dataTime
        """.stripMargin)
     }
+    e.printStackTrace()
     assert(e.getMessage.equals("Only one granularity level can be defined"))
   }
 

http://git-wip-us.apache.org/repos/asf/carbondata/blob/c7a9f15e/integration/spark-common-test/src/test/scala/org/apache/carbondata/spark/testsuite/datacompaction/CompactionSupportGlobalSortBigFileTest.scala
----------------------------------------------------------------------
diff --git a/integration/spark-common-test/src/test/scala/org/apache/carbondata/spark/testsuite/datacompaction/CompactionSupportGlobalSortBigFileTest.scala b/integration/spark-common-test/src/test/scala/org/apache/carbondata/spark/testsuite/datacompaction/CompactionSupportGlobalSortBigFileTest.scala
index c522c1e..e31896f 100644
--- a/integration/spark-common-test/src/test/scala/org/apache/carbondata/spark/testsuite/datacompaction/CompactionSupportGlobalSortBigFileTest.scala
+++ b/integration/spark-common-test/src/test/scala/org/apache/carbondata/spark/testsuite/datacompaction/CompactionSupportGlobalSortBigFileTest.scala
@@ -113,7 +113,7 @@ object CompactionSupportGlobalSortBigFileTest {
     try {
       val write = new PrintWriter(fileName);
       for (i <- start until (start + line)) {
-        write.println(i + "," + "n" + i + "," + "c" + Random.nextInt(line) + "," + Random.nextInt(80))
+        write.println(i + "," + "n" + i + "," + "c" + (i % 10000) + "," + Random.nextInt(80))
       }
       write.close()
     } catch {

http://git-wip-us.apache.org/repos/asf/carbondata/blob/c7a9f15e/integration/spark-common-test/src/test/scala/org/apache/carbondata/spark/testsuite/dataload/TestLoadDataFrame.scala
----------------------------------------------------------------------
diff --git a/integration/spark-common-test/src/test/scala/org/apache/carbondata/spark/testsuite/dataload/TestLoadDataFrame.scala b/integration/spark-common-test/src/test/scala/org/apache/carbondata/spark/testsuite/dataload/TestLoadDataFrame.scala
index 693c145..a214444 100644
--- a/integration/spark-common-test/src/test/scala/org/apache/carbondata/spark/testsuite/dataload/TestLoadDataFrame.scala
+++ b/integration/spark-common-test/src/test/scala/org/apache/carbondata/spark/testsuite/dataload/TestLoadDataFrame.scala
@@ -92,18 +92,18 @@ class TestLoadDataFrame extends QueryTest with BeforeAndAfterAll {
     buildTestData
   }
 
-test("test the boolean data type"){
-  booldf.write
-    .format("carbondata")
-    .option("tableName", "carbon10")
-    .option("tempCSV", "true")
-    .option("compress", "true")
-    .mode(SaveMode.Overwrite)
-    .save()
-  checkAnswer(
-    sql("SELECT * FROM CARBON10"),
-    Seq(Row("anubhav", true), Row("prince", false)))
-}
+  test("test the boolean data type"){
+    booldf.write
+      .format("carbondata")
+      .option("tableName", "carbon0")
+      .option("tempCSV", "true")
+      .option("compress", "true")
+      .mode(SaveMode.Overwrite)
+      .save()
+    checkAnswer(
+      sql("SELECT * FROM CARBON0"),
+      Seq(Row("anubhav", true), Row("prince", false)))
+  }
 
   test("test load dataframe with saving compressed csv files") {
     // save dataframe to carbon file

http://git-wip-us.apache.org/repos/asf/carbondata/blob/c7a9f15e/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
index 1cbbcb4..041a63a 100644
--- 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
@@ -36,6 +36,7 @@ import org.apache.carbondata.core.datastore.impl.FileFactory
 import org.apache.carbondata.core.datastore.page.ColumnPage
 import org.apache.carbondata.core.indexstore.{Blocklet, PartitionSpec}
 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
@@ -48,22 +49,21 @@ import org.apache.carbondata.spark.testsuite.datacompaction.CompactionSupportGlo
 
 class CGDataMapFactory extends AbstractCoarseGrainDataMapFactory {
   var identifier: AbsoluteTableIdentifier = _
-  var dataMapName: String = _
+  var dataMapSchema: DataMapSchema = _
 
   /**
    * Initialization of Datamap factory with the identifier and datamap name
    */
-  override def init(identifier: AbsoluteTableIdentifier,
-      dataMapName: String): Unit = {
+  override def init(identifier: AbsoluteTableIdentifier, dataMapSchema: DataMapSchema): Unit = {
     this.identifier = identifier
-    this.dataMapName = dataMapName
+    this.dataMapSchema = dataMapSchema
   }
 
   /**
    * Return a new write for this datamap
    */
   override def createWriter(segment: Segment, dataWritePath: String): AbstractDataMapWriter = {
-    new CGDataMapWriter(identifier, segment, dataWritePath, dataMapName)
+    new CGDataMapWriter(identifier, segment, dataWritePath, dataMapSchema)
   }
 
   /**
@@ -140,7 +140,8 @@ class CGDataMapFactory extends AbstractCoarseGrainDataMapFactory {
    * Return metadata of this datamap
    */
   override def getMeta: DataMapMeta = {
-    new DataMapMeta(Seq("name").toList.asJava, new ArrayBuffer[ExpressionType]().toList.asJava)
+    new DataMapMeta(dataMapSchema.getProperties.get("indexcolumns").split(",").toList.asJava,
+      List(ExpressionType.EQUALS, ExpressionType.IN).asJava)
   }
 }
 
@@ -198,12 +199,16 @@ class CGDataMap extends AbstractCoarseGrainDataMap {
   }
 
   private def getEqualToExpression(expression: Expression, buffer: ArrayBuffer[Expression]): Unit = {
-    if (expression.getChildren != null) {
-      expression.getChildren.asScala.map { f =>
-        if (f.isInstanceOf[EqualToExpression]) {
-          buffer += f
+    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)
         }
-        getEqualToExpression(f, buffer)
       }
     }
   }
@@ -221,12 +226,12 @@ class CGDataMap extends AbstractCoarseGrainDataMap {
 class CGDataMapWriter(identifier: AbsoluteTableIdentifier,
     segment: Segment,
     dataWritePath: String,
-    dataMapName: String)
+    dataMapSchema: DataMapSchema)
   extends AbstractDataMapWriter(identifier, segment, dataWritePath) {
 
   var currentBlockId: String = null
   val cgwritepath = dataWritePath + "/" +
-                    dataMapName + System.nanoTime() + ".datamap"
+                    dataMapSchema.getDataMapName + System.nanoTime() + ".datamap"
   lazy val stream: DataOutputStream = FileFactory
     .getDataOutputStream(cgwritepath, FileFactory.getFileType(cgwritepath))
   val blockletList = new ArrayBuffer[Array[Byte]]()
@@ -345,14 +350,29 @@ class CGDataMapTestCase extends QueryTest with BeforeAndAfterAll {
       """.stripMargin)
     val table = CarbonMetadata.getInstance().getCarbonTable("default_datamap_test_cg")
     // register datamap writer
-    DataMapStoreManager.getInstance().createAndRegisterDataMap(
-      table.getAbsoluteTableIdentifier,
-      classOf[CGDataMapFactory].getName, "cgdatamap")
+    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")

http://git-wip-us.apache.org/repos/asf/carbondata/blob/c7a9f15e/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
index 7e93959..00d13a9 100644
--- 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
@@ -32,6 +32,7 @@ import org.apache.carbondata.core.datamap.dev.cgdatamap.{AbstractCoarseGrainData
 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.util.CarbonProperties
 import org.apache.carbondata.events.Event
@@ -41,7 +42,7 @@ class C2DataMapFactory() extends AbstractCoarseGrainDataMapFactory {
   var identifier: AbsoluteTableIdentifier = _
 
   override def init(identifier: AbsoluteTableIdentifier,
-      dataMapName: String): Unit = {
+      dataMapSchema: DataMapSchema): Unit = {
     this.identifier = identifier
   }
 
@@ -89,12 +90,9 @@ class DataMapWriterSuite extends QueryTest with BeforeAndAfterAll {
   }
 
   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
-    DataMapStoreManager.getInstance().createAndRegisterDataMap(
-      AbsoluteTableIdentifier.from(storeLocation + "/carbon1", "default", "carbon1"),
-      classOf[C2DataMapFactory].getName,
-      "test")
-
+    sql(s"CREATE DATAMAP test ON TABLE carbon1 USING '${classOf[C2DataMapFactory].getName}'")
     val df = buildTestData(33000)
 
     // save dataframe to carbon file
@@ -119,11 +117,8 @@ class DataMapWriterSuite extends QueryTest with BeforeAndAfterAll {
   }
 
   test("test write datamap 2 blocklet") {
-    // register datamap writer
-    DataMapStoreManager.getInstance().createAndRegisterDataMap(
-      AbsoluteTableIdentifier.from(storeLocation + "/carbon2", "default", "carbon2"),
-      classOf[C2DataMapFactory].getName,
-      "test")
+    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")

http://git-wip-us.apache.org/repos/asf/carbondata/blob/c7a9f15e/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
index 9c8cc15..90f0972 100644
--- 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
@@ -36,6 +36,7 @@ import org.apache.carbondata.core.datastore.impl.FileFactory
 import org.apache.carbondata.core.datastore.page.ColumnPage
 import org.apache.carbondata.core.indexstore.{Blocklet, FineGrainBlocklet, PartitionSpec}
 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
@@ -48,22 +49,21 @@ import org.apache.carbondata.spark.testsuite.datacompaction.CompactionSupportGlo
 
 class FGDataMapFactory extends AbstractFineGrainDataMapFactory {
   var identifier: AbsoluteTableIdentifier = _
-  var dataMapName: String = _
+  var dataMapSchema: DataMapSchema = _
 
   /**
    * Initialization of Datamap factory with the identifier and datamap name
    */
-  override def init(identifier: AbsoluteTableIdentifier,
-      dataMapName: String): Unit = {
+  override def init(identifier: AbsoluteTableIdentifier, dataMapSchema: DataMapSchema): Unit = {
     this.identifier = identifier
-    this.dataMapName = dataMapName
+    this.dataMapSchema = dataMapSchema
   }
 
   /**
    * Return a new write for this datamap
    */
   override def createWriter(segment: Segment, dataWritePath: String): AbstractDataMapWriter = {
-    new FGDataMapWriter(identifier, segment, dataWritePath, dataMapName)
+    new FGDataMapWriter(identifier, segment, dataWritePath, dataMapSchema)
   }
 
   /**
@@ -136,7 +136,8 @@ class FGDataMapFactory extends AbstractFineGrainDataMapFactory {
    * Return metadata of this datamap
    */
   override def getMeta: DataMapMeta = {
-    new DataMapMeta(Seq("name").toList.asJava, new ArrayBuffer[ExpressionType]().toList.asJava)
+    new DataMapMeta(dataMapSchema.getProperties.get("indexcolumns").split(",").toList.asJava,
+      List(ExpressionType.EQUALS, ExpressionType.IN).asJava)
   }
 }
 
@@ -225,12 +226,16 @@ class FGDataMap extends AbstractFineGrainDataMap {
   }
 
   def getEqualToExpression(expression: Expression, buffer: ArrayBuffer[Expression]): Unit = {
-    if (expression.getChildren != null) {
-      expression.getChildren.asScala.map { f =>
-        if (f.isInstanceOf[EqualToExpression]) {
-          buffer += f
+    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)
         }
-        getEqualToExpression(f, buffer)
       }
     }
   }
@@ -246,11 +251,12 @@ class FGDataMap extends AbstractFineGrainDataMap {
 }
 
 class FGDataMapWriter(identifier: AbsoluteTableIdentifier,
-    segment: Segment, dataWriterPath: String, dataMapName: String)
+    segment: Segment, dataWriterPath: String, dataMapSchema: DataMapSchema)
   extends AbstractDataMapWriter(identifier, segment, dataWriterPath) {
 
   var currentBlockId: String = null
-  val fgwritepath = dataWriterPath + "/" + System.nanoTime() + ".datamap"
+  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])]()
@@ -421,14 +427,44 @@ class FGDataMapTestCase extends QueryTest with BeforeAndAfterAll {
       """.stripMargin)
     val table = CarbonMetadata.getInstance().getCarbonTable("default_datamap_test")
     // register datamap writer
-    DataMapStoreManager.getInstance().createAndRegisterDataMap(
-      table.getAbsoluteTableIdentifier,
-      classOf[FGDataMapFactory].getName, "fgdatamap")
+    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")

http://git-wip-us.apache.org/repos/asf/carbondata/blob/c7a9f15e/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
index 6ac9c7a..717af6f 100644
--- 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
@@ -23,7 +23,8 @@ import org.apache.spark.sql.Row
 import org.apache.spark.sql.test.util.QueryTest
 import org.scalatest.BeforeAndAfterAll
 
-import org.apache.carbondata.common.exceptions.sql.MalformedDataMapCommandException
+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
@@ -42,39 +43,22 @@ class TestDataMapCommand extends QueryTest with BeforeAndAfterAll {
 
   val newClass = "org.apache.spark.sql.CarbonSource"
 
-  test("test datamap create: don't support using class, only support short name") {
-    intercept[MalformedDataMapCommandException] {
+  test("test datamap create: don't support using non-exist class") {
+    intercept[MetadataProcessException] {
       sql(s"CREATE DATAMAP datamap1 ON TABLE datamaptest USING '$newClass'")
-      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("datamap1"))
-      assert(dataMapSchemaList.get(0).getClassName.equals(newClass))
     }
   }
 
-  test("test datamap create with dmproperties: don't support using class") {
-    intercept[MalformedDataMapCommandException] {
+  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')")
-      val table = CarbonMetadata.getInstance().getCarbonTable("default", "datamaptest")
-      assert(table != null)
-      val dataMapSchemaList = table.getTableInfo.getDataMapSchemaList
-      assert(dataMapSchemaList.size() == 2)
-      assert(dataMapSchemaList.get(1).getDataMapName.equals("datamap2"))
-      assert(dataMapSchemaList.get(1).getClassName.equals(newClass))
-      assert(dataMapSchemaList.get(1).getProperties.get("key").equals("value"))
     }
   }
 
-  test("test datamap create with existing name: don't support using class") {
-    intercept[MalformedDataMapCommandException] {
+  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')")
-      val table = CarbonMetadata.getInstance().getCarbonTable("default", "datamaptest")
-      assert(table != null)
-      val dataMapSchemaList = table.getTableInfo.getDataMapSchemaList
-      assert(dataMapSchemaList.size() == 2)
     }
   }
 
@@ -106,8 +90,7 @@ class TestDataMapCommand extends QueryTest with BeforeAndAfterAll {
       sql("drop datamap datamap_hiveMetaStoreTable on table hiveMetaStoreTable")
       checkExistence(sql("show datamap on table hiveMetaStoreTable"), false, "datamap_hiveMetaStoreTable")
 
-    }
-    finally {
+    } finally {
       sql("drop table hiveMetaStoreTable")
       CarbonProperties.getInstance()
         .addProperty(CarbonCommonConstants.ENABLE_HIVE_SCHEMA_META_STORE,
@@ -136,6 +119,8 @@ class TestDataMapCommand extends QueryTest with BeforeAndAfterAll {
         "datamap_hiveMetaStoreTable_1")
       assert(sql("show datamap on table hiveMetaStoreTable_1").collect().length == 0)
       sql("drop table hiveMetaStoreTable_1")
+
+      checkExistence(sql("show tables"), false, "datamap_hiveMetaStoreTable_1")
     }
     finally {
       CarbonProperties.getInstance()
@@ -145,17 +130,17 @@ class TestDataMapCommand extends QueryTest with BeforeAndAfterAll {
   }
 
   test("test datamap create with preagg with duplicate name") {
-    intercept[Exception] {
-      sql(
-        s"""
-           | CREATE DATAMAP datamap2 ON TABLE datamaptest
-           | USING 'preaggregate'
-           | DMPROPERTIES('key'='value')
-           | AS SELECT COUNT(a) FROM datamaptest
+    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 datamap2 ON TABLE datamaptest
+           | CREATE DATAMAP datamap10 ON TABLE datamaptest
            | USING 'preaggregate'
            | DMPROPERTIES('key'='value')
            | AS SELECT COUNT(a) FROM datamaptest
@@ -167,10 +152,9 @@ class TestDataMapCommand extends QueryTest with BeforeAndAfterAll {
     assert(dataMapSchemaList.size() == 2)
   }
 
-  test("test datamap drop with preagg") {
-    intercept[Exception] {
-      sql("drop table datamap3")
-
+  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)
@@ -178,8 +162,8 @@ class TestDataMapCommand extends QueryTest with BeforeAndAfterAll {
     assert(dataMapSchemaList.size() == 2)
   }
 
-  test("test show datamap without preaggregate: don't support using class") {
-    intercept[MalformedDataMapCommandException] {
+  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')")
@@ -188,8 +172,8 @@ class TestDataMapCommand extends QueryTest with BeforeAndAfterAll {
     }
   }
 
-  test("test show datamap with preaggregate: don't support using class") {
-    intercept[MalformedDataMapCommandException] {
+  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")
@@ -206,8 +190,8 @@ class TestDataMapCommand extends QueryTest with BeforeAndAfterAll {
     assert(sql("show datamap on table datamapshowtest").collect().length == 0)
   }
 
-  test("test show datamap after dropping datamap: don't support using class") {
-    intercept[MalformedDataMapCommandException] {
+  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")

http://git-wip-us.apache.org/repos/asf/carbondata/blob/c7a9f15e/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
new file mode 100644
index 0000000..e69de29

http://git-wip-us.apache.org/repos/asf/carbondata/blob/c7a9f15e/integration/spark-common/src/main/scala/org/apache/carbondata/spark/rdd/SparkDataMapJob.scala
----------------------------------------------------------------------
diff --git a/integration/spark-common/src/main/scala/org/apache/carbondata/spark/rdd/SparkDataMapJob.scala b/integration/spark-common/src/main/scala/org/apache/carbondata/spark/rdd/SparkDataMapJob.scala
index 60052f0..8d2f9ee 100644
--- a/integration/spark-common/src/main/scala/org/apache/carbondata/spark/rdd/SparkDataMapJob.scala
+++ b/integration/spark-common/src/main/scala/org/apache/carbondata/spark/rdd/SparkDataMapJob.scala
@@ -37,8 +37,8 @@ import org.apache.carbondata.hadoop.api.{DataMapJob, DistributableDataMapFormat}
 class SparkDataMapJob extends DataMapJob {
 
   override def execute(dataMapFormat: DistributableDataMapFormat,
-      resolverIntf: FilterResolverIntf): util.List[ExtendedBlocklet] = {
-    new DataMapPruneRDD(SparkContext.getOrCreate(), dataMapFormat, resolverIntf).collect().toList
+      filter: FilterResolverIntf): util.List[ExtendedBlocklet] = {
+    new DataMapPruneRDD(SparkContext.getOrCreate(), dataMapFormat, filter).collect().toList
       .asJava
   }
 }
@@ -53,7 +53,6 @@ class DataMapRDDPartition(rddId: Int, idx: Int, val inputSplit: InputSplit) exte
  * RDD to prune the datamaps across spark cluster
  * @param sc
  * @param dataMapFormat
- * @param resolverIntf
  */
 class DataMapPruneRDD(sc: SparkContext,
     dataMapFormat: DistributableDataMapFormat,
@@ -70,7 +69,6 @@ class DataMapPruneRDD(sc: SparkContext,
     val attemptId = new TaskAttemptID(jobTrackerId, id, TaskType.MAP, split.index, 0)
     val attemptContext = new TaskAttemptContextImpl(new Configuration(), attemptId)
     val inputSplit = split.asInstanceOf[DataMapRDDPartition].inputSplit
-    DistributableDataMapFormat.setFilterExp(attemptContext.getConfiguration, resolverIntf)
     val reader = dataMapFormat.createRecordReader(inputSplit, attemptContext)
     reader.initialize(inputSplit, attemptContext)
     val iter = new Iterator[ExtendedBlocklet] {

http://git-wip-us.apache.org/repos/asf/carbondata/blob/c7a9f15e/integration/spark2/src/main/scala/org/apache/spark/sql/CarbonSource.scala
----------------------------------------------------------------------
diff --git a/integration/spark2/src/main/scala/org/apache/spark/sql/CarbonSource.scala b/integration/spark2/src/main/scala/org/apache/spark/sql/CarbonSource.scala
index adf5e04..aab2897 100644
--- a/integration/spark2/src/main/scala/org/apache/spark/sql/CarbonSource.scala
+++ b/integration/spark2/src/main/scala/org/apache/spark/sql/CarbonSource.scala
@@ -101,8 +101,12 @@ class CarbonSource extends CreatableRelationProvider with RelationProvider
         CarbonException.analysisException(s"table path already exists.")
       case (SaveMode.Overwrite, true) =>
         val dbName = CarbonEnv.getDatabaseName(options.dbName)(sqlContext.sparkSession)
-        sqlContext.sparkSession
-          .sql(s"DROP TABLE IF EXISTS $dbName.${options.tableName}")
+        // In order to overwrite, delete all segments in the table
+        sqlContext.sparkSession.sql(
+          s"""
+             | DELETE FROM TABLE $dbName.${options.tableName}
+             | WHERE SEGMENT.STARTTIME BEFORE '2099-06-01 01:00:00'
+           """.stripMargin)
         (true, false)
       case (SaveMode.Overwrite, false) | (SaveMode.ErrorIfExists, false) =>
         (true, false)

http://git-wip-us.apache.org/repos/asf/carbondata/blob/c7a9f15e/integration/spark2/src/main/scala/org/apache/spark/sql/SparkUnknownExpression.scala
----------------------------------------------------------------------
diff --git a/integration/spark2/src/main/scala/org/apache/spark/sql/SparkUnknownExpression.scala b/integration/spark2/src/main/scala/org/apache/spark/sql/SparkUnknownExpression.scala
index d536746..1de66c1 100644
--- a/integration/spark2/src/main/scala/org/apache/spark/sql/SparkUnknownExpression.scala
+++ b/integration/spark2/src/main/scala/org/apache/spark/sql/SparkUnknownExpression.scala
@@ -32,7 +32,9 @@ import org.apache.carbondata.core.scan.filter.intf.{ExpressionType, RowIntf}
 import org.apache.carbondata.spark.util.CarbonScalaUtil
 
 
-class SparkUnknownExpression(var sparkExp: SparkExpression)
+class SparkUnknownExpression(
+    var sparkExp: SparkExpression,
+    expressionType: ExpressionType = ExpressionType.UNKNOWN)
   extends UnknownExpression with ConditionalExpression {
 
   private var evaluateExpression: (InternalRow) => Any = sparkExp.eval
@@ -64,7 +66,7 @@ class SparkUnknownExpression(var sparkExp: SparkExpression)
   }
 
   override def getFilterExpressionType: ExpressionType = {
-    ExpressionType.UNKNOWN
+    expressionType
   }
 
   override def getString: String = {

http://git-wip-us.apache.org/repos/asf/carbondata/blob/c7a9f15e/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 6508777..c6d86b3 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
@@ -16,17 +16,23 @@
  */
 package org.apache.spark.sql.execution.command.datamap
 
+import scala.collection.JavaConverters._
+
 import org.apache.spark.sql._
 import org.apache.spark.sql.catalyst.analysis.TableAlreadyExistsException
 import org.apache.spark.sql.catalyst.TableIdentifier
 import org.apache.spark.sql.execution.command._
-import org.apache.spark.sql.execution.command.preaaggregate.CreatePreAggregateTableCommand
+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.metadata.schema.datamap.DataMapProvider
+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}
 
 /**
  * Below command class will be used to create datamap on table
@@ -52,61 +58,62 @@ case class CarbonCreateDataMapCommand(
     if (carbonTable.isStreamingTable) {
       throw new MalformedCarbonCommandException("Streaming table does not support creating datamap")
     }
-    val LOGGER = LogServiceFactory.getLogService(this.getClass.getCanonicalName)
-    val dbName = tableIdentifier.database.getOrElse(sparkSession.catalog.currentDatabase)
-    val tableName = tableIdentifier.table + "_" + dataMapName
-    val newDmProperties = if (dmProperties.get(TimeSeriesUtil.TIMESERIES_EVENTTIME).isDefined) {
-      dmProperties.updated(TimeSeriesUtil.TIMESERIES_EVENTTIME,
-        dmProperties.get(TimeSeriesUtil.TIMESERIES_EVENTTIME).get.trim)
-    } else {
-      dmProperties
-    }
-    val dataMapProvider = {
-      try {
-        DataMapProvider.getDataMapProvider(dmClassName)
-      } catch {
-        case e: UnsupportedOperationException =>
-          throw new MalformedDataMapCommandException(e.getMessage)
-      }
-    }
-    if (sparkSession.sessionState.catalog.listTables(dbName)
-      .exists(_.table.equalsIgnoreCase(tableName))) {
-      LOGGER.audit(
-        s"Table creation with Database name [$dbName] and Table name [$tableName] failed. " +
-          s"Table [$tableName] already exists under database [$dbName]")
-      tableIsExists = true
-      if (!ifNotExistsSet) {
-        throw new TableAlreadyExistsException(dbName, tableName)
-      }
-    } else {
-      TimeSeriesUtil.validateTimeSeriesGranularity(newDmProperties, dmClassName)
-      createPreAggregateTableCommands = if (dataMapProvider == TIMESERIES) {
-        val details = TimeSeriesUtil
-          .getTimeSeriesGranularityDetails(newDmProperties, dmClassName)
-        val updatedDmProperties = newDmProperties - details._1
+    validateDataMapName(carbonTable)
+
+    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,
-          dataMapProvider,
+          DataMapProvider.TIMESERIES,
           updatedDmProperties,
           queryString.get,
-          Some(details._1),
-          ifNotExistsSet = ifNotExistsSet)
+          Some(details._1))
       } else {
         CreatePreAggregateTableCommand(
           dataMapName,
           tableIdentifier,
-          dataMapProvider,
-          newDmProperties,
-          queryString.get,
-          ifNotExistsSet = ifNotExistsSet)
+          DataMapProvider.PREAGGREGATE,
+          dmProperties,
+          queryString.get
+        )
+      }
+      try {
+        createPreAggregateTableCommands.processMetadata(sparkSession)
+      } catch {
+        case e: Throwable => throw new MetadataProcessException(s"Failed to create datamap " +
+                                                                s"'$dataMapName'", e)
       }
-      createPreAggregateTableCommands.processMetadata(sparkSession)
+    } 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): Unit = {
+    val existingDataMaps = carbonTable.getTableInfo.getDataMapSchemaList
+    existingDataMaps.asScala.foreach { dataMapSchema =>
+      if (dataMapSchema.getDataMapName.equalsIgnoreCase(dataMapName)) {
+        throw new MalformedDataMapCommandException(s"DataMap name '$dataMapName' already exist")
+      }
+    }
+  }
+
   override def processData(sparkSession: SparkSession): Seq[Row] = {
     if (dmClassName.equalsIgnoreCase(PREAGGREGATE.toString) ||
       dmClassName.equalsIgnoreCase(TIMESERIES.toString)) {
@@ -116,7 +123,7 @@ case class CarbonCreateDataMapCommand(
         Seq.empty
       }
     } else {
-      throw new MalformedDataMapCommandException("Unknown datamap provider/class " + dmClassName)
+      Seq.empty
     }
   }
 

http://git-wip-us.apache.org/repos/asf/carbondata/blob/c7a9f15e/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 dcc71a2..e89d15a 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
@@ -27,6 +27,7 @@ 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
@@ -74,7 +75,7 @@ case class CarbonDropDataMapCommand(
         Some(CarbonEnv.getCarbonTable(databaseNameOp, tableName)(sparkSession))
       } catch {
         case ex: NoSuchTableException =>
-          throw ex
+          throw new MetadataProcessException(s"Dropping datamap $dataMapName failed", ex)
       }
       // If datamap to be dropped in parent table then drop the datamap from metastore and remove
       // entry from parent table.

http://git-wip-us.apache.org/repos/asf/carbondata/blob/c7a9f15e/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
index c836584..c02ac4f 100644
--- 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
@@ -30,6 +30,7 @@ import org.apache.spark.sql.execution.command.timeseries.TimeSeriesUtil
 import org.apache.spark.sql.optimizer.CarbonFilters
 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
@@ -67,8 +68,10 @@ case class CreatePreAggregateTableCommand(
     dmProperties.foreach(t => tableProperties.put(t._1, t._2))
 
     parentTable = PreAggregateUtil.getParentCarbonTable(df.logicalPlan)
-    assert(parentTable.getTableName.equalsIgnoreCase(parentTableIdentifier.table),
-      "Parent table name is different in select and create")
+    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 =>
@@ -131,23 +134,19 @@ case class CreatePreAggregateTableCommand(
     dmProperties.foreach(f => childSchema.getProperties.put(f._1, f._2))
 
     // updating the parent table about child table
-    try {
-      PreAggregateUtil.updateMainTable(
-        CarbonEnv.getDatabaseName(parentTableIdentifier.database)(sparkSession),
-        parentTableIdentifier.table,
-        childSchema,
-        sparkSession)
-    } catch {
-      case ex: Exception =>
-        undoMetadata(sparkSession, ex)
-        throw ex
-    }
+    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) {
       PreAggregateUtil.createTimeSeriesSelectQueryFromMain(childSchema.getChildSchema,
         parentTable.getTableName,
         parentTable.getDatabaseName)
-    }
-    else {
+    } else {
       queryString
     }
     val dataFrame = sparkSession.sql(new CarbonSpark2SqlParser().addPreAggLoadFunction(

http://git-wip-us.apache.org/repos/asf/carbondata/blob/c7a9f15e/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 a5d256c..dd42b50 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
@@ -34,6 +34,7 @@ import org.apache.spark.sql.hive.CarbonRelation
 import org.apache.spark.sql.parser.CarbonSpark2SqlParser
 import org.apache.spark.sql.types.DataType
 
+import org.apache.carbondata.common.exceptions.MetadataProcessException
 import org.apache.carbondata.common.exceptions.sql.MalformedCarbonCommandException
 import org.apache.carbondata.common.logging.{LogService, LogServiceFactory}
 import org.apache.carbondata.core.constants.CarbonCommonConstants
@@ -406,21 +407,20 @@ 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 dbName
-   * @param tableName
    * @param childSchema
    * @param sparkSession
    */
-  def updateMainTable(dbName: String, tableName: String,
+  def updateMainTable(carbonTable: CarbonTable,
       childSchema: DataMapSchema, sparkSession: SparkSession): Unit = {
     val LOGGER: LogService = LogServiceFactory.getLogService(this.getClass.getCanonicalName)
     val locksToBeAcquired = List(LockUsage.METADATA_LOCK,
       LockUsage.DROP_TABLE_LOCK)
     var locks = List.empty[ICarbonLock]
-    var carbonTable: CarbonTable = null
+    var numberOfCurrentChild: Int = 0
+    val dbName = carbonTable.getDatabaseName
+    val tableName = carbonTable.getTableName
     try {
       val metastore = CarbonEnv.getInstance(sparkSession).carbonMetastore
-      carbonTable = CarbonEnv.getCarbonTable(Some(dbName), tableName)(sparkSession)
       locks = acquireLock(dbName, tableName, locksToBeAcquired, carbonTable)
       // get the latest carbon table and check for column existence
       // read the latest schema file
@@ -433,7 +433,7 @@ object PreAggregateUtil {
         carbonTable.getTablePath)
       if (wrapperTableInfo.getDataMapSchemaList.asScala.
         exists(f => f.getDataMapName.equalsIgnoreCase(childSchema.getDataMapName))) {
-        throw new Exception("Duplicate datamap")
+        throw new MetadataProcessException("DataMap name already exist")
       }
       wrapperTableInfo.getDataMapSchemaList.add(childSchema)
       val thriftTable = schemaConverter

http://git-wip-us.apache.org/repos/asf/carbondata/blob/c7a9f15e/integration/spark2/src/main/scala/org/apache/spark/sql/execution/strategy/CarbonLateDecodeStrategy.scala
----------------------------------------------------------------------
diff --git a/integration/spark2/src/main/scala/org/apache/spark/sql/execution/strategy/CarbonLateDecodeStrategy.scala b/integration/spark2/src/main/scala/org/apache/spark/sql/execution/strategy/CarbonLateDecodeStrategy.scala
index f6d5e25..e82c9d7 100644
--- a/integration/spark2/src/main/scala/org/apache/spark/sql/execution/strategy/CarbonLateDecodeStrategy.scala
+++ b/integration/spark2/src/main/scala/org/apache/spark/sql/execution/strategy/CarbonLateDecodeStrategy.scala
@@ -614,7 +614,7 @@ private[sql] class CarbonLateDecodeStrategy extends SparkStrategy {
         CastExpressionOptimization.checkIfCastCanBeRemove(c)
       case c@LessThanOrEqual(Literal(v, t), Cast(a: Attribute, _)) =>
         CastExpressionOptimization.checkIfCastCanBeRemove(c)
-      case StartsWith(a: Attribute, Literal(v, t)) =>
+      case s@StartsWith(a: Attribute, Literal(v, t)) =>
         Some(sources.StringStartsWith(a.name, v.toString))
       case c@EndsWith(a: Attribute, Literal(v, t)) =>
         Some(CarbonEndsWith(c))

http://git-wip-us.apache.org/repos/asf/carbondata/blob/c7a9f15e/integration/spark2/src/main/scala/org/apache/spark/sql/optimizer/CarbonFilters.scala
----------------------------------------------------------------------
diff --git a/integration/spark2/src/main/scala/org/apache/spark/sql/optimizer/CarbonFilters.scala b/integration/spark2/src/main/scala/org/apache/spark/sql/optimizer/CarbonFilters.scala
index e5eb53c..38c5146 100644
--- a/integration/spark2/src/main/scala/org/apache/spark/sql/optimizer/CarbonFilters.scala
+++ b/integration/spark2/src/main/scala/org/apache/spark/sql/optimizer/CarbonFilters.scala
@@ -41,6 +41,10 @@ import org.apache.carbondata.core.scan.expression.{ColumnExpression => CarbonCol
 import org.apache.carbondata.core.scan.expression.conditional._
 import org.apache.carbondata.core.scan.expression.logical.{AndExpression, FalseExpression, OrExpression}
 import org.apache.carbondata.core.util.{CarbonProperties, ThreadLocalSessionInfo}
+import org.apache.carbondata.core.scan.filter.intf.ExpressionType
+import org.apache.carbondata.core.util.ThreadLocalSessionInfo
+import org.apache.carbondata.core.util.path.CarbonTablePath
+import org.apache.carbondata.core.util.CarbonProperties
 import org.apache.carbondata.spark.CarbonAliasDecoderRelation
 import org.apache.carbondata.spark.util.CarbonScalaUtil
 
@@ -50,6 +54,7 @@ import org.apache.carbondata.spark.util.CarbonScalaUtil
  */
 object CarbonFilters {
 
+  val carbonProperties = CarbonProperties.getInstance()
   /**
    * Converts data sources filters to carbon filter predicates.
    */
@@ -117,25 +122,20 @@ object CarbonFilters {
             new OrExpression(lhsFilter, rhsFilter)
           }
         case sources.StringStartsWith(name, value) if value.length > 0 =>
-          val l = new GreaterThanEqualToExpression(getCarbonExpression(name),
-            getCarbonLiteralExpression(name, value))
-          val maxValueLimit = value.substring(0, value.length - 1) +
-                              (value.charAt(value.length - 1).toInt + 1).toChar
-          val r = new LessThanExpression(
-            getCarbonExpression(name), getCarbonLiteralExpression(name, maxValueLimit))
-          Some(new AndExpression(l, r))
+          Some(new StartsWithExpression(getCarbonExpression(name),
+            getCarbonLiteralExpression(name, value)))
         case CarbonEndsWith(expr: Expression) =>
           Some(new SparkUnknownExpression(expr.transform {
             case AttributeReference(name, dataType, _, _) =>
               CarbonBoundReference(new CarbonColumnExpression(name.toString,
                 CarbonScalaUtil.convertSparkToCarbonDataType(dataType)), dataType, expr.nullable)
-          }))
+          }, ExpressionType.ENDSWITH))
         case CarbonContainsWith(expr: Expression) =>
           Some(new SparkUnknownExpression(expr.transform {
             case AttributeReference(name, dataType, _, _) =>
               CarbonBoundReference(new CarbonColumnExpression(name.toString,
                 CarbonScalaUtil.convertSparkToCarbonDataType(dataType)), dataType, expr.nullable)
-          }))
+          }, ExpressionType.CONTAINSWITH))
         case CastExpr(expr: Expression) =>
           Some(transformExpression(expr))
         case FalseExpr() =>
@@ -266,7 +266,7 @@ object CarbonFilters {
           CastExpressionOptimization.checkIfCastCanBeRemove(c)
         case c@LessThanOrEqual(Literal(v, t), Cast(a: Attribute, _)) =>
           CastExpressionOptimization.checkIfCastCanBeRemove(c)
-        case StartsWith(a: Attribute, Literal(v, t)) =>
+        case s@StartsWith(a: Attribute, Literal(v, t)) =>
           Some(sources.StringStartsWith(a.name, v.toString))
         case c@EndsWith(a: Attribute, Literal(v, t)) =>
           Some(CarbonEndsWith(c))

http://git-wip-us.apache.org/repos/asf/carbondata/blob/c7a9f15e/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 66f8bc5..f514074 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
@@ -33,7 +33,7 @@ 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.datastore.page.ColumnPage;
-import org.apache.carbondata.core.metadata.AbsoluteTableIdentifier;
+import org.apache.carbondata.core.metadata.schema.table.CarbonTable;
 import org.apache.carbondata.processing.store.TablePage;
 
 /**
@@ -50,9 +50,9 @@ public class DataMapWriterListener {
   /**
    * register all datamap writer for specified table and segment
    */
-  public void registerAllWriter(AbsoluteTableIdentifier identifier, String segmentId,
+  public void registerAllWriter(CarbonTable carbonTable, String segmentId,
       String dataWritePath) {
-    List<TableDataMap> tableDataMaps = DataMapStoreManager.getInstance().getAllDataMap(identifier);
+    List<TableDataMap> tableDataMaps = DataMapStoreManager.getInstance().getAllDataMap(carbonTable);
     if (tableDataMaps != null) {
       for (TableDataMap tableDataMap : tableDataMaps) {
         DataMapFactory factory = tableDataMap.getDataMapFactory();

http://git-wip-us.apache.org/repos/asf/carbondata/blob/c7a9f15e/processing/src/main/java/org/apache/carbondata/processing/loading/DataLoadProcessBuilder.java
----------------------------------------------------------------------
diff --git a/processing/src/main/java/org/apache/carbondata/processing/loading/DataLoadProcessBuilder.java b/processing/src/main/java/org/apache/carbondata/processing/loading/DataLoadProcessBuilder.java
index 0ee77da..21a4b3e 100644
--- a/processing/src/main/java/org/apache/carbondata/processing/loading/DataLoadProcessBuilder.java
+++ b/processing/src/main/java/org/apache/carbondata/processing/loading/DataLoadProcessBuilder.java
@@ -263,7 +263,7 @@ public final class DataLoadProcessBuilder {
     if (carbonTable.isHivePartitionTable()) {
       configuration.setWritingCoresCount((short) 1);
     }
-    TableSpec tableSpec = new TableSpec(dimensions, measures);
+    TableSpec tableSpec = new TableSpec(carbonTable);
     configuration.setTableSpec(tableSpec);
     return configuration;
   }

http://git-wip-us.apache.org/repos/asf/carbondata/blob/c7a9f15e/processing/src/main/java/org/apache/carbondata/processing/store/CarbonFactDataHandlerModel.java
----------------------------------------------------------------------
diff --git a/processing/src/main/java/org/apache/carbondata/processing/store/CarbonFactDataHandlerModel.java b/processing/src/main/java/org/apache/carbondata/processing/store/CarbonFactDataHandlerModel.java
index d6af747..8aa5bde 100644
--- a/processing/src/main/java/org/apache/carbondata/processing/store/CarbonFactDataHandlerModel.java
+++ b/processing/src/main/java/org/apache/carbondata/processing/store/CarbonFactDataHandlerModel.java
@@ -260,8 +260,8 @@ public class CarbonFactDataHandlerModel {
     carbonFactDataHandlerModel.sortScope = CarbonDataProcessorUtil.getSortScope(configuration);
 
     DataMapWriterListener listener = new DataMapWriterListener();
-    listener.registerAllWriter(configuration.getTableIdentifier(), configuration.getSegmentId(),
-        storeLocation[new Random().nextInt(storeLocation.length)]);
+    listener.registerAllWriter(configuration.getTableSpec().getCarbonTable(),
+        configuration.getSegmentId(), storeLocation[new Random().nextInt(storeLocation.length)]);
     carbonFactDataHandlerModel.dataMapWriterlistener = listener;
     carbonFactDataHandlerModel.writingCoresCount = configuration.getWritingCoresCount();
 
@@ -320,13 +320,11 @@ public class CarbonFactDataHandlerModel {
     carbonFactDataHandlerModel.setPrimitiveDimLens(segmentProperties.getDimColumnsCardinality());
     carbonFactDataHandlerModel.setBlockSizeInMB(carbonTable.getBlockSizeInMB());
 
-    carbonFactDataHandlerModel.tableSpec = new TableSpec(
-        segmentProperties.getDimensions(),
-        segmentProperties.getMeasures());
-
+    carbonFactDataHandlerModel.tableSpec =
+        new TableSpec(loadModel.getCarbonDataLoadSchema().getCarbonTable());
     DataMapWriterListener listener = new DataMapWriterListener();
     listener.registerAllWriter(
-        loadModel.getCarbonDataLoadSchema().getCarbonTable().getAbsoluteTableIdentifier(),
+        loadModel.getCarbonDataLoadSchema().getCarbonTable(),
         loadModel.getSegmentId(),
         tempStoreLocation[new Random().nextInt(tempStoreLocation.length)]);
     carbonFactDataHandlerModel.dataMapWriterlistener = listener;


[16/20] carbondata git commit: [CARBONDATA-2206] support lucene index datamap

Posted by ja...@apache.org.
[CARBONDATA-2206] support lucene index datamap

This PR is an initial effort to integrate lucene as an index datamap into carbondata.
A new module called carbondata-lucene is added to support lucene datamap:

1.Add LuceneFineGrainDataMap, implement FineGrainDataMap interface.
2.Add LuceneCoarseGrainDataMap, implement CoarseGrainDataMap interface.
3.Support writing lucene index via LuceneDataMapWriter.
4.Implement LuceneDataMapFactory
5.A UDF called TEXT_MATCH is added, use it to do filtering on string column by lucene

This closes #2003


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

Branch: refs/heads/datamap-rebase1
Commit: c0133aac80db1403754bea93a7654185b98ea9fb
Parents: 96ee82b
Author: Jacky Li <ja...@qq.com>
Authored: Mon Feb 26 16:30:38 2018 +0800
Committer: Jacky Li <ja...@qq.com>
Committed: Sun Mar 4 22:45:15 2018 +0800

----------------------------------------------------------------------
 .../carbondata/core/datamap/DataMapChooser.java |   4 +
 .../core/datamap/DataMapStoreManager.java       |   5 +-
 .../carbondata/core/datamap/dev/DataMap.java    |   2 +-
 .../core/datamap/dev/DataMapFactory.java        |   2 +-
 .../core/datamap/dev/DataMapWriter.java         |   7 +-
 .../cgdatamap/CoarseGrainDataMapFactory.java    |   1 +
 .../core/scan/filter/intf/ExpressionType.java   |   3 +-
 datamap/lucene/pom.xml                          | 149 +++++++++
 .../lucene/LuceneCoarseGrainDataMap.java        | 232 +++++++++++++
 .../lucene/LuceneCoarseGrainDataMapFactory.java |  72 ++++
 .../lucene/LuceneDataMapDistributable.java      |  36 ++
 .../lucene/LuceneDataMapFactoryBase.java        | 180 ++++++++++
 .../datamap/lucene/LuceneDataMapWriter.java     | 328 +++++++++++++++++++
 .../datamap/lucene/LuceneFineGrainDataMap.java  | 280 ++++++++++++++++
 .../lucene/LuceneFineGrainDataMapFactory.java   |  68 ++++
 .../lucene/LuceneCoarseGrainDataMapSuite.scala  |  73 +++++
 .../lucene/LuceneFineGrainDataMapSuite.scala    |  98 ++++++
 integration/spark-common-test/pom.xml           |   6 +
 .../testsuite/datamap/FGDataMapTestCase.scala   |   2 +-
 .../carbondata/datamap/DataMapProvider.java     |   4 +-
 .../datamap/IndexDataMapProvider.java           |   4 +-
 .../datamap/expression/MatchExpression.java     |  56 ++++
 .../carbondata/datamap/TextMatchUDF.scala       |  34 ++
 .../scala/org/apache/spark/sql/CarbonEnv.scala  |   5 +
 .../strategy/CarbonLateDecodeStrategy.scala     |   9 +
 .../spark/sql/optimizer/CarbonFilters.scala     |   4 +
 pom.xml                                         |   3 +
 .../datamap/DataMapWriterListener.java          |   6 +-
 .../store/writer/AbstractFactDataWriter.java    |  12 +-
 .../writer/v3/CarbonFactDataWriterImplV3.java   |   4 +-
 30 files changed, 1671 insertions(+), 18 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/carbondata/blob/c0133aac/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 94b48c6..c8c971d 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
@@ -228,6 +228,10 @@ public class DataMapChooser {
 
   private boolean contains(DataMapMeta mapMeta, List<ColumnExpression> columnExpressions,
       Set<ExpressionType> expressionTypes) {
+    if (mapMeta.getOptimizedOperation().contains(ExpressionType.TEXT_MATCH)) {
+      // TODO: fix it with right logic
+      return true;
+    }
     if (mapMeta.getIndexedColumns().size() == 0 || columnExpressions.size() == 0) {
       return false;
     }

http://git-wip-us.apache.org/repos/asf/carbondata/blob/c0133aac/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 e57a841..ab339e8 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
@@ -16,6 +16,7 @@
  */
 package org.apache.carbondata.core.datamap;
 
+import java.io.IOException;
 import java.util.ArrayList;
 import java.util.HashMap;
 import java.util.List;
@@ -144,7 +145,7 @@ public final class DataMapStoreManager {
    * The datamap is created using datamap name, datamap factory class and table identifier.
    */
   private TableDataMap createAndRegisterDataMap(AbsoluteTableIdentifier identifier,
-      DataMapSchema dataMapSchema) throws MalformedDataMapCommandException {
+      DataMapSchema dataMapSchema) throws MalformedDataMapCommandException, IOException {
     DataMapFactory dataMapFactory;
     try {
       // try to create datamap by reflection to test whether it is a valid DataMapFactory class
@@ -162,7 +163,7 @@ public final class DataMapStoreManager {
   }
 
   public TableDataMap registerDataMap(AbsoluteTableIdentifier identifier,
-      DataMapSchema dataMapSchema,  DataMapFactory dataMapFactory) {
+      DataMapSchema dataMapSchema,  DataMapFactory dataMapFactory) throws IOException {
     String table = identifier.getCarbonTableIdentifier().getTableUniqueName();
     // Just update the segmentRefreshMap with the table if not added.
     getTableSegmentRefresher(identifier);

http://git-wip-us.apache.org/repos/asf/carbondata/blob/c0133aac/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
index f036b0b..9fbdd90 100644
--- a/core/src/main/java/org/apache/carbondata/core/datamap/dev/DataMap.java
+++ b/core/src/main/java/org/apache/carbondata/core/datamap/dev/DataMap.java
@@ -42,7 +42,7 @@ public interface DataMap<T extends Blocklet> {
    * blocklets where these filters can exist.
    */
   List<T> prune(FilterResolverIntf filterExp, SegmentProperties segmentProperties,
-      List<PartitionSpec> partitions);
+      List<PartitionSpec> partitions) throws IOException;
 
   // TODO Move this method to Abstract class
   /**

http://git-wip-us.apache.org/repos/asf/carbondata/blob/c0133aac/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
index 7fe910e..f2c376a 100644
--- a/core/src/main/java/org/apache/carbondata/core/datamap/dev/DataMapFactory.java
+++ b/core/src/main/java/org/apache/carbondata/core/datamap/dev/DataMapFactory.java
@@ -35,7 +35,7 @@ public interface DataMapFactory<T extends DataMap> {
   /**
    * Initialization of Datamap factory with the identifier and datamap name
    */
-  void init(AbsoluteTableIdentifier identifier, DataMapSchema dataMapSchema);
+  void init(AbsoluteTableIdentifier identifier, DataMapSchema dataMapSchema) throws IOException;
 
   /**
    * Return a new write for this datamap

http://git-wip-us.apache.org/repos/asf/carbondata/blob/c0133aac/core/src/main/java/org/apache/carbondata/core/datamap/dev/DataMapWriter.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/carbondata/core/datamap/dev/DataMapWriter.java b/core/src/main/java/org/apache/carbondata/core/datamap/dev/DataMapWriter.java
index 4911f7b..6a3ee18 100644
--- a/core/src/main/java/org/apache/carbondata/core/datamap/dev/DataMapWriter.java
+++ b/core/src/main/java/org/apache/carbondata/core/datamap/dev/DataMapWriter.java
@@ -54,12 +54,12 @@ public abstract class DataMapWriter {
    *
    * @param blockId file name of the carbondata file
    */
-  public abstract void onBlockStart(String blockId);
+  public abstract void onBlockStart(String blockId) throws IOException;
 
   /**
    * End of block notification
    */
-  public abstract void onBlockEnd(String blockId);
+  public abstract void onBlockEnd(String blockId) throws IOException;
 
   /**
    * Start of new blocklet notification.
@@ -81,7 +81,8 @@ public abstract class DataMapWriter {
    * Implementation should copy the content of `pages` as needed, because `pages` memory
    * may be freed after this method returns, if using unsafe column page.
    */
-  public abstract void onPageAdded(int blockletId, int pageId, ColumnPage[] pages);
+  public abstract void onPageAdded(int blockletId, int pageId, ColumnPage[] pages)
+      throws IOException;
 
   /**
    * This is called during closing of writer.So after this call no more data will be sent to this

http://git-wip-us.apache.org/repos/asf/carbondata/blob/c0133aac/core/src/main/java/org/apache/carbondata/core/datamap/dev/cgdatamap/CoarseGrainDataMapFactory.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/carbondata/core/datamap/dev/cgdatamap/CoarseGrainDataMapFactory.java b/core/src/main/java/org/apache/carbondata/core/datamap/dev/cgdatamap/CoarseGrainDataMapFactory.java
index 4d20cdb..f9fdafb 100644
--- a/core/src/main/java/org/apache/carbondata/core/datamap/dev/cgdatamap/CoarseGrainDataMapFactory.java
+++ b/core/src/main/java/org/apache/carbondata/core/datamap/dev/cgdatamap/CoarseGrainDataMapFactory.java
@@ -14,6 +14,7 @@
  * 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.common.annotations.InterfaceAudience;

http://git-wip-us.apache.org/repos/asf/carbondata/blob/c0133aac/core/src/main/java/org/apache/carbondata/core/scan/filter/intf/ExpressionType.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/carbondata/core/scan/filter/intf/ExpressionType.java b/core/src/main/java/org/apache/carbondata/core/scan/filter/intf/ExpressionType.java
index 831acc8..d66c5b4 100644
--- a/core/src/main/java/org/apache/carbondata/core/scan/filter/intf/ExpressionType.java
+++ b/core/src/main/java/org/apache/carbondata/core/scan/filter/intf/ExpressionType.java
@@ -42,5 +42,6 @@ public enum ExpressionType {
   TRUE,
   STARTSWITH,
   ENDSWITH,
-  CONTAINSWITH
+  CONTAINSWITH,
+  TEXT_MATCH
 }

http://git-wip-us.apache.org/repos/asf/carbondata/blob/c0133aac/datamap/lucene/pom.xml
----------------------------------------------------------------------
diff --git a/datamap/lucene/pom.xml b/datamap/lucene/pom.xml
new file mode 100644
index 0000000..ee504c6
--- /dev/null
+++ b/datamap/lucene/pom.xml
@@ -0,0 +1,149 @@
+<project xmlns="http://maven.apache.org/POM/4.0.0"
+         xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance"
+         xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 http://maven.apache.org/xsd/maven-4.0.0.xsd">
+  <modelVersion>4.0.0</modelVersion>
+
+  <parent>
+    <groupId>org.apache.carbondata</groupId>
+    <artifactId>carbondata-parent</artifactId>
+    <version>1.4.0-SNAPSHOT</version>
+    <relativePath>../../pom.xml</relativePath>
+  </parent>
+
+  <artifactId>carbondata-lucene</artifactId>
+  <name>Apache CarbonData :: Lucene Index DataMap</name>
+
+  <properties>
+    <dev.path>${basedir}/../../dev</dev.path>
+    <lucene.version>6.3.0</lucene.version>
+    <solr.version>6.3.0</solr.version>
+  </properties>
+
+  <dependencies>
+    <dependency>
+      <groupId>org.apache.carbondata</groupId>
+      <artifactId>carbondata-spark2</artifactId>
+      <version>${project.version}</version>
+    </dependency>
+    <dependency>
+      <groupId>org.apache.lucene</groupId>
+      <artifactId>lucene-core</artifactId>
+      <version>${lucene.version}</version>
+      <exclusions>
+        <exclusion>
+          <groupId>*</groupId>
+          <artifactId>*</artifactId>
+        </exclusion>
+      </exclusions>
+    </dependency>
+    <dependency>
+      <groupId>org.apache.lucene</groupId>
+      <artifactId>lucene-analyzers-common</artifactId>
+      <version>${lucene.version}</version>
+      <exclusions>
+        <exclusion>
+          <groupId>*</groupId>
+          <artifactId>*</artifactId>
+        </exclusion>
+      </exclusions>
+    </dependency>
+    <dependency>
+      <groupId>org.apache.lucene</groupId>
+      <artifactId>lucene-queryparser</artifactId>
+      <version>${lucene.version}</version>
+      <exclusions>
+        <exclusion>
+          <groupId>*</groupId>
+          <artifactId>*</artifactId>
+        </exclusion>
+      </exclusions>
+    </dependency>
+    <dependency>
+      <groupId>org.apache.lucene</groupId>
+      <artifactId>lucene-sandbox</artifactId>
+      <version>${lucene.version}</version>
+      <exclusions>
+        <exclusion>
+          <groupId>*</groupId>
+          <artifactId>*</artifactId>
+        </exclusion>
+      </exclusions>
+    </dependency>
+    <dependency>
+      <groupId>org.apache.solr</groupId>
+      <artifactId>solr-core</artifactId>
+      <version>${solr.version}</version>
+      <exclusions>
+        <exclusion>
+          <groupId>*</groupId>
+          <artifactId>*</artifactId>
+        </exclusion>
+      </exclusions>
+    </dependency>
+    <dependency>
+      <groupId>org.apache.solr</groupId>
+      <artifactId>solr-solrj</artifactId>
+      <version>${solr.version}</version>
+      <exclusions>
+        <exclusion>
+          <groupId>*</groupId>
+          <artifactId>*</artifactId>
+        </exclusion>
+      </exclusions>
+    </dependency>
+    <dependency>
+      <groupId>org.scalatest</groupId>
+      <artifactId>scalatest_${scala.binary.version}</artifactId>
+      <scope>test</scope>
+    </dependency>
+  </dependencies>
+
+  <build>
+    <testSourceDirectory>src/test/scala</testSourceDirectory>
+    <resources>
+      <resource>
+        <directory>src/resources</directory>
+      </resource>
+      <resource>
+        <directory>.</directory>
+      </resource>
+    </resources>
+    <plugins>
+      <plugin>
+        <groupId>org.scala-tools</groupId>
+        <artifactId>maven-scala-plugin</artifactId>
+        <version>2.15.2</version>
+        <executions>
+          <execution>
+            <id>compile</id>
+            <goals>
+              <goal>compile</goal>
+            </goals>
+            <phase>compile</phase>
+          </execution>
+          <execution>
+            <id>testCompile</id>
+            <goals>
+              <goal>testCompile</goal>
+            </goals>
+            <phase>test</phase>
+          </execution>
+          <execution>
+            <phase>process-resources</phase>
+            <goals>
+              <goal>compile</goal>
+            </goals>
+          </execution>
+        </executions>
+      </plugin>
+      <plugin>
+        <artifactId>maven-compiler-plugin</artifactId>
+        <configuration>
+          <source>1.7</source>
+          <target>1.7</target>
+        </configuration>
+      </plugin>
+    </plugins>
+  </build>
+
+</project>

http://git-wip-us.apache.org/repos/asf/carbondata/blob/c0133aac/datamap/lucene/src/main/java/org/apache/carbondata/datamap/lucene/LuceneCoarseGrainDataMap.java
----------------------------------------------------------------------
diff --git a/datamap/lucene/src/main/java/org/apache/carbondata/datamap/lucene/LuceneCoarseGrainDataMap.java b/datamap/lucene/src/main/java/org/apache/carbondata/datamap/lucene/LuceneCoarseGrainDataMap.java
new file mode 100644
index 0000000..0b7df86
--- /dev/null
+++ b/datamap/lucene/src/main/java/org/apache/carbondata/datamap/lucene/LuceneCoarseGrainDataMap.java
@@ -0,0 +1,232 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.carbondata.datamap.lucene;
+
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+
+import org.apache.carbondata.common.annotations.InterfaceAudience;
+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.CoarseGrainDataMap;
+import org.apache.carbondata.core.datastore.block.SegmentProperties;
+import org.apache.carbondata.core.datastore.impl.FileFactory;
+import org.apache.carbondata.core.indexstore.Blocklet;
+import org.apache.carbondata.core.memory.MemoryException;
+import org.apache.carbondata.core.scan.filter.resolver.FilterResolverIntf;
+
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
+import org.apache.lucene.analysis.Analyzer;
+import org.apache.lucene.analysis.standard.StandardAnalyzer;
+import org.apache.lucene.document.Document;
+import org.apache.lucene.index.DirectoryReader;
+import org.apache.lucene.index.IndexReader;
+import org.apache.lucene.index.IndexableField;
+import org.apache.lucene.queryparser.classic.MultiFieldQueryParser;
+import org.apache.lucene.queryparser.classic.ParseException;
+import org.apache.lucene.queryparser.classic.QueryParser;
+import org.apache.lucene.search.IndexSearcher;
+import org.apache.lucene.search.Query;
+import org.apache.lucene.search.ScoreDoc;
+import org.apache.lucene.search.TopDocs;
+import org.apache.lucene.store.Directory;
+import org.apache.solr.store.hdfs.HdfsDirectory;
+
+@InterfaceAudience.Internal
+public class LuceneCoarseGrainDataMap extends CoarseGrainDataMap {
+
+  /**
+   * log information
+   */
+  private static final LogService LOGGER =
+      LogServiceFactory.getLogService(LuceneCoarseGrainDataMap.class.getName());
+
+  public static final int BLOCKID_ID = 0;
+
+  public static final int BLOCKLETID_ID = 1;
+
+  public static final int PAGEID_ID = 2;
+
+  public static final int ROWID_ID = 3;
+  /**
+   * searcher object for this datamap
+   */
+  private IndexSearcher indexSearcher = null;
+
+  /**
+   * default max values to return
+   */
+  private static int MAX_RESULT_NUMBER = 100;
+
+  /**
+   * analyzer for lucene index
+   */
+  private Analyzer analyzer;
+
+  LuceneCoarseGrainDataMap(Analyzer analyzer) {
+    this.analyzer = analyzer;
+  }
+
+  /**
+   * It is called to load the data map to memory or to initialize it.
+   */
+  @Override
+  public void init(DataMapModel dataMapModel) throws MemoryException, IOException {
+    // get this path from file path
+    Path indexPath = FileFactory.getPath(dataMapModel.getFilePath());
+
+    LOGGER.info("Lucene index read path " + indexPath.toString());
+
+    // get file system , use hdfs file system , realized in solr project
+    FileSystem fs = FileFactory.getFileSystem(indexPath);
+
+    // check this path valid
+    if (!fs.exists(indexPath)) {
+      String errorMessage = String.format("index directory %s not exists.", indexPath);
+      LOGGER.error(errorMessage);
+      throw new IOException(errorMessage);
+    }
+
+    if (!fs.isDirectory(indexPath)) {
+      String errorMessage = String.format("error index path %s, must be directory", indexPath);
+      LOGGER.error(errorMessage);
+      throw new IOException(errorMessage);
+    }
+
+    // open this index path , use HDFS default configuration
+    Directory indexDir = new HdfsDirectory(indexPath, FileFactory.getConfiguration());
+
+    IndexReader indexReader = DirectoryReader.open(indexDir);
+    if (indexReader == null) {
+      throw new RuntimeException("failed to create index reader object");
+    }
+
+    // create a index searcher object
+    indexSearcher = new IndexSearcher(indexReader);
+  }
+
+  /**
+   * Prune the datamap with filter expression. It returns the list of
+   * blocklets where these filters can exist.
+   */
+  @Override
+  public List<Blocklet> prune(FilterResolverIntf filterExp, SegmentProperties segmentProperties,
+      List<String> partitions) throws IOException {
+
+    // convert filter expr into lucene list query
+    List<String> fields = new ArrayList<String>();
+
+    // only for test , query all data
+    String strQuery = "*:*";
+
+    String[] sFields = new String[fields.size()];
+    fields.toArray(sFields);
+
+    // get analyzer
+    if (analyzer == null) {
+      analyzer = new StandardAnalyzer();
+    }
+
+    // use MultiFieldQueryParser to parser query
+    QueryParser queryParser = new MultiFieldQueryParser(sFields, analyzer);
+    Query query;
+    try {
+      query = queryParser.parse(strQuery);
+    } catch (ParseException e) {
+      String errorMessage = String
+          .format("failed to filter block with query %s, detail is %s", strQuery, e.getMessage());
+      LOGGER.error(errorMessage);
+      return null;
+    }
+
+    // execute index search
+    TopDocs result;
+    try {
+      result = indexSearcher.search(query, MAX_RESULT_NUMBER);
+    } catch (IOException e) {
+      String errorMessage =
+          String.format("failed to search lucene data, detail is %s", e.getMessage());
+      LOGGER.error(errorMessage);
+      throw new IOException(errorMessage);
+    }
+
+    // temporary data, delete duplicated data
+    // Map<BlockId, Map<BlockletId, Map<PageId, Set<RowId>>>>
+    Map<String, Set<Number>> mapBlocks = new HashMap<String, Set<Number>>();
+
+    for (ScoreDoc scoreDoc : result.scoreDocs) {
+      // get a document
+      Document doc = indexSearcher.doc(scoreDoc.doc);
+
+      // get all fields
+      List<IndexableField> fieldsInDoc = doc.getFields();
+
+      // get this block id Map<BlockId, Set<BlockletId>>>>
+      String blockId = fieldsInDoc.get(BLOCKID_ID).stringValue();
+      Set<Number> setBlocklets = mapBlocks.get(blockId);
+      if (setBlocklets == null) {
+        setBlocklets = new HashSet<Number>();
+        mapBlocks.put(blockId, setBlocklets);
+      }
+
+      // get the blocklet id Set<BlockletId>
+      Number blockletId = fieldsInDoc.get(BLOCKLETID_ID).numericValue();
+      if (!setBlocklets.contains(blockletId.intValue())) {
+        setBlocklets.add(blockletId.intValue());
+      }
+    }
+
+    // result blocklets
+    List<Blocklet> blocklets = new ArrayList<Blocklet>();
+
+    // transform all blocks into result type blocklets Map<BlockId, Set<BlockletId>>
+    for (Map.Entry<String, Set<Number>> mapBlock : mapBlocks.entrySet()) {
+      String blockId = mapBlock.getKey();
+      Set<Number> setBlocklets = mapBlock.getValue();
+
+      // for blocklets in this block Set<BlockletId>
+      for (Number blockletId : setBlocklets) {
+
+        // add a CoarseGrainBlocklet
+        blocklets.add(new Blocklet(blockId, blockletId.toString()));
+      }
+    }
+
+    return blocklets;
+  }
+
+  @Override
+  public boolean isScanRequired(FilterResolverIntf filterExp) {
+    return true;
+  }
+
+  /**
+   * Clear complete index table and release memory.
+   */
+  @Override
+  public void clear() {
+
+  }
+}

http://git-wip-us.apache.org/repos/asf/carbondata/blob/c0133aac/datamap/lucene/src/main/java/org/apache/carbondata/datamap/lucene/LuceneCoarseGrainDataMapFactory.java
----------------------------------------------------------------------
diff --git a/datamap/lucene/src/main/java/org/apache/carbondata/datamap/lucene/LuceneCoarseGrainDataMapFactory.java b/datamap/lucene/src/main/java/org/apache/carbondata/datamap/lucene/LuceneCoarseGrainDataMapFactory.java
new file mode 100644
index 0000000..7f9cc1c
--- /dev/null
+++ b/datamap/lucene/src/main/java/org/apache/carbondata/datamap/lucene/LuceneCoarseGrainDataMapFactory.java
@@ -0,0 +1,72 @@
+/*
+ * 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.lucene;
+
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.List;
+
+import org.apache.carbondata.common.annotations.InterfaceAudience;
+import org.apache.carbondata.common.logging.LogService;
+import org.apache.carbondata.common.logging.LogServiceFactory;
+import org.apache.carbondata.core.datamap.DataMapDistributable;
+import org.apache.carbondata.core.datamap.DataMapLevel;
+import org.apache.carbondata.core.datamap.dev.DataMapModel;
+import org.apache.carbondata.core.datamap.dev.cgdatamap.CoarseGrainDataMap;
+import org.apache.carbondata.core.memory.MemoryException;
+
+/**
+ * FG level of lucene DataMap
+ */
+@InterfaceAudience.Internal
+public class LuceneCoarseGrainDataMapFactory extends LuceneDataMapFactoryBase<CoarseGrainDataMap> {
+  private static final LogService LOGGER =
+      LogServiceFactory.getLogService(LuceneCoarseGrainDataMapFactory.class.getName());
+
+  /**
+   * Get the datamap for segmentid
+   */
+  public List<CoarseGrainDataMap> getDataMaps(String segmentId) throws IOException {
+    List<CoarseGrainDataMap> lstDataMap = new ArrayList<>();
+    CoarseGrainDataMap dataMap = new LuceneCoarseGrainDataMap(analyzer);
+    try {
+      dataMap.init(new DataMapModel(
+          LuceneDataMapWriter.genDataMapStorePath(
+              tableIdentifier.getTablePath(), segmentId, dataMapName)));
+    } catch (MemoryException e) {
+      LOGGER.error("failed to get lucene datamap , detail is {}" + e.getMessage());
+      return lstDataMap;
+    }
+    lstDataMap.add(dataMap);
+    return lstDataMap;
+  }
+
+  /**
+   * Get datamaps for distributable object.
+   */
+  public List<CoarseGrainDataMap> getDataMaps(DataMapDistributable distributable)
+      throws IOException {
+    return getDataMaps(distributable.getSegmentId());
+  }
+
+  @Override
+  public DataMapLevel getDataMapType() {
+    return DataMapLevel.CG;
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/carbondata/blob/c0133aac/datamap/lucene/src/main/java/org/apache/carbondata/datamap/lucene/LuceneDataMapDistributable.java
----------------------------------------------------------------------
diff --git a/datamap/lucene/src/main/java/org/apache/carbondata/datamap/lucene/LuceneDataMapDistributable.java b/datamap/lucene/src/main/java/org/apache/carbondata/datamap/lucene/LuceneDataMapDistributable.java
new file mode 100644
index 0000000..19e4035
--- /dev/null
+++ b/datamap/lucene/src/main/java/org/apache/carbondata/datamap/lucene/LuceneDataMapDistributable.java
@@ -0,0 +1,36 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.carbondata.datamap.lucene;
+
+import org.apache.carbondata.common.annotations.InterfaceAudience;
+import org.apache.carbondata.core.datamap.DataMapDistributable;
+
+@InterfaceAudience.Internal
+class LuceneDataMapDistributable extends DataMapDistributable {
+
+  // TODO: seems no one use this?
+  private String dataPath;
+
+  LuceneDataMapDistributable(String dataPath) {
+    this.dataPath = dataPath;
+  }
+
+  public String getDataPath() {
+    return dataPath;
+  }
+}

http://git-wip-us.apache.org/repos/asf/carbondata/blob/c0133aac/datamap/lucene/src/main/java/org/apache/carbondata/datamap/lucene/LuceneDataMapFactoryBase.java
----------------------------------------------------------------------
diff --git a/datamap/lucene/src/main/java/org/apache/carbondata/datamap/lucene/LuceneDataMapFactoryBase.java b/datamap/lucene/src/main/java/org/apache/carbondata/datamap/lucene/LuceneDataMapFactoryBase.java
new file mode 100644
index 0000000..5eb7054
--- /dev/null
+++ b/datamap/lucene/src/main/java/org/apache/carbondata/datamap/lucene/LuceneDataMapFactoryBase.java
@@ -0,0 +1,180 @@
+/*
+ * 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.lucene;
+
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.List;
+import java.util.Objects;
+
+import org.apache.carbondata.common.annotations.InterfaceAudience;
+import org.apache.carbondata.common.logging.LogService;
+import org.apache.carbondata.common.logging.LogServiceFactory;
+import org.apache.carbondata.core.datamap.DataMapDistributable;
+import org.apache.carbondata.core.datamap.DataMapMeta;
+import org.apache.carbondata.core.datamap.dev.DataMap;
+import org.apache.carbondata.core.datamap.dev.DataMapFactory;
+import org.apache.carbondata.core.datamap.dev.DataMapWriter;
+import org.apache.carbondata.core.metadata.AbsoluteTableIdentifier;
+import org.apache.carbondata.core.metadata.CarbonMetadata;
+import org.apache.carbondata.core.metadata.schema.table.CarbonTable;
+import org.apache.carbondata.core.metadata.schema.table.DataMapSchema;
+import org.apache.carbondata.core.metadata.schema.table.TableInfo;
+import org.apache.carbondata.core.metadata.schema.table.column.ColumnSchema;
+import org.apache.carbondata.core.scan.filter.intf.ExpressionType;
+import org.apache.carbondata.core.util.path.CarbonTablePath;
+import org.apache.carbondata.events.Event;
+
+import org.apache.lucene.analysis.Analyzer;
+import org.apache.lucene.analysis.standard.StandardAnalyzer;
+
+/**
+ * Base implementation for CG and FG lucene DataMapFactory.
+ */
+@InterfaceAudience.Internal
+abstract class LuceneDataMapFactoryBase<T extends DataMap> implements DataMapFactory<T> {
+
+  /**
+   * Logger
+   */
+  final LogService LOGGER = LogServiceFactory.getLogService(this.getClass().getName());
+
+  /**
+   * table's index columns
+   */
+  DataMapMeta dataMapMeta = null;
+
+  /**
+   * analyzer for lucene
+   */
+  Analyzer analyzer = null;
+
+  /**
+   * index name
+   */
+  String dataMapName = null;
+
+  /**
+   * table identifier
+   */
+  AbsoluteTableIdentifier tableIdentifier = null;
+
+  @Override
+  public void init(AbsoluteTableIdentifier identifier, DataMapSchema dataMapSchema)
+      throws IOException {
+    Objects.requireNonNull(identifier);
+    Objects.requireNonNull(dataMapSchema);
+
+    this.tableIdentifier = identifier;
+    this.dataMapName = dataMapSchema.getDataMapName();
+
+    // get carbonmetadata from carbonmetadata instance
+    CarbonMetadata carbonMetadata = CarbonMetadata.getInstance();
+
+    String tableUniqueName = identifier.getCarbonTableIdentifier().getTableUniqueName();
+
+    // get carbon table
+    CarbonTable carbonTable = carbonMetadata.getCarbonTable(tableUniqueName);
+    if (carbonTable == null) {
+      String errorMessage =
+          String.format("failed to get carbon table with name %s", tableUniqueName);
+      LOGGER.error(errorMessage);
+      throw new IOException(errorMessage);
+    }
+
+    TableInfo tableInfo = carbonTable.getTableInfo();
+    List<ColumnSchema> lstCoumnSchemas = tableInfo.getFactTable().getListOfColumns();
+
+    // currently add all columns into lucene indexer
+    // TODO:only add index columns
+    List<String> indexedColumns = new ArrayList<String>();
+    for (ColumnSchema columnSchema : lstCoumnSchemas) {
+      if (!columnSchema.isInvisible()) {
+        indexedColumns.add(columnSchema.getColumnName());
+      }
+    }
+
+    // get indexed columns
+    //    Map<String, String> properties = dataMapSchema.getProperties();
+    //    String columns = properties.get("text_column");
+    //    if (columns != null) {
+    //      String[] columnArray = columns.split(CarbonCommonConstants.COMMA, -1);
+    //      Collections.addAll(indexedColumns, columnArray);
+    //    }
+
+    // add optimizedOperations
+    List<ExpressionType> optimizedOperations = new ArrayList<ExpressionType>();
+    //    optimizedOperations.add(ExpressionType.EQUALS);
+    //    optimizedOperations.add(ExpressionType.GREATERTHAN);
+    //    optimizedOperations.add(ExpressionType.GREATERTHAN_EQUALTO);
+    //    optimizedOperations.add(ExpressionType.LESSTHAN);
+    //    optimizedOperations.add(ExpressionType.LESSTHAN_EQUALTO);
+    //    optimizedOperations.add(ExpressionType.NOT);
+    optimizedOperations.add(ExpressionType.TEXT_MATCH);
+    this.dataMapMeta = new DataMapMeta(indexedColumns, optimizedOperations);
+
+    // get analyzer
+    // TODO: how to get analyzer ?
+    analyzer = new StandardAnalyzer();
+  }
+
+  /**
+   * Return a new write for this datamap
+   */
+  public DataMapWriter createWriter(String segmentId, String writeDirectoryPath) {
+    LOGGER.info("lucene data write to " + writeDirectoryPath);
+    return new LuceneDataMapWriter(
+        tableIdentifier, dataMapName, segmentId, writeDirectoryPath, true);
+  }
+
+  /**
+   * Get all distributable objects of a segmentid
+   */
+  public List<DataMapDistributable> toDistributable(String segmentId) {
+    List<DataMapDistributable> lstDataMapDistribute = new ArrayList<DataMapDistributable>();
+    DataMapDistributable luceneDataMapDistributable = new LuceneDataMapDistributable(
+        CarbonTablePath.getSegmentPath(tableIdentifier.getTablePath(), segmentId));
+    lstDataMapDistribute.add(luceneDataMapDistributable);
+    return lstDataMapDistribute;
+  }
+
+  public void fireEvent(Event event) {
+
+  }
+
+  /**
+   * Clears datamap of the segment
+   */
+  public void clear(String segmentId) {
+
+  }
+
+  /**
+   * Clear all datamaps from memory
+   */
+  public void clear() {
+
+  }
+
+  /**
+   * Return metadata of this datamap
+   */
+  public DataMapMeta getMeta() {
+    return dataMapMeta;
+  }
+}

http://git-wip-us.apache.org/repos/asf/carbondata/blob/c0133aac/datamap/lucene/src/main/java/org/apache/carbondata/datamap/lucene/LuceneDataMapWriter.java
----------------------------------------------------------------------
diff --git a/datamap/lucene/src/main/java/org/apache/carbondata/datamap/lucene/LuceneDataMapWriter.java b/datamap/lucene/src/main/java/org/apache/carbondata/datamap/lucene/LuceneDataMapWriter.java
new file mode 100644
index 0000000..849fc2e
--- /dev/null
+++ b/datamap/lucene/src/main/java/org/apache/carbondata/datamap/lucene/LuceneDataMapWriter.java
@@ -0,0 +1,328 @@
+/*
+ * 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.lucene;
+
+import java.io.File;
+import java.io.IOException;
+import java.io.UnsupportedEncodingException;
+
+import org.apache.carbondata.common.annotations.InterfaceAudience;
+import org.apache.carbondata.common.logging.LogService;
+import org.apache.carbondata.common.logging.LogServiceFactory;
+import org.apache.carbondata.core.datamap.dev.DataMapWriter;
+import org.apache.carbondata.core.datastore.impl.FileFactory;
+import org.apache.carbondata.core.datastore.page.ColumnPage;
+import org.apache.carbondata.core.metadata.AbsoluteTableIdentifier;
+import org.apache.carbondata.core.metadata.datatype.DataType;
+import org.apache.carbondata.core.metadata.datatype.DataTypes;
+import org.apache.carbondata.core.util.path.CarbonTablePath;
+
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
+import org.apache.lucene.analysis.Analyzer;
+import org.apache.lucene.analysis.standard.StandardAnalyzer;
+import org.apache.lucene.document.Document;
+import org.apache.lucene.document.DoublePoint;
+import org.apache.lucene.document.Field;
+import org.apache.lucene.document.FloatPoint;
+import org.apache.lucene.document.IntPoint;
+import org.apache.lucene.document.IntRangeField;
+import org.apache.lucene.document.LongPoint;
+import org.apache.lucene.document.StoredField;
+import org.apache.lucene.document.StringField;
+import org.apache.lucene.document.TextField;
+import org.apache.lucene.index.IndexWriter;
+import org.apache.lucene.index.IndexWriterConfig;
+import org.apache.lucene.store.Directory;
+import org.apache.lucene.store.RAMDirectory;
+import org.apache.solr.store.hdfs.HdfsDirectory;
+
+/**
+ * Implementation to write lucene index while loading
+ */
+@InterfaceAudience.Internal
+public class LuceneDataMapWriter extends DataMapWriter {
+  /**
+   * logger
+   */
+  private static final LogService LOGGER =
+      LogServiceFactory.getLogService(LuceneDataMapWriter.class.getName());
+
+  /**
+   * index writer
+   */
+  private IndexWriter indexWriter = null;
+
+  private Analyzer analyzer = null;
+
+  private String blockId = null;
+
+  private String dataMapName = null;
+
+  private boolean isFineGrain = true;
+
+  private static final String BLOCKID_NAME = "blockId";
+
+  private static final String BLOCKLETID_NAME = "blockletId";
+
+  private static final String PAGEID_NAME = "pageId";
+
+  private static final String ROWID_NAME = "rowId";
+
+  LuceneDataMapWriter(AbsoluteTableIdentifier identifier, String dataMapName, String segmentId,
+      String writeDirectoryPath, boolean isFineGrain) {
+    super(identifier, segmentId, writeDirectoryPath);
+    this.dataMapName = dataMapName;
+    this.isFineGrain = isFineGrain;
+  }
+
+  private String getIndexPath() {
+    if (isFineGrain) {
+      return genDataMapStorePath(identifier.getTablePath(), segmentId, dataMapName);
+    } else {
+      // TODO: where write data in coarse grain data map
+      return genDataMapStorePath(identifier.getTablePath(), segmentId, dataMapName);
+    }
+  }
+
+  /**
+   * Start of new block notification.
+   */
+  public void onBlockStart(String blockId) throws IOException {
+    // save this block id for lucene index , used in onPageAdd function
+    this.blockId = blockId;
+
+    // get index path, put index data into segment's path
+    String strIndexPath = getIndexPath();
+    Path indexPath = FileFactory.getPath(strIndexPath);
+    FileSystem fs = FileFactory.getFileSystem(indexPath);
+
+    // if index path not exists, create it
+    if (fs.exists(indexPath)) {
+      fs.mkdirs(indexPath);
+    }
+
+    if (null == analyzer) {
+      analyzer = new StandardAnalyzer();
+    }
+
+    // create a index writer
+    Directory indexDir = new HdfsDirectory(indexPath, FileFactory.getConfiguration());
+    indexWriter = new IndexWriter(indexDir, new IndexWriterConfig(analyzer));
+
+  }
+
+  /**
+   * End of block notification
+   */
+  public void onBlockEnd(String blockId) throws IOException {
+    // clean this block id
+    this.blockId = null;
+
+    // finished a file , close this index writer
+    if (indexWriter != null) {
+      indexWriter.close();
+    }
+
+  }
+
+  /**
+   * Start of new blocklet notification.
+   */
+  public void onBlockletStart(int blockletId) {
+
+  }
+
+  /**
+   * End of blocklet notification
+   */
+  public void onBlockletEnd(int blockletId) {
+
+  }
+
+  /**
+   * 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.
+   */
+  public void onPageAdded(int blockletId, int pageId, ColumnPage[] pages) throws IOException {
+    // save index data into ram, write into disk after one page finished
+    RAMDirectory ramDir = new RAMDirectory();
+    IndexWriter ramIndexWriter = new IndexWriter(ramDir, new IndexWriterConfig(analyzer));
+
+    int columnsCount = pages.length;
+    if (columnsCount <= 0) {
+      LOGGER.warn("empty data");
+      ramIndexWriter.close();
+      ramDir.close();
+      return;
+    }
+    int pageSize = pages[0].getPageSize();
+    for (int rowId = 0; rowId < pageSize; rowId++) {
+      // create a new document
+      Document doc = new Document();
+
+      // add block id, save this id
+      doc.add(new StringField(BLOCKID_NAME, blockId, Field.Store.YES));
+
+      // add blocklet Id
+      doc.add(new IntPoint(BLOCKLETID_NAME, new int[] { blockletId }));
+      doc.add(new StoredField(BLOCKLETID_NAME, blockletId));
+      //doc.add(new NumericDocValuesField(BLOCKLETID_NAME,blockletId));
+
+      // add page id and row id in Fine Grain data map
+      if (isFineGrain) {
+        // add page Id
+        doc.add(new IntPoint(PAGEID_NAME, new int[] { pageId }));
+        doc.add(new StoredField(PAGEID_NAME, pageId));
+        //doc.add(new NumericDocValuesField(PAGEID_NAME,pageId));
+
+        // add row id
+        doc.add(new IntPoint(ROWID_NAME, new int[] { rowId }));
+        doc.add(new StoredField(ROWID_NAME, rowId));
+        //doc.add(new NumericDocValuesField(ROWID_NAME,rowId));
+      }
+
+      // add other fields
+      for (int colIdx = 0; colIdx < columnsCount; colIdx++) {
+        if (!pages[colIdx].getNullBits().get(rowId)) {
+          addField(doc, pages[colIdx], rowId, Field.Store.NO);
+        }
+      }
+
+      // add this document
+      ramIndexWriter.addDocument(doc);
+
+    }
+    // close ram writer
+    ramIndexWriter.close();
+
+    // add ram index data into disk
+    indexWriter.addIndexes(new Directory[] { ramDir });
+
+    // delete this ram data
+    ramDir.close();
+  }
+
+  private boolean addField(Document doc, ColumnPage page, int rowId, Field.Store store) {
+    //get field name
+    String fieldName = page.getColumnSpec().getFieldName();
+
+    //get field type
+    DataType type = page.getDataType();
+
+    if (type == DataTypes.BYTE) {
+      // byte type , use int range to deal with byte, lucene has no byte type
+      byte value = page.getByte(rowId);
+      IntRangeField field =
+          new IntRangeField(fieldName, new int[] { Byte.MIN_VALUE }, new int[] { Byte.MAX_VALUE });
+      field.setIntValue(value);
+      doc.add(field);
+
+      // if need store it , add StoredField
+      if (store == Field.Store.YES) {
+        doc.add(new StoredField(fieldName, (int) value));
+      }
+    } else if (type == DataTypes.SHORT) {
+      // short type , use int range to deal with short type, lucene has no short type
+      short value = page.getShort(rowId);
+      IntRangeField field = new IntRangeField(fieldName, new int[] { Short.MIN_VALUE },
+          new int[] { Short.MAX_VALUE });
+      field.setShortValue(value);
+      doc.add(field);
+
+      // if need store it , add StoredField
+      if (store == Field.Store.YES) {
+        doc.add(new StoredField(fieldName, (int) value));
+      }
+    } else if (type == DataTypes.INT) {
+      // int type , use int point to deal with int type
+      int value = page.getInt(rowId);
+      doc.add(new IntPoint(fieldName, new int[] { value }));
+
+      // if need store it , add StoredField
+      if (store == Field.Store.YES) {
+        doc.add(new StoredField(fieldName, value));
+      }
+    } else if (type == DataTypes.LONG) {
+      // long type , use long point to deal with long type
+      long value = page.getLong(rowId);
+      doc.add(new LongPoint(fieldName, new long[] { value }));
+
+      // if need store it , add StoredField
+      if (store == Field.Store.YES) {
+        doc.add(new StoredField(fieldName, value));
+      }
+    } else if (type == DataTypes.FLOAT) {
+      float value = page.getFloat(rowId);
+      doc.add(new FloatPoint(fieldName, new float[] { value }));
+      if (store == Field.Store.YES) {
+        doc.add(new FloatPoint(fieldName, value));
+      }
+    } else if (type == DataTypes.DOUBLE) {
+      double value = page.getDouble(rowId);
+      doc.add(new DoublePoint(fieldName, new double[] { value }));
+      if (store == Field.Store.YES) {
+        doc.add(new DoublePoint(fieldName, value));
+      }
+    } else if (type == DataTypes.STRING) {
+      byte[] value = page.getBytes(rowId);
+      // TODO: how to get string value
+      String strValue = null;
+      try {
+        strValue = new String(value, 2, value.length - 2, "UTF-8");
+      } catch (UnsupportedEncodingException e) {
+        throw new RuntimeException(e);
+      }
+      doc.add(new TextField(fieldName, strValue, store));
+    } else if (type == DataTypes.DATE) {
+      // TODO: how to get data value
+    } else if (type == DataTypes.TIMESTAMP) {
+      // TODO: how to get
+    } else if (type == DataTypes.BOOLEAN) {
+      boolean value = page.getBoolean(rowId);
+      IntRangeField field = new IntRangeField(fieldName, new int[] { 0 }, new int[] { 1 });
+      field.setIntValue(value ? 1 : 0);
+      doc.add(field);
+      if (store == Field.Store.YES) {
+        doc.add(new StoredField(fieldName, value ? 1 : 0));
+      }
+    } else {
+      LOGGER.error("unsupport data type " + type);
+      throw new RuntimeException("unsupported data type " + type);
+    }
+    return true;
+  }
+
+  /**
+   * This is called during closing of writer.So after this call no more data will be sent to this
+   * class.
+   */
+  public void finish() throws IOException {
+
+  }
+
+  /**
+   * Return store path for datamap
+   */
+  static String genDataMapStorePath(String tablePath, String segmentId, String dataMapName) {
+    return CarbonTablePath.getSegmentPath(tablePath, segmentId) + File.separator + dataMapName;
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/carbondata/blob/c0133aac/datamap/lucene/src/main/java/org/apache/carbondata/datamap/lucene/LuceneFineGrainDataMap.java
----------------------------------------------------------------------
diff --git a/datamap/lucene/src/main/java/org/apache/carbondata/datamap/lucene/LuceneFineGrainDataMap.java b/datamap/lucene/src/main/java/org/apache/carbondata/datamap/lucene/LuceneFineGrainDataMap.java
new file mode 100644
index 0000000..c649545
--- /dev/null
+++ b/datamap/lucene/src/main/java/org/apache/carbondata/datamap/lucene/LuceneFineGrainDataMap.java
@@ -0,0 +1,280 @@
+/*
+ * 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.lucene;
+
+import java.io.IOException;
+import java.util.*;
+
+import org.apache.carbondata.common.annotations.InterfaceAudience;
+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.fgdatamap.FineGrainBlocklet;
+import org.apache.carbondata.core.datamap.dev.fgdatamap.FineGrainDataMap;
+import org.apache.carbondata.core.datastore.block.SegmentProperties;
+import org.apache.carbondata.core.datastore.impl.FileFactory;
+import org.apache.carbondata.core.memory.MemoryException;
+import org.apache.carbondata.core.scan.expression.Expression;
+import org.apache.carbondata.core.scan.filter.intf.ExpressionType;
+import org.apache.carbondata.core.scan.filter.resolver.FilterResolverIntf;
+
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
+import org.apache.lucene.analysis.Analyzer;
+import org.apache.lucene.analysis.standard.StandardAnalyzer;
+import org.apache.lucene.document.Document;
+import org.apache.lucene.index.DirectoryReader;
+import org.apache.lucene.index.IndexReader;
+import org.apache.lucene.index.IndexableField;
+import org.apache.lucene.queryparser.classic.MultiFieldQueryParser;
+import org.apache.lucene.queryparser.classic.ParseException;
+import org.apache.lucene.queryparser.classic.QueryParser;
+import org.apache.lucene.search.*;
+import org.apache.lucene.store.Directory;
+import org.apache.solr.store.hdfs.HdfsDirectory;
+
+@InterfaceAudience.Internal
+public class LuceneFineGrainDataMap extends FineGrainDataMap {
+
+  private static final int BLOCKID_ID = 0;
+
+  private static final int BLOCKLETID_ID = 1;
+
+  private static final int PAGEID_ID = 2;
+
+  private static final int ROWID_ID = 3;
+
+  /**
+   * log information
+   */
+  private static final LogService LOGGER =
+      LogServiceFactory.getLogService(LuceneFineGrainDataMap.class.getName());
+
+  /**
+   * searcher object for this datamap
+   */
+  private IndexSearcher indexSearcher = null;
+
+  /**
+   * default max values to return
+   */
+  private static int MAX_RESULT_NUMBER = 100;
+
+  /**
+   * analyzer for lucene index
+   */
+  private Analyzer analyzer;
+
+  LuceneFineGrainDataMap(Analyzer analyzer) {
+    this.analyzer = analyzer;
+  }
+
+  /**
+   * It is called to load the data map to memory or to initialize it.
+   */
+  public void init(DataMapModel dataMapModel) throws MemoryException, IOException {
+    // get this path from file path
+    Path indexPath = FileFactory.getPath(dataMapModel.getFilePath());
+
+    LOGGER.info("Lucene index read path " + indexPath.toString());
+
+    // get file system , use hdfs file system , realized in solr project
+    FileSystem fs = FileFactory.getFileSystem(indexPath);
+
+    // check this path valid
+    if (!fs.exists(indexPath)) {
+      String errorMessage = String.format("index directory %s not exists.", indexPath);
+      LOGGER.error(errorMessage);
+      throw new IOException(errorMessage);
+    }
+
+    if (!fs.isDirectory(indexPath)) {
+      String errorMessage = String.format("error index path %s, must be directory", indexPath);
+      LOGGER.error(errorMessage);
+      throw new IOException(errorMessage);
+    }
+
+    // open this index path , use HDFS default configuration
+    Directory indexDir = new HdfsDirectory(indexPath, FileFactory.getConfiguration());
+
+    IndexReader indexReader = DirectoryReader.open(indexDir);
+    if (indexReader == null) {
+      throw new RuntimeException("failed to create index reader object");
+    }
+
+    // create a index searcher object
+    indexSearcher = new IndexSearcher(indexReader);
+  }
+
+  /**
+   * Return the query string in the first TEXT_MATCH expression in the expression tree
+   */
+  private String getQueryString(Expression expression) {
+    if (expression.getFilterExpressionType() == ExpressionType.TEXT_MATCH) {
+      return expression.getString();
+    }
+
+    for (Expression child : expression.getChildren()) {
+      String queryString = getQueryString(child);
+      if (queryString != null) {
+        return queryString;
+      }
+    }
+    return null;
+  }
+
+  /**
+   * Prune the datamap with filter expression. It returns the list of
+   * blocklets where these filters can exist.
+   */
+  @Override
+  public List<FineGrainBlocklet> prune(FilterResolverIntf filterExp,
+      SegmentProperties segmentProperties, List<String> partitions) throws IOException {
+
+    // convert filter expr into lucene list query
+    List<String> fields = new ArrayList<String>();
+
+    // only for test , query all data
+    String strQuery = getQueryString(filterExp.getFilterExpression());
+
+    String[] sFields = new String[fields.size()];
+    fields.toArray(sFields);
+
+    // get analyzer
+    if (analyzer == null) {
+      analyzer = new StandardAnalyzer();
+    }
+
+    // use MultiFieldQueryParser to parser query
+    QueryParser queryParser = new MultiFieldQueryParser(sFields, analyzer);
+    Query query;
+    try {
+      query = queryParser.parse(strQuery);
+    } catch (ParseException e) {
+      String errorMessage = String.format(
+          "failed to filter block with query %s, detail is %s", strQuery, e.getMessage());
+      LOGGER.error(errorMessage);
+      return null;
+    }
+
+    // execute index search
+    TopDocs result;
+    try {
+      result = indexSearcher.search(query, MAX_RESULT_NUMBER);
+    } catch (IOException e) {
+      String errorMessage =
+          String.format("failed to search lucene data, detail is %s", e.getMessage());
+      LOGGER.error(errorMessage);
+      throw new IOException(errorMessage);
+    }
+
+    // temporary data, delete duplicated data
+    // Map<BlockId, Map<BlockletId, Map<PageId, Set<RowId>>>>
+    Map<String, Map<String, Map<Integer, Set<Integer>>>> mapBlocks = new HashMap<>();
+
+    for (ScoreDoc scoreDoc : result.scoreDocs) {
+      // get a document
+      Document doc = indexSearcher.doc(scoreDoc.doc);
+
+      // get all fields
+      List<IndexableField> fieldsInDoc = doc.getFields();
+
+      // get this block id Map<BlockId, Map<BlockletId, Map<PageId, Set<RowId>>>>
+      String blockId = fieldsInDoc.get(BLOCKID_ID).stringValue();
+      Map<String, Map<Integer, Set<Integer>>> mapBlocklets = mapBlocks.get(blockId);
+      if (mapBlocklets == null) {
+        mapBlocklets = new HashMap<>();
+        mapBlocks.put(blockId, mapBlocklets);
+      }
+
+      // get the blocklet id Map<BlockletId, Map<PageId, Set<RowId>>>
+      String blockletId = fieldsInDoc.get(BLOCKLETID_ID).stringValue();
+      Map<Integer, Set<Integer>> mapPageIds = mapBlocklets.get(blockletId);
+      if (mapPageIds == null) {
+        mapPageIds = new HashMap<>();
+        mapBlocklets.put(blockletId, mapPageIds);
+      }
+
+      // get the page id Map<PageId, Set<RowId>>
+      Number pageId = fieldsInDoc.get(PAGEID_ID).numericValue();
+      Set<Integer> setRowId = mapPageIds.get(pageId.intValue());
+      if (setRowId == null) {
+        setRowId = new HashSet<>();
+        mapPageIds.put(pageId.intValue(), setRowId);
+      }
+
+      // get the row id Set<RowId>
+      Number rowId = fieldsInDoc.get(ROWID_ID).numericValue();
+      setRowId.add(rowId.intValue());
+    }
+
+    // result blocklets
+    List<FineGrainBlocklet> blocklets = new ArrayList<>();
+
+    // transform all blocks into result type blocklets
+    // Map<BlockId, Map<BlockletId, Map<PageId, Set<RowId>>>>
+    for (Map.Entry<String, Map<String, Map<Integer, Set<Integer>>>> mapBlock :
+        mapBlocks.entrySet()) {
+      String blockId = mapBlock.getKey();
+      Map<String, Map<Integer, Set<Integer>>> mapBlocklets = mapBlock.getValue();
+      // for blocklets in this block Map<BlockletId, Map<PageId, Set<RowId>>>
+      for (Map.Entry<String, Map<Integer, Set<Integer>>> mapBlocklet : mapBlocklets.entrySet()) {
+        String blockletId = mapBlocklet.getKey();
+        Map<Integer, Set<Integer>> mapPageIds = mapBlocklet.getValue();
+        List<FineGrainBlocklet.Page> pages = new ArrayList<FineGrainBlocklet.Page>();
+
+        // for pages in this blocklet Map<PageId, Set<RowId>>>
+        for (Map.Entry<Integer, Set<Integer>> mapPageId : mapPageIds.entrySet()) {
+          // construct array rowid
+          int[] rowIds = new int[mapPageId.getValue().size()];
+          int i = 0;
+          // for rowids in this page Set<RowId>
+          for (Integer rowid : mapPageId.getValue()) {
+            rowIds[i++] = rowid;
+          }
+          // construct one page
+          FineGrainBlocklet.Page page = new FineGrainBlocklet.Page();
+          page.setPageId(mapPageId.getKey());
+          page.setRowId(rowIds);
+
+          // add this page into list pages
+          pages.add(page);
+        }
+
+        // add a FineGrainBlocklet
+        blocklets.add(new FineGrainBlocklet(blockId, blockletId, pages));
+      }
+    }
+
+    return blocklets;
+  }
+
+  @Override
+  public boolean isScanRequired(FilterResolverIntf filterExp) {
+    return true;
+  }
+
+  /**
+   * Clear complete index table and release memory.
+   */
+  @Override
+  public void clear() {
+
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/carbondata/blob/c0133aac/datamap/lucene/src/main/java/org/apache/carbondata/datamap/lucene/LuceneFineGrainDataMapFactory.java
----------------------------------------------------------------------
diff --git a/datamap/lucene/src/main/java/org/apache/carbondata/datamap/lucene/LuceneFineGrainDataMapFactory.java b/datamap/lucene/src/main/java/org/apache/carbondata/datamap/lucene/LuceneFineGrainDataMapFactory.java
new file mode 100644
index 0000000..e35d5bf
--- /dev/null
+++ b/datamap/lucene/src/main/java/org/apache/carbondata/datamap/lucene/LuceneFineGrainDataMapFactory.java
@@ -0,0 +1,68 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.carbondata.datamap.lucene;
+
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.List;
+
+import org.apache.carbondata.common.annotations.InterfaceAudience;
+import org.apache.carbondata.core.datamap.DataMapDistributable;
+import org.apache.carbondata.core.datamap.DataMapLevel;
+import org.apache.carbondata.core.datamap.dev.DataMapModel;
+import org.apache.carbondata.core.datamap.dev.fgdatamap.FineGrainDataMap;
+import org.apache.carbondata.core.memory.MemoryException;
+
+/**
+ * CG level of lucene DataMap
+ */
+@InterfaceAudience.Internal
+public class LuceneFineGrainDataMapFactory extends LuceneDataMapFactoryBase<FineGrainDataMap> {
+
+  /**
+   * Get the datamap for segmentid
+   */
+  public List<FineGrainDataMap> getDataMaps(String segmentId) throws IOException {
+    List<FineGrainDataMap> lstDataMap = new ArrayList<>();
+    FineGrainDataMap dataMap = new LuceneFineGrainDataMap(analyzer);
+    try {
+      dataMap.init(new DataMapModel(
+          LuceneDataMapWriter.genDataMapStorePath(
+              tableIdentifier.getTablePath(), segmentId, dataMapName)));
+    } catch (MemoryException e) {
+      LOGGER.error("failed to get lucene datamap , detail is {}" + e.getMessage());
+      return lstDataMap;
+    }
+    lstDataMap.add(dataMap);
+    return lstDataMap;
+  }
+
+  /**
+   * Get datamaps for distributable object.
+   */
+  public List<FineGrainDataMap> getDataMaps(DataMapDistributable distributable)
+      throws IOException {
+    return getDataMaps(distributable.getSegmentId());
+  }
+
+  @Override
+  public DataMapLevel getDataMapType() {
+    return DataMapLevel.FG;
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/carbondata/blob/c0133aac/datamap/lucene/src/test/scala/org/apache/carbondata/datamap/lucene/LuceneCoarseGrainDataMapSuite.scala
----------------------------------------------------------------------
diff --git a/datamap/lucene/src/test/scala/org/apache/carbondata/datamap/lucene/LuceneCoarseGrainDataMapSuite.scala b/datamap/lucene/src/test/scala/org/apache/carbondata/datamap/lucene/LuceneCoarseGrainDataMapSuite.scala
new file mode 100644
index 0000000..a461f04
--- /dev/null
+++ b/datamap/lucene/src/test/scala/org/apache/carbondata/datamap/lucene/LuceneCoarseGrainDataMapSuite.scala
@@ -0,0 +1,73 @@
+/*
+ * 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.lucene
+
+import org.apache.spark.sql.test.util.QueryTest
+import org.scalatest.BeforeAndAfterAll
+
+import org.apache.carbondata.core.datamap.DataMapStoreManager
+import org.apache.carbondata.core.metadata.CarbonMetadata
+import org.apache.carbondata.core.metadata.schema.table.DataMapSchema
+
+class LuceneCoarseGrainDataMapSuite extends QueryTest with BeforeAndAfterAll {
+
+  val file2 = resourcesPath + "/datamap_input.csv"
+
+  override protected def beforeAll(): Unit = {
+    //n should be about 5000000 of reset if size is default 1024
+    val n = 15000
+    LuceneFineGrainDataMapSuite.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 lucene coarse grain data map") {
+    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)
+
+    sql(
+      s"""
+         | CREATE DATAMAP dm ON TABLE datamap_test
+         | USING '${classOf[LuceneCoarseGrainDataMapFactory].getName}'
+      """.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'"))
+  }
+
+  override protected def afterAll(): Unit = {
+    LuceneFineGrainDataMapSuite.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/c0133aac/datamap/lucene/src/test/scala/org/apache/carbondata/datamap/lucene/LuceneFineGrainDataMapSuite.scala
----------------------------------------------------------------------
diff --git a/datamap/lucene/src/test/scala/org/apache/carbondata/datamap/lucene/LuceneFineGrainDataMapSuite.scala b/datamap/lucene/src/test/scala/org/apache/carbondata/datamap/lucene/LuceneFineGrainDataMapSuite.scala
new file mode 100644
index 0000000..4766281
--- /dev/null
+++ b/datamap/lucene/src/test/scala/org/apache/carbondata/datamap/lucene/LuceneFineGrainDataMapSuite.scala
@@ -0,0 +1,98 @@
+/*
+ * 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.lucene
+
+import java.io.{File, PrintWriter}
+
+import scala.util.Random
+
+import org.apache.spark.sql.test.util.QueryTest
+import org.scalatest.BeforeAndAfterAll
+
+import org.apache.carbondata.core.metadata.CarbonMetadata
+
+class LuceneFineGrainDataMapSuite extends QueryTest with BeforeAndAfterAll {
+
+  val file2 = resourcesPath + "/datamap_input.csv"
+
+  override protected def beforeAll(): Unit = {
+    //n should be about 5000000 of reset if size is default 1024
+    val n = 15000
+    LuceneFineGrainDataMapSuite.createFile(file2)
+    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 lucene fine grain data map") {
+    sql("DROP TABLE IF EXISTS datamap_test")
+    sql(
+      """
+        | CREATE TABLE datamap_test(id INT, name STRING, city STRING, age INT)
+        | STORED BY 'carbondata'
+        | TBLPROPERTIES('SORT_COLUMNS'='city,name', 'SORT_SCOPE'='LOCAL_SORT')
+      """.stripMargin)
+
+    sql(
+      s"""
+         | CREATE DATAMAP dm ON TABLE datamap_test
+         | USING 'org.apache.carbondata.datamap.lucene.LuceneFineGrainDataMapFactory'
+      """.stripMargin)
+
+    sql(s"LOAD DATA LOCAL INPATH '$file2' INTO TABLE datamap_test OPTIONS('header'='false')")
+
+    sql("SELECT * FROM datamap_test ORDER BY id").show
+
+    //    sql("select * from normal_test where name='n34000'").show
+    sql("SELECT * FROM datamap_test WHERE TEXT_MATCH('name:n10')").show
+    sql("SELECT * FROM datamap_test WHERE TEXT_MATCH('name:n10*')").show
+    sql("SELECT * FROM datamap_test WHERE TEXT_MATCH('city:c020')").show
+
+    //    checkAnswer(
+    //      sql("select * from datamap_test where match('name:n34000')"),
+    //      sql("select * from normal_test where name='n34000'"))
+  }
+
+  override protected def afterAll(): Unit = {
+    LuceneFineGrainDataMapSuite.deleteFile(file2)
+    sql("DROP TABLE IF EXISTS normal_test")
+    sql("DROP TABLE IF EXISTS datamap_test")
+  }
+}
+
+object LuceneFineGrainDataMapSuite {
+  def createFile(fileName: String, line: Int = 10000, start: Int = 0) = {
+    val write = new PrintWriter(new File(fileName))
+    for (i <- start until (start + line)) {
+      write.println(i + "," + "n" + i + "," + "c0" + i + "," + Random.nextInt(80))
+    }
+    write.close()
+  }
+
+  def deleteFile(fileName: String): Unit = {
+      val file = new File(fileName)
+      if (file.exists()) {
+        file.delete()
+      }
+  }
+}

http://git-wip-us.apache.org/repos/asf/carbondata/blob/c0133aac/integration/spark-common-test/pom.xml
----------------------------------------------------------------------
diff --git a/integration/spark-common-test/pom.xml b/integration/spark-common-test/pom.xml
index d1c04ae..b7f19fd 100644
--- a/integration/spark-common-test/pom.xml
+++ b/integration/spark-common-test/pom.xml
@@ -99,6 +99,12 @@
       <scope>test</scope>
     </dependency>
     <dependency>
+      <groupId>org.apache.carbondata</groupId>
+      <artifactId>carbondata-lucene</artifactId>
+      <version>${project.version}</version>
+      <scope>test</scope>
+    </dependency>
+    <dependency>
       <groupId>org.apache.spark</groupId>
       <artifactId>spark-hive-thriftserver_${scala.binary.version}</artifactId>
       <scope>test</scope>

http://git-wip-us.apache.org/repos/asf/carbondata/blob/c0133aac/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
index 977b31d..fae87a5 100644
--- 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
@@ -246,7 +246,7 @@ class FGDataMap extends FineGrainDataMap {
    * Clear complete index table and release memory.
    */
   override def clear():Unit = {
-    ???
+
   }
 
   override def isScanRequired(filterExp: FilterResolverIntf): Boolean = ???

http://git-wip-us.apache.org/repos/asf/carbondata/blob/c0133aac/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
index 10955a3..ea571d7 100644
--- a/integration/spark2/src/main/java/org/apache/carbondata/datamap/DataMapProvider.java
+++ b/integration/spark2/src/main/java/org/apache/carbondata/datamap/DataMapProvider.java
@@ -17,6 +17,8 @@
 
 package org.apache.carbondata.datamap;
 
+import java.io.IOException;
+
 import org.apache.carbondata.common.annotations.InterfaceAudience;
 import org.apache.carbondata.common.exceptions.sql.MalformedDataMapCommandException;
 import org.apache.carbondata.core.metadata.schema.table.CarbonTable;
@@ -62,7 +64,7 @@ public interface DataMapProvider {
    * Implementation should initialize metadata for datamap, like creating table
    */
   void initMeta(CarbonTable mainTable, DataMapSchema dataMapSchema, String ctasSqlStatement,
-      SparkSession sparkSession) throws MalformedDataMapCommandException;
+      SparkSession sparkSession) throws MalformedDataMapCommandException, IOException;
 
   /**
    * Initialize a datamap's data.

http://git-wip-us.apache.org/repos/asf/carbondata/blob/c0133aac/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
index 2a6a70a..c7651bb 100644
--- a/integration/spark2/src/main/java/org/apache/carbondata/datamap/IndexDataMapProvider.java
+++ b/integration/spark2/src/main/java/org/apache/carbondata/datamap/IndexDataMapProvider.java
@@ -17,6 +17,8 @@
 
 package org.apache.carbondata.datamap;
 
+import java.io.IOException;
+
 import org.apache.carbondata.common.annotations.InterfaceAudience;
 import org.apache.carbondata.common.exceptions.MetadataProcessException;
 import org.apache.carbondata.common.exceptions.sql.MalformedDataMapCommandException;
@@ -37,7 +39,7 @@ public class IndexDataMapProvider implements DataMapProvider {
 
   @Override
   public void initMeta(CarbonTable mainTable, DataMapSchema dataMapSchema, String ctasSqlStatement,
-      SparkSession sparkSession) throws MalformedDataMapCommandException {
+      SparkSession sparkSession) throws MalformedDataMapCommandException, IOException {
     DataMapFactory dataMapFactory = createIndexDataMapFactory(dataMapSchema);
     DataMapStoreManager.getInstance().registerDataMap(
         mainTable.getAbsoluteTableIdentifier(), dataMapSchema, dataMapFactory);

http://git-wip-us.apache.org/repos/asf/carbondata/blob/c0133aac/integration/spark2/src/main/java/org/apache/carbondata/datamap/expression/MatchExpression.java
----------------------------------------------------------------------
diff --git a/integration/spark2/src/main/java/org/apache/carbondata/datamap/expression/MatchExpression.java b/integration/spark2/src/main/java/org/apache/carbondata/datamap/expression/MatchExpression.java
new file mode 100644
index 0000000..fceb729
--- /dev/null
+++ b/integration/spark2/src/main/java/org/apache/carbondata/datamap/expression/MatchExpression.java
@@ -0,0 +1,56 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.carbondata.datamap.expression;
+
+import org.apache.carbondata.common.annotations.InterfaceAudience;
+import org.apache.carbondata.core.scan.expression.Expression;
+import org.apache.carbondata.core.scan.expression.ExpressionResult;
+import org.apache.carbondata.core.scan.expression.exception.FilterIllegalMemberException;
+import org.apache.carbondata.core.scan.expression.exception.FilterUnsupportedException;
+import org.apache.carbondata.core.scan.filter.intf.ExpressionType;
+import org.apache.carbondata.core.scan.filter.intf.RowIntf;
+
+@InterfaceAudience.Internal
+public class MatchExpression extends Expression {
+  private String queryString;
+
+  public MatchExpression(String queryString) {
+    this.queryString = queryString;
+  }
+
+  @Override
+  public ExpressionResult evaluate(RowIntf value)
+      throws FilterUnsupportedException, FilterIllegalMemberException {
+    return null;
+  }
+
+  @Override
+  public ExpressionType getFilterExpressionType() {
+    return ExpressionType.TEXT_MATCH;
+  }
+
+  @Override
+  public void findAndSetChild(Expression oldExpr, Expression newExpr) {
+
+  }
+
+  @Override
+  public String getString() {
+    return queryString;
+  }
+}

http://git-wip-us.apache.org/repos/asf/carbondata/blob/c0133aac/integration/spark2/src/main/scala/org/apache/carbondata/datamap/TextMatchUDF.scala
----------------------------------------------------------------------
diff --git a/integration/spark2/src/main/scala/org/apache/carbondata/datamap/TextMatchUDF.scala b/integration/spark2/src/main/scala/org/apache/carbondata/datamap/TextMatchUDF.scala
new file mode 100644
index 0000000..093e479
--- /dev/null
+++ b/integration/spark2/src/main/scala/org/apache/carbondata/datamap/TextMatchUDF.scala
@@ -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.datamap
+
+import org.apache.spark.sql.sources.Filter
+
+import org.apache.carbondata.common.annotations.InterfaceAudience
+
+@InterfaceAudience.Internal
+class TextMatchUDF extends ((String) => Boolean) with Serializable {
+  override def apply(v1: String): Boolean = {
+    v1.length > 0
+  }
+}
+
+@InterfaceAudience.Internal
+case class TextMatch(queryString: String) extends Filter {
+  override def references: Array[String] = null
+}

http://git-wip-us.apache.org/repos/asf/carbondata/blob/c0133aac/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 9315208..8c3ca0f 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
@@ -32,6 +32,7 @@ import org.apache.carbondata.core.datastore.impl.FileFactory
 import org.apache.carbondata.core.metadata.AbsoluteTableIdentifier
 import org.apache.carbondata.core.metadata.schema.table.CarbonTable
 import org.apache.carbondata.core.util._
+import org.apache.carbondata.datamap.TextMatchUDF
 import org.apache.carbondata.events._
 import org.apache.carbondata.processing.loading.events.LoadEvents.{LoadMetadataEvent, LoadTablePostStatusUpdateEvent, LoadTablePreExecutionEvent, LoadTablePreStatusUpdateEvent}
 import org.apache.carbondata.spark.rdd.SparkReadSupport
@@ -66,6 +67,10 @@ class CarbonEnv {
     // column to sum and count.
     sparkSession.udf.register("preAggLoad", () => "")
 
+    // register for lucene datamap
+    // TODO: move it to proper place, it should be registered by datamap implementation
+    sparkSession.udf.register("text_match", new TextMatchUDF)
+
     // added for handling timeseries function like hour, minute, day , month , year
     sparkSession.udf.register("timeseries", new TimeSeriesFunction)
     // acquiring global level lock so global configuration will be updated by only one thread

http://git-wip-us.apache.org/repos/asf/carbondata/blob/c0133aac/integration/spark2/src/main/scala/org/apache/spark/sql/execution/strategy/CarbonLateDecodeStrategy.scala
----------------------------------------------------------------------
diff --git a/integration/spark2/src/main/scala/org/apache/spark/sql/execution/strategy/CarbonLateDecodeStrategy.scala b/integration/spark2/src/main/scala/org/apache/spark/sql/execution/strategy/CarbonLateDecodeStrategy.scala
index e82c9d7..47da9a5 100644
--- a/integration/spark2/src/main/scala/org/apache/spark/sql/execution/strategy/CarbonLateDecodeStrategy.scala
+++ b/integration/spark2/src/main/scala/org/apache/spark/sql/execution/strategy/CarbonLateDecodeStrategy.scala
@@ -35,12 +35,14 @@ import org.apache.spark.sql.sources.{BaseRelation, Filter}
 import org.apache.spark.sql.types._
 import org.apache.spark.sql.CarbonExpressions.{MatchCast => Cast}
 
+import org.apache.carbondata.common.exceptions.sql.MalformedCarbonCommandException
 import org.apache.carbondata.core.constants.CarbonCommonConstants
 import org.apache.carbondata.core.indexstore.PartitionSpec
 import org.apache.carbondata.core.metadata.schema.BucketingInfo
 import org.apache.carbondata.core.metadata.schema.table.CarbonTable
 import org.apache.carbondata.core.statusmanager.SegmentUpdateStatusManager
 import org.apache.carbondata.core.util.CarbonProperties
+import org.apache.carbondata.datamap.{TextMatch, TextMatchUDF}
 import org.apache.carbondata.spark.CarbonAliasDecoderRelation
 import org.apache.carbondata.spark.rdd.CarbonScanRDD
 import org.apache.carbondata.spark.util.CarbonScalaUtil
@@ -527,6 +529,13 @@ private[sql] class CarbonLateDecodeStrategy extends SparkStrategy {
    */
   protected[sql] def translateFilter(predicate: Expression, or: Boolean = false): Option[Filter] = {
     predicate match {
+      case u: ScalaUDF if u.function.isInstanceOf[TextMatchUDF] =>
+        if (u.children.size > 1) {
+          throw new MalformedCarbonCommandException(
+            "TEXT_MATCH UDF syntax: TEXT_MATCH('luceneQuerySyntax')")
+        }
+        Some(TextMatch(u.children.head.toString()))
+
       case or@Or(left, right) =>
 
         val leftFilter = translateFilter(left, true)

http://git-wip-us.apache.org/repos/asf/carbondata/blob/c0133aac/integration/spark2/src/main/scala/org/apache/spark/sql/optimizer/CarbonFilters.scala
----------------------------------------------------------------------
diff --git a/integration/spark2/src/main/scala/org/apache/spark/sql/optimizer/CarbonFilters.scala b/integration/spark2/src/main/scala/org/apache/spark/sql/optimizer/CarbonFilters.scala
index 38c5146..0aedd40 100644
--- a/integration/spark2/src/main/scala/org/apache/spark/sql/optimizer/CarbonFilters.scala
+++ b/integration/spark2/src/main/scala/org/apache/spark/sql/optimizer/CarbonFilters.scala
@@ -45,6 +45,8 @@ import org.apache.carbondata.core.scan.filter.intf.ExpressionType
 import org.apache.carbondata.core.util.ThreadLocalSessionInfo
 import org.apache.carbondata.core.util.path.CarbonTablePath
 import org.apache.carbondata.core.util.CarbonProperties
+import org.apache.carbondata.datamap.TextMatch
+import org.apache.carbondata.datamap.expression.MatchExpression
 import org.apache.carbondata.spark.CarbonAliasDecoderRelation
 import org.apache.carbondata.spark.util.CarbonScalaUtil
 
@@ -140,6 +142,8 @@ object CarbonFilters {
           Some(transformExpression(expr))
         case FalseExpr() =>
           Some(new FalseExpression(null))
+        case TextMatch(queryString) =>
+          Some(new MatchExpression(queryString))
         case _ => None
       }
     }

http://git-wip-us.apache.org/repos/asf/carbondata/blob/c0133aac/pom.xml
----------------------------------------------------------------------
diff --git a/pom.xml b/pom.xml
index 0891972..fdc9210 100644
--- a/pom.xml
+++ b/pom.xml
@@ -342,6 +342,7 @@
         <artifactId>findbugs-maven-plugin</artifactId>
         <version>3.0.4</version>
         <configuration>
+          <skip>true</skip>
           <excludeFilterFile>${dev.path}/findbugs-exclude.xml</excludeFilterFile>
           <failOnError>true</failOnError>
           <findbugsXmlOutput>true</findbugsXmlOutput>
@@ -481,6 +482,7 @@
         <module>streaming</module>
         <module>examples/spark2</module>
         <module>datamap/examples</module>
+        <module>datamap/lucene</module>
       </modules>
       <build>
         <plugins>
@@ -535,6 +537,7 @@
         <module>integration/presto</module>
         <module>streaming</module>
         <module>examples/spark2</module>
+        <module>datamap/lucene</module>
       </modules>
       <build>
         <plugins>

http://git-wip-us.apache.org/repos/asf/carbondata/blob/c0133aac/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 22a273b..2e39c91 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
@@ -85,7 +85,7 @@ public class DataMapWriterListener {
     LOG.info("DataMapWriter " + writer + " added");
   }
 
-  public void onBlockStart(String blockId, String blockPath) {
+  public void onBlockStart(String blockId, String blockPath) throws IOException {
     for (List<DataMapWriter> writers : registry.values()) {
       for (DataMapWriter writer : writers) {
         writer.onBlockStart(blockId);
@@ -93,7 +93,7 @@ public class DataMapWriterListener {
     }
   }
 
-  public void onBlockEnd(String blockId) {
+  public void onBlockEnd(String blockId) throws IOException {
     for (List<DataMapWriter> writers : registry.values()) {
       for (DataMapWriter writer : writers) {
         writer.onBlockEnd(blockId);
@@ -123,7 +123,7 @@ public class DataMapWriterListener {
    * @param pageId     sequence number of page, start from 0
    * @param tablePage  page data
    */
-  public void onPageAdded(int blockletId, int pageId, TablePage tablePage) {
+  public void onPageAdded(int blockletId, int pageId, TablePage tablePage) throws IOException {
     Set<Map.Entry<List<String>, List<DataMapWriter>>> entries = registry.entrySet();
     for (Map.Entry<List<String>, List<DataMapWriter>> entry : entries) {
       List<String> indexedColumns = entry.getKey();

http://git-wip-us.apache.org/repos/asf/carbondata/blob/c0133aac/processing/src/main/java/org/apache/carbondata/processing/store/writer/AbstractFactDataWriter.java
----------------------------------------------------------------------
diff --git a/processing/src/main/java/org/apache/carbondata/processing/store/writer/AbstractFactDataWriter.java b/processing/src/main/java/org/apache/carbondata/processing/store/writer/AbstractFactDataWriter.java
index 8d26ad2..4064c0d 100644
--- a/processing/src/main/java/org/apache/carbondata/processing/store/writer/AbstractFactDataWriter.java
+++ b/processing/src/main/java/org/apache/carbondata/processing/store/writer/AbstractFactDataWriter.java
@@ -228,13 +228,21 @@ public abstract class AbstractFactDataWriter implements CarbonFactDataWriter {
 
   private void notifyDataMapBlockStart() {
     if (listener != null) {
-      listener.onBlockStart(carbonDataFileName, constructFactFileFullPath());
+      try {
+        listener.onBlockStart(carbonDataFileName, constructFactFileFullPath());
+      } catch (IOException e) {
+        throw new CarbonDataWriterException("Problem while writing datamap", e);
+      }
     }
   }
 
   private void notifyDataMapBlockEnd() {
     if (listener != null) {
-      listener.onBlockEnd(carbonDataFileName);
+      try {
+        listener.onBlockEnd(carbonDataFileName);
+      } catch (IOException e) {
+        throw new CarbonDataWriterException("Problem while writing datamap", e);
+      }
     }
     blockletId = 0;
   }


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

Posted by ja...@apache.org.
http://git-wip-us.apache.org/repos/asf/carbondata/blob/f8ded96e/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..34e11ac
--- /dev/null
+++ b/core/src/main/java/org/apache/carbondata/core/indexstore/blockletindex/BlockletIndexDataMap.java
@@ -0,0 +1,971 @@
+/*
+ * 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.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.PartitionSpec;
+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.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.CarbonUtil;
+import org.apache.carbondata.core.util.DataFileFooterConverter;
+import org.apache.carbondata.core.util.DataTypeUtil;
+import org.apache.carbondata.core.util.path.CarbonTablePath;
+
+import org.apache.commons.lang3.StringUtils;
+import org.apache.hadoop.fs.Path;
+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 INDEX_PATH = 3;
+
+  private static int INDEX_FILE_NAME = 4;
+
+  private static int SEGMENTID = 5;
+
+  private UnsafeMemoryDMStore unsafeMemoryDMStore;
+
+  private UnsafeMemoryDMStore unsafeMemorySummaryDMStore;
+
+  private SegmentProperties segmentProperties;
+
+  private int[] columnCardinality;
+
+  @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());
+    Path path = new Path(blockletDataMapInfo.getFilePath());
+    byte[] filePath = path.getParent().toString().getBytes(CarbonCommonConstants.DEFAULT_CHARSET);
+    byte[] fileName = path.getName().toString().getBytes(CarbonCommonConstants.DEFAULT_CHARSET);
+    byte[] segmentId =
+        blockletDataMapInfo.getSegmentId().getBytes(CarbonCommonConstants.DEFAULT_CHARSET);
+    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, schemaBinary, filePath, fileName,
+            segmentId);
+      }
+      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,
+          schemaBinary,
+          filePath,
+          fileName,
+          segmentId);
+      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);
+    byte[][] maxValues = updateMaxValues(minMaxIndex.getMaxValues(), minMaxLen);
+    // update min max values in case of old store
+    byte[][] updatedMinValues =
+        CarbonUtil.updateMinMaxValues(fileFooter, maxValues, minValues, true);
+    byte[][] updatedMaxValues =
+        CarbonUtil.updateMinMaxValues(fileFooter, maxValues, minValues, false);
+    row.setRow(addMinMax(minMaxLen, schema[ordinal], updatedMinValues), ordinal);
+    // compute and set task level min values
+    addTaskMinMaxValues(summaryRow, minMaxLen,
+        unsafeMemorySummaryDMStore.getSchema()[taskMinMaxOrdinal], updatedMinValues,
+        TASK_MIN_VALUES_INDEX, true);
+    ordinal++;
+    taskMinMaxOrdinal++;
+    row.setRow(addMinMax(minMaxLen, schema[ordinal], updatedMaxValues), ordinal);
+    // compute and set task level max values
+    addTaskMinMaxValues(summaryRow, minMaxLen,
+        unsafeMemorySummaryDMStore.getSchema()[taskMinMaxOrdinal], updatedMaxValues,
+        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, byte[] schemaBinary,
+      byte[] filePath, byte[] fileName, byte[] segmentId) {
+    // 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);
+      }
+      summaryRow.setByteArray(filePath, INDEX_PATH);
+      summaryRow.setByteArray(fileName, INDEX_FILE_NAME);
+      summaryRow.setByteArray(segmentId, SEGMENTID);
+      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
+   * @throws MemoryException
+   */
+  private void createSummarySchema(SegmentProperties segmentProperties, byte[] schemaBinary,
+      byte[] filePath, byte[] fileName, byte[] segmentId)
+      throws MemoryException {
+    List<CarbonRowSchema> taskMinMaxSchemas = new ArrayList<>();
+    getMinMaxSchema(segmentProperties, taskMinMaxSchemas);
+    // for storing column schema
+    taskMinMaxSchemas.add(
+        new CarbonRowSchema.FixedCarbonRowSchema(DataTypes.BYTE_ARRAY, schemaBinary.length));
+    // for storing file path
+    taskMinMaxSchemas.add(
+        new CarbonRowSchema.FixedCarbonRowSchema(DataTypes.BYTE_ARRAY, filePath.length));
+    // for storing file name
+    taskMinMaxSchemas.add(
+        new CarbonRowSchema.FixedCarbonRowSchema(DataTypes.BYTE_ARRAY, fileName.length));
+    // for storing segmentid
+    taskMinMaxSchemas.add(
+        new CarbonRowSchema.FixedCarbonRowSchema(DataTypes.BYTE_ARRAY, segmentId.length));
+    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<>();
+    }
+    List<Blocklet> blocklets = new ArrayList<>();
+    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 {
+      // Remove B-tree jump logic as start and end key prepared is not
+      // correct for old store scenarios
+      int startIndex = 0;
+      int endIndex = unsafeMemoryDMStore.getRowCount();
+      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<PartitionSpec> partitions) {
+    if (unsafeMemoryDMStore.getRowCount() == 0) {
+      return new ArrayList<>();
+    }
+    // if it has partitioned datamap but there is no partitioned information stored, it means
+    // partitions are dropped so return empty list.
+    if (partitions != null) {
+      // First get the partitions which are stored inside datamap.
+      String[] fileDetails = getFileDetails();
+      // Check the exact match of partition information inside the stored partitions.
+      boolean found = false;
+      Path folderPath = new Path(fileDetails[0]);
+      for (PartitionSpec spec : partitions) {
+        if (folderPath.equals(spec.getLocation()) && isCorrectUUID(fileDetails, spec)) {
+          found = true;
+          break;
+        }
+      }
+      if (!found) {
+        return new ArrayList<>();
+      }
+    }
+    // Prune with filters if the partitions are existed in this datamap
+    return prune(filterExp, segmentProperties);
+  }
+
+  private boolean isCorrectUUID(String[] fileDetails, PartitionSpec spec) {
+    boolean needToScan = false;
+    if (spec.getUuid() != null) {
+      String[] split = spec.getUuid().split("_");
+      if (split[0].equals(fileDetails[2]) && CarbonTablePath.DataFileUtil
+          .getTimeStampFromFileName(fileDetails[1]).equals(split[1])) {
+        needToScan = true;
+      }
+    } else {
+      needToScan = true;
+    }
+    return needToScan;
+  }
+
+  /**
+   * 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;
+  }
+
+  private String[] getFileDetails() {
+    try {
+      String[] fileDetails = new String[3];
+      DataMapRow unsafeRow = unsafeMemorySummaryDMStore.getUnsafeRow(0);
+      fileDetails[0] =
+          new String(unsafeRow.getByteArray(INDEX_PATH), CarbonCommonConstants.DEFAULT_CHARSET);
+      fileDetails[1] = new String(unsafeRow.getByteArray(INDEX_FILE_NAME),
+          CarbonCommonConstants.DEFAULT_CHARSET);
+      fileDetails[2] = new String(unsafeRow.getByteArray(SEGMENTID),
+          CarbonCommonConstants.DEFAULT_CHARSET);
+      return fileDetails;
+    } catch (Exception e) {
+      throw new RuntimeException(e);
+    }
+  }
+
+
+  /**
+   * 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 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/f8ded96e/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..f1b97a4
--- /dev/null
+++ b/core/src/main/java/org/apache/carbondata/core/indexstore/blockletindex/BlockletIndexDataMapFactory.java
@@ -0,0 +1,285 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.carbondata.core.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.Segment;
+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.PartitionSpec;
+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.SegmentFileStore;
+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(Segment segment, String writeDirectoryPath) {
+    throw new UnsupportedOperationException("not implemented");
+  }
+
+  @Override
+  public List<AbstractCoarseGrainIndexDataMap> getDataMaps(Segment segment) throws IOException {
+    List<TableBlockIndexUniqueIdentifier> tableBlockIndexUniqueIdentifiers =
+        getTableBlockIndexUniqueIdentifiers(segment);
+    return cache.getAll(tableBlockIndexUniqueIdentifiers);
+  }
+
+  private List<TableBlockIndexUniqueIdentifier> getTableBlockIndexUniqueIdentifiers(
+      Segment segment) throws IOException {
+    List<TableBlockIndexUniqueIdentifier> tableBlockIndexUniqueIdentifiers =
+        segmentMap.get(segment.getSegmentNo());
+    if (tableBlockIndexUniqueIdentifiers == null) {
+      tableBlockIndexUniqueIdentifiers = new ArrayList<>();
+      Map<String, String> indexFiles;
+      if (segment.getSegmentFileName() == null) {
+        String path =
+            CarbonTablePath.getSegmentPath(identifier.getTablePath(), segment.getSegmentNo());
+        indexFiles = new SegmentIndexFileStore().getIndexFilesFromSegment(path);
+      } else {
+        SegmentFileStore fileStore =
+            new SegmentFileStore(identifier.getTablePath(), segment.getSegmentFileName());
+        indexFiles = fileStore.getIndexFiles();
+      }
+      for (Map.Entry<String, String> indexFileEntry: indexFiles.entrySet()) {
+        Path indexFile = new Path(indexFileEntry.getKey());
+        tableBlockIndexUniqueIdentifiers.add(
+            new TableBlockIndexUniqueIdentifier(indexFile.getParent().toString(),
+                indexFile.getName(), indexFileEntry.getValue(), segment.getSegmentNo()));
+      }
+      segmentMap.put(segment.getSegmentNo(), 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, Segment segment)
+      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(segment);
+    // 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, Segment segment)
+      throws IOException {
+    if (blocklet instanceof ExtendedBlocklet) {
+      return (ExtendedBlocklet) blocklet;
+    }
+    List<TableBlockIndexUniqueIdentifier> identifiers =
+        getTableBlockIndexUniqueIdentifiers(segment);
+    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.getIndexFilePath().equals(carbonIndexFileName)) {
+        IndexDataMap dataMap = cache.get(identifier);
+        return ((BlockletIndexDataMap) dataMap).getDetailedBlocklet(blocklet.getBlockletId());
+      }
+    }
+    throw new IOException("Blocklet with blockid " + blocklet.getBlockletId() + " not found ");
+  }
+
+
+  @Override
+  public List<DataMapDistributable> toDistributable(Segment segment) {
+    List<DataMapDistributable> distributables = new ArrayList<>();
+    try {
+      CarbonFile[] carbonIndexFiles;
+      if (segment.getSegmentFileName() == null) {
+        carbonIndexFiles = SegmentIndexFileStore.getCarbonIndexFiles(
+            CarbonTablePath.getSegmentPath(identifier.getTablePath(), segment.getSegmentNo()));
+      } else {
+        SegmentFileStore fileStore =
+            new SegmentFileStore(identifier.getTablePath(), segment.getSegmentFileName());
+        Map<String, String> indexFiles = fileStore.getIndexFiles();
+        carbonIndexFiles = new CarbonFile[indexFiles.size()];
+        int i = 0;
+        for (String indexFile : indexFiles.keySet()) {
+          carbonIndexFiles[i++] = FileFactory.getCarbonFile(indexFile);
+        }
+      }
+      for (int i = 0; i < carbonIndexFiles.length; i++) {
+        Path path = new Path(carbonIndexFiles[i].getPath());
+
+        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.toString());
+        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(Segment segment) {
+    List<TableBlockIndexUniqueIdentifier> blockIndexes = segmentMap.remove(segment.getSegmentNo());
+    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(new Segment(segmentId, null));
+    }
+  }
+
+  @Override
+  public List<AbstractCoarseGrainIndexDataMap> getDataMaps(DataMapDistributable distributable)
+      throws IOException {
+    BlockletDataMapDistributable mapDistributable = (BlockletDataMapDistributable) distributable;
+    List<TableBlockIndexUniqueIdentifier> identifiers = new ArrayList<>();
+    Path indexPath = new Path(mapDistributable.getFilePath());
+    String segmentNo = mapDistributable.getSegment().getSegmentNo();
+    if (indexPath.getName().endsWith(CarbonTablePath.INDEX_FILE_EXT)) {
+      String parent = indexPath.getParent().toString();
+      identifiers
+          .add(new TableBlockIndexUniqueIdentifier(parent, indexPath.getName(), null, segmentNo));
+    } else if (indexPath.getName().endsWith(CarbonTablePath.MERGE_INDEX_FILE_EXT)) {
+      SegmentIndexFileStore fileStore = new SegmentIndexFileStore();
+      CarbonFile carbonFile = FileFactory.getCarbonFile(indexPath.toString());
+      String parentPath = carbonFile.getParentFile().getAbsolutePath();
+      List<String> indexFiles = fileStore.getIndexFilesFromMergeFile(carbonFile.getAbsolutePath());
+      for (String indexFile : indexFiles) {
+        identifiers.add(
+            new TableBlockIndexUniqueIdentifier(parentPath, indexFile, carbonFile.getName(),
+                segmentNo));
+      }
+    }
+    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(Segment segment) throws IOException {
+    List<AbstractCoarseGrainIndexDataMap> dataMaps = getDataMaps(segment);
+    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(Segment segment, List<PartitionSpec> partitions)
+      throws IOException {
+    List<Blocklet> blocklets = new ArrayList<>();
+    List<AbstractCoarseGrainIndexDataMap> dataMaps = getDataMaps(segment);
+    for (AbstractCoarseGrainIndexDataMap dataMap : dataMaps) {
+      blocklets.addAll(dataMap.prune(null, getSegmentProperties(segment), partitions));
+    }
+    return blocklets;
+  }
+}

http://git-wip-us.apache.org/repos/asf/carbondata/blob/f8ded96e/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 4d54ad5..721845d 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/f8ded96e/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/f8ded96e/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 d0c7386..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
@@ -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) {
-    if (DataMapProvider.PREAGGREGATE.getClassName().equals(className) ||
-        DataMapProvider.TIMESERIES.getClassName().equals(className)) {
-      return new AggregationDataMapSchema(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, className);
+      return new DataMapSchema(dataMapName, providerName);
     }
   }
 }

http://git-wip-us.apache.org/repos/asf/carbondata/blob/f8ded96e/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/f8ded96e/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/f8ded96e/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/f8ded96e/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 75dac9e..0000000
--- a/datamap/examples/src/minmaxdatamap/main/java/org/apache/carbondata/datamap/examples/MinMaxDataMap.java
+++ /dev/null
@@ -1,152 +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.PartitionSpec;
-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<PartitionSpec> 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/f8ded96e/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 9a67644..0000000
--- a/datamap/examples/src/minmaxdatamap/main/java/org/apache/carbondata/datamap/examples/MinMaxDataMapFactory.java
+++ /dev/null
@@ -1,117 +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.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.Segment;
-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.core.util.path.CarbonTablePath;
-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 segment
-   * @return
-   */
-  @Override public AbstractDataMapWriter createWriter(Segment segment, String writeDirectoryPath) {
-    return new MinMaxDataWriter(identifier, segment, writeDirectoryPath);
-  }
-
-  /**
-   * getDataMaps Factory method Initializes the Min Max Data Map and returns.
-   *
-   * @param segment
-   * @return
-   * @throws IOException
-   */
-  @Override public List<AbstractCoarseGrainDataMap> getDataMaps(Segment segment)
-      throws IOException {
-    List<AbstractCoarseGrainDataMap> dataMapList = new ArrayList<>();
-    // Form a dataMap of Type MinMaxDataMap.
-    MinMaxDataMap dataMap = new MinMaxDataMap();
-    try {
-      dataMap.init(new DataMapModel(
-          CarbonTablePath.getSegmentPath(identifier.getTablePath(), segment.getSegmentNo())));
-    } catch (MemoryException ex) {
-      throw new IOException(ex);
-    }
-    dataMapList.add(dataMap);
-    return dataMapList;
-  }
-
-  /**
-   * @param segment
-   * @return
-   */
-  @Override public List<DataMapDistributable> toDistributable(Segment segment) {
-    return null;
-  }
-
-  /**
-   * Clear the DataMap.
-   *
-   * @param segment
-   */
-  @Override public void clear(Segment segment) {
-  }
-
-  /**
-   * 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/f8ded96e/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..8569081
--- /dev/null
+++ b/datamap/examples/src/minmaxdatamap/main/java/org/apache/carbondata/datamap/examples/MinMaxIndexDataMap.java
@@ -0,0 +1,150 @@
+/*
+ * 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.indexstore.PartitionSpec;
+import org.apache.carbondata.core.indexstore.row.DataMapRow;
+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<PartitionSpec> 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/f8ded96e/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..9624766
--- /dev/null
+++ b/datamap/examples/src/minmaxdatamap/main/java/org/apache/carbondata/datamap/examples/MinMaxIndexDataMapFactory.java
@@ -0,0 +1,117 @@
+/*
+ * 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.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.Segment;
+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.core.util.path.CarbonTablePath;
+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 segment
+   * @return
+   */
+  @Override public AbstractDataMapWriter createWriter(Segment segment, String writeDirectoryPath) {
+    return new MinMaxDataWriter(identifier, segment, writeDirectoryPath);
+  }
+
+  /**
+   * getDataMaps Factory method Initializes the Min Max Data Map and returns.
+   *
+   * @param segment
+   * @return
+   * @throws IOException
+   */
+  @Override public List<AbstractCoarseGrainIndexDataMap> getDataMaps(Segment segment)
+      throws IOException {
+    List<AbstractCoarseGrainIndexDataMap> dataMapList = new ArrayList<>();
+    // Form a dataMap of Type MinMaxIndexDataMap.
+    MinMaxIndexDataMap dataMap = new MinMaxIndexDataMap();
+    try {
+      dataMap.init(new DataMapModel(
+          CarbonTablePath.getSegmentPath(identifier.getTablePath(), segment.getSegmentNo())));
+    } catch (MemoryException ex) {
+      throw new IOException(ex);
+    }
+    dataMapList.add(dataMap);
+    return dataMapList;
+  }
+
+  /**
+   * @param segment
+   * @return
+   */
+  @Override public List<DataMapDistributable> toDistributable(Segment segment) {
+    return null;
+  }
+
+  /**
+   * Clear the DataMap.
+   *
+   * @param segment
+   */
+  @Override public void clear(Segment segment) {
+  }
+
+  /**
+   * 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/f8ded96e/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")
 


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

Posted by ja...@apache.org.
http://git-wip-us.apache.org/repos/asf/carbondata/blob/f8ded96e/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 90178b1..0000000
--- a/core/src/main/java/org/apache/carbondata/core/indexstore/blockletindex/BlockletDataMap.java
+++ /dev/null
@@ -1,971 +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.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.PartitionSpec;
-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.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.CarbonUtil;
-import org.apache.carbondata.core.util.DataFileFooterConverter;
-import org.apache.carbondata.core.util.DataTypeUtil;
-import org.apache.carbondata.core.util.path.CarbonTablePath;
-
-import org.apache.commons.lang3.StringUtils;
-import org.apache.hadoop.fs.Path;
-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 INDEX_PATH = 3;
-
-  private static int INDEX_FILE_NAME = 4;
-
-  private static int SEGMENTID = 5;
-
-  private UnsafeMemoryDMStore unsafeMemoryDMStore;
-
-  private UnsafeMemoryDMStore unsafeMemorySummaryDMStore;
-
-  private SegmentProperties segmentProperties;
-
-  private int[] columnCardinality;
-
-  @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());
-    Path path = new Path(blockletDataMapInfo.getFilePath());
-    byte[] filePath = path.getParent().toString().getBytes(CarbonCommonConstants.DEFAULT_CHARSET);
-    byte[] fileName = path.getName().toString().getBytes(CarbonCommonConstants.DEFAULT_CHARSET);
-    byte[] segmentId =
-        blockletDataMapInfo.getSegmentId().getBytes(CarbonCommonConstants.DEFAULT_CHARSET);
-    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, schemaBinary, filePath, fileName,
-            segmentId);
-      }
-      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,
-          schemaBinary,
-          filePath,
-          fileName,
-          segmentId);
-      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);
-    byte[][] maxValues = updateMaxValues(minMaxIndex.getMaxValues(), minMaxLen);
-    // update min max values in case of old store
-    byte[][] updatedMinValues =
-        CarbonUtil.updateMinMaxValues(fileFooter, maxValues, minValues, true);
-    byte[][] updatedMaxValues =
-        CarbonUtil.updateMinMaxValues(fileFooter, maxValues, minValues, false);
-    row.setRow(addMinMax(minMaxLen, schema[ordinal], updatedMinValues), ordinal);
-    // compute and set task level min values
-    addTaskMinMaxValues(summaryRow, minMaxLen,
-        unsafeMemorySummaryDMStore.getSchema()[taskMinMaxOrdinal], updatedMinValues,
-        TASK_MIN_VALUES_INDEX, true);
-    ordinal++;
-    taskMinMaxOrdinal++;
-    row.setRow(addMinMax(minMaxLen, schema[ordinal], updatedMaxValues), ordinal);
-    // compute and set task level max values
-    addTaskMinMaxValues(summaryRow, minMaxLen,
-        unsafeMemorySummaryDMStore.getSchema()[taskMinMaxOrdinal], updatedMaxValues,
-        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, byte[] schemaBinary,
-      byte[] filePath, byte[] fileName, byte[] segmentId) {
-    // 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);
-      }
-      summaryRow.setByteArray(filePath, INDEX_PATH);
-      summaryRow.setByteArray(fileName, INDEX_FILE_NAME);
-      summaryRow.setByteArray(segmentId, SEGMENTID);
-      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
-   * @throws MemoryException
-   */
-  private void createSummarySchema(SegmentProperties segmentProperties, byte[] schemaBinary,
-      byte[] filePath, byte[] fileName, byte[] segmentId)
-      throws MemoryException {
-    List<CarbonRowSchema> taskMinMaxSchemas = new ArrayList<>();
-    getMinMaxSchema(segmentProperties, taskMinMaxSchemas);
-    // for storing column schema
-    taskMinMaxSchemas.add(
-        new CarbonRowSchema.FixedCarbonRowSchema(DataTypes.BYTE_ARRAY, schemaBinary.length));
-    // for storing file path
-    taskMinMaxSchemas.add(
-        new CarbonRowSchema.FixedCarbonRowSchema(DataTypes.BYTE_ARRAY, filePath.length));
-    // for storing file name
-    taskMinMaxSchemas.add(
-        new CarbonRowSchema.FixedCarbonRowSchema(DataTypes.BYTE_ARRAY, fileName.length));
-    // for storing segmentid
-    taskMinMaxSchemas.add(
-        new CarbonRowSchema.FixedCarbonRowSchema(DataTypes.BYTE_ARRAY, segmentId.length));
-    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<>();
-    }
-    List<Blocklet> blocklets = new ArrayList<>();
-    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 {
-      // Remove B-tree jump logic as start and end key prepared is not
-      // correct for old store scenarios
-      int startIndex = 0;
-      int endIndex = unsafeMemoryDMStore.getRowCount();
-      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<PartitionSpec> partitions) {
-    if (unsafeMemoryDMStore.getRowCount() == 0) {
-      return new ArrayList<>();
-    }
-    // if it has partitioned datamap but there is no partitioned information stored, it means
-    // partitions are dropped so return empty list.
-    if (partitions != null) {
-      // First get the partitions which are stored inside datamap.
-      String[] fileDetails = getFileDetails();
-      // Check the exact match of partition information inside the stored partitions.
-      boolean found = false;
-      Path folderPath = new Path(fileDetails[0]);
-      for (PartitionSpec spec : partitions) {
-        if (folderPath.equals(spec.getLocation()) && isCorrectUUID(fileDetails, spec)) {
-          found = true;
-          break;
-        }
-      }
-      if (!found) {
-        return new ArrayList<>();
-      }
-    }
-    // Prune with filters if the partitions are existed in this datamap
-    return prune(filterExp, segmentProperties);
-  }
-
-  private boolean isCorrectUUID(String[] fileDetails, PartitionSpec spec) {
-    boolean needToScan = false;
-    if (spec.getUuid() != null) {
-      String[] split = spec.getUuid().split("_");
-      if (split[0].equals(fileDetails[2]) && CarbonTablePath.DataFileUtil
-          .getTimeStampFromFileName(fileDetails[1]).equals(split[1])) {
-        needToScan = true;
-      }
-    } else {
-      needToScan = true;
-    }
-    return needToScan;
-  }
-
-  /**
-   * 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;
-  }
-
-  private String[] getFileDetails() {
-    try {
-      String[] fileDetails = new String[3];
-      DataMapRow unsafeRow = unsafeMemorySummaryDMStore.getUnsafeRow(0);
-      fileDetails[0] =
-          new String(unsafeRow.getByteArray(INDEX_PATH), CarbonCommonConstants.DEFAULT_CHARSET);
-      fileDetails[1] = new String(unsafeRow.getByteArray(INDEX_FILE_NAME),
-          CarbonCommonConstants.DEFAULT_CHARSET);
-      fileDetails[2] = new String(unsafeRow.getByteArray(SEGMENTID),
-          CarbonCommonConstants.DEFAULT_CHARSET);
-      return fileDetails;
-    } catch (Exception e) {
-      throw new RuntimeException(e);
-    }
-  }
-
-
-  /**
-   * 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 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/f8ded96e/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 43df813..0000000
--- a/core/src/main/java/org/apache/carbondata/core/indexstore/blockletindex/BlockletDataMapFactory.java
+++ /dev/null
@@ -1,285 +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.Segment;
-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.PartitionSpec;
-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.SegmentFileStore;
-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(Segment segment, String writeDirectoryPath) {
-    throw new UnsupportedOperationException("not implemented");
-  }
-
-  @Override
-  public List<AbstractCoarseGrainDataMap> getDataMaps(Segment segment) throws IOException {
-    List<TableBlockIndexUniqueIdentifier> tableBlockIndexUniqueIdentifiers =
-        getTableBlockIndexUniqueIdentifiers(segment);
-    return cache.getAll(tableBlockIndexUniqueIdentifiers);
-  }
-
-  private List<TableBlockIndexUniqueIdentifier> getTableBlockIndexUniqueIdentifiers(
-      Segment segment) throws IOException {
-    List<TableBlockIndexUniqueIdentifier> tableBlockIndexUniqueIdentifiers =
-        segmentMap.get(segment.getSegmentNo());
-    if (tableBlockIndexUniqueIdentifiers == null) {
-      tableBlockIndexUniqueIdentifiers = new ArrayList<>();
-      Map<String, String> indexFiles;
-      if (segment.getSegmentFileName() == null) {
-        String path =
-            CarbonTablePath.getSegmentPath(identifier.getTablePath(), segment.getSegmentNo());
-        indexFiles = new SegmentIndexFileStore().getIndexFilesFromSegment(path);
-      } else {
-        SegmentFileStore fileStore =
-            new SegmentFileStore(identifier.getTablePath(), segment.getSegmentFileName());
-        indexFiles = fileStore.getIndexFiles();
-      }
-      for (Map.Entry<String, String> indexFileEntry: indexFiles.entrySet()) {
-        Path indexFile = new Path(indexFileEntry.getKey());
-        tableBlockIndexUniqueIdentifiers.add(
-            new TableBlockIndexUniqueIdentifier(indexFile.getParent().toString(),
-                indexFile.getName(), indexFileEntry.getValue(), segment.getSegmentNo()));
-      }
-      segmentMap.put(segment.getSegmentNo(), 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, Segment segment)
-      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(segment);
-    // 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, Segment segment)
-      throws IOException {
-    if (blocklet instanceof ExtendedBlocklet) {
-      return (ExtendedBlocklet) blocklet;
-    }
-    List<TableBlockIndexUniqueIdentifier> identifiers =
-        getTableBlockIndexUniqueIdentifiers(segment);
-    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.getIndexFilePath().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(Segment segment) {
-    List<DataMapDistributable> distributables = new ArrayList<>();
-    try {
-      CarbonFile[] carbonIndexFiles;
-      if (segment.getSegmentFileName() == null) {
-        carbonIndexFiles = SegmentIndexFileStore.getCarbonIndexFiles(
-            CarbonTablePath.getSegmentPath(identifier.getTablePath(), segment.getSegmentNo()));
-      } else {
-        SegmentFileStore fileStore =
-            new SegmentFileStore(identifier.getTablePath(), segment.getSegmentFileName());
-        Map<String, String> indexFiles = fileStore.getIndexFiles();
-        carbonIndexFiles = new CarbonFile[indexFiles.size()];
-        int i = 0;
-        for (String indexFile : indexFiles.keySet()) {
-          carbonIndexFiles[i++] = FileFactory.getCarbonFile(indexFile);
-        }
-      }
-      for (int i = 0; i < carbonIndexFiles.length; i++) {
-        Path path = new Path(carbonIndexFiles[i].getPath());
-
-        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.toString());
-        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(Segment segment) {
-    List<TableBlockIndexUniqueIdentifier> blockIndexes = segmentMap.remove(segment.getSegmentNo());
-    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(new Segment(segmentId, null));
-    }
-  }
-
-  @Override
-  public List<AbstractCoarseGrainDataMap> getDataMaps(DataMapDistributable distributable)
-      throws IOException {
-    BlockletDataMapDistributable mapDistributable = (BlockletDataMapDistributable) distributable;
-    List<TableBlockIndexUniqueIdentifier> identifiers = new ArrayList<>();
-    Path indexPath = new Path(mapDistributable.getFilePath());
-    String segmentNo = mapDistributable.getSegment().getSegmentNo();
-    if (indexPath.getName().endsWith(CarbonTablePath.INDEX_FILE_EXT)) {
-      String parent = indexPath.getParent().toString();
-      identifiers
-          .add(new TableBlockIndexUniqueIdentifier(parent, indexPath.getName(), null, segmentNo));
-    } else if (indexPath.getName().endsWith(CarbonTablePath.MERGE_INDEX_FILE_EXT)) {
-      SegmentIndexFileStore fileStore = new SegmentIndexFileStore();
-      CarbonFile carbonFile = FileFactory.getCarbonFile(indexPath.toString());
-      String parentPath = carbonFile.getParentFile().getAbsolutePath();
-      List<String> indexFiles = fileStore.getIndexFilesFromMergeFile(carbonFile.getAbsolutePath());
-      for (String indexFile : indexFiles) {
-        identifiers.add(
-            new TableBlockIndexUniqueIdentifier(parentPath, indexFile, carbonFile.getName(),
-                segmentNo));
-      }
-    }
-    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(Segment segment) throws IOException {
-    List<AbstractCoarseGrainDataMap> dataMaps = getDataMaps(segment);
-    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(Segment segment, List<PartitionSpec> partitions)
-      throws IOException {
-    List<Blocklet> blocklets = new ArrayList<>();
-    List<AbstractCoarseGrainDataMap> dataMaps = getDataMaps(segment);
-    for (AbstractCoarseGrainDataMap dataMap : dataMaps) {
-      blocklets.addAll(dataMap.prune(null, getSegmentProperties(segment), partitions));
-    }
-    return blocklets;
-  }
-}

http://git-wip-us.apache.org/repos/asf/carbondata/blob/f8ded96e/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 ebeb278..018c3f7 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
@@ -22,7 +22,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 {
 


[08/20] 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/f8ded96e
Tree: http://git-wip-us.apache.org/repos/asf/carbondata/tree/f8ded96e
Diff: http://git-wip-us.apache.org/repos/asf/carbondata/diff/f8ded96e

Branch: refs/heads/datamap-rebase1
Commit: f8ded96e659cb1e99cc6ac511d1db7cbc25dddb7
Parents: c7a9f15
Author: Jacky Li <ja...@qq.com>
Authored: Thu Feb 22 20:59:59 2018 +0800
Committer: Jacky Li <ja...@qq.com>
Committed: Sun Mar 4 22:31:26 2018 +0800

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


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

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

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

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

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

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

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

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

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


[10/20] carbondata git commit: [HOTFIX] Add dava doc for datamap interface

Posted by ja...@apache.org.
http://git-wip-us.apache.org/repos/asf/carbondata/blob/1134431d/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
deleted file mode 100644
index 72a88ea..0000000
--- a/integration/spark-common-test/src/test/scala/org/apache/carbondata/spark/testsuite/datamap/CGIndexDataMapTestCase.scala
+++ /dev/null
@@ -1,383 +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.{AbstractDataMapWriter, DataMapModel}
-import org.apache.carbondata.core.datamap.{DataMapDistributable, DataMapMeta, DataMapStoreManager, Segment}
-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, PartitionSpec}
-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(segment: Segment, dataWritePath: String): AbstractDataMapWriter = {
-    new CGDataMapWriter(identifier, segment, dataWritePath, dataMapSchema)
-  }
-
-  /**
-   * Get the datamap for segmentid
-   */
-  override def getDataMaps(segment: Segment): java.util.List[AbstractCoarseGrainDataMap] = {
-    val file = FileFactory.getCarbonFile(
-      CarbonTablePath.getSegmentPath(identifier.getTablePath, segment.getSegmentNo))
-
-    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(segment: Segment): java.util.List[DataMapDistributable] = {
-    val file = FileFactory.getCarbonFile(
-      CarbonTablePath.getSegmentPath(identifier.getTablePath, segment.getSegmentNo))
-
-    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(segment: Segment): 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[PartitionSpec]): 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,
-    segment: Segment,
-    dataWritePath: String,
-    dataMapSchema: DataMapSchema)
-  extends AbstractDataMapWriter(identifier, segment, 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/1134431d/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
new file mode 100644
index 0000000..d5cae15
--- /dev/null
+++ b/integration/spark-common-test/src/test/scala/org/apache/carbondata/spark/testsuite/datamap/DataMapWriterSuite.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.{DataMapDistributable, DataMapMeta, DataMapStoreManager, Segment}
+import org.apache.carbondata.core.datamap.dev.DataMapWriter
+import org.apache.carbondata.core.datamap.dev.cgdatamap.{CoarseGrainDataMap, CoarseGrainDataMapFactory}
+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.util.CarbonProperties
+import org.apache.carbondata.events.Event
+
+class C2DataMapFactory() extends CoarseGrainDataMapFactory {
+
+  var identifier: AbsoluteTableIdentifier = _
+
+  override def init(identifier: AbsoluteTableIdentifier,
+      dataMapSchema: DataMapSchema): Unit = {
+    this.identifier = identifier
+  }
+
+  override def fireEvent(event: Event): Unit = ???
+
+  override def clear(segment: Segment): Unit = {}
+
+  override def clear(): Unit = {}
+
+  override def getDataMaps(distributable: DataMapDistributable): util.List[CoarseGrainDataMap] = ???
+
+  override def getDataMaps(segment: Segment): util.List[CoarseGrainDataMap] = ???
+
+  override def createWriter(segment: Segment, dataWritePath: String): DataMapWriter =
+    DataMapWriterSuite.dataMapWriterC2Mock(identifier, segment, 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: Segment): 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, segment: Segment,
+      dataWritePath: String) =
+    new DataMapWriter(identifier, segment, 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/1134431d/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
new file mode 100644
index 0000000..977b31d
--- /dev/null
+++ b/integration/spark-common-test/src/test/scala/org/apache/carbondata/spark/testsuite/datamap/FGDataMapTestCase.scala
@@ -0,0 +1,473 @@
+/*
+ * 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.{DataMapModel}
+import org.apache.carbondata.core.datamap.{DataMapDistributable, DataMapMeta, DataMapStoreManager, Segment}
+import org.apache.carbondata.core.datamap.dev.fgdatamap.{FineGrainBlocklet, FineGrainDataMap, FineGrainDataMapFactory}
+import org.apache.carbondata.core.datamap.dev.{DataMapModel, DataMapWriter}
+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.{Blocklet, PartitionSpec}
+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 FineGrainDataMapFactory {
+  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(segment: Segment, dataWritePath: String): DataMapWriter = {
+    new FGDataMapWriter(identifier, segment, dataWritePath, dataMapSchema)
+  }
+
+  /**
+   * Get the datamap for segmentid
+   */
+  override def getDataMaps(segment: Segment): java.util.List[FineGrainDataMap] = {
+    val file = FileFactory
+      .getCarbonFile(CarbonTablePath.getSegmentPath(identifier.getTablePath, segment.getSegmentNo))
+
+    val files = file.listFiles(new CarbonFileFilter {
+      override def accept(file: CarbonFile): Boolean = file.getName.endsWith(".datamap")
+    })
+    files.map { f =>
+      val dataMap: FineGrainDataMap = new FGDataMap()
+      dataMap.init(new DataMapModel(f.getCanonicalPath))
+      dataMap
+    }.toList.asJava
+  }
+
+  /**
+   * Get datamap for distributable object.
+   */
+  override def getDataMaps(
+      distributable: DataMapDistributable): java.util.List[FineGrainDataMap]= {
+    val mapDistributable = distributable.asInstanceOf[BlockletDataMapDistributable]
+    val dataMap: FineGrainDataMap = new FGDataMap()
+    dataMap.init(new DataMapModel(mapDistributable.getFilePath))
+    Seq(dataMap).asJava
+  }
+
+  /**
+   * Get all distributable objects of a segmentid
+   *
+   * @return
+   */
+  override def toDistributable(segment: Segment): java.util.List[DataMapDistributable] = {
+    val file = FileFactory.getCarbonFile(
+      CarbonTablePath.getSegmentPath(identifier.getTablePath, segment.getSegmentNo))
+
+    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(segment: Segment): 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 FineGrainDataMap {
+
+  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[PartitionSpec]): 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 =>
+      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[Blocklet] = {
+    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
+      }
+      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,
+    segment: Segment, dataWriterPath: String, dataMapSchema: DataMapSchema)
+  extends DataMapWriter(identifier, segment, 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/1134431d/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
deleted file mode 100644
index d92efe7..0000000
--- a/integration/spark-common-test/src/test/scala/org/apache/carbondata/spark/testsuite/datamap/FGIndexDataMapTestCase.scala
+++ /dev/null
@@ -1,472 +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.{AbstractFineGrainIndexDataMap, AbstractFineGrainIndexDataMapFactory}
-import org.apache.carbondata.core.datamap.dev.{AbstractDataMapWriter, DataMapModel}
-import org.apache.carbondata.core.datamap.{DataMapDistributable, DataMapMeta, DataMapStoreManager, Segment}
-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.{Blocklet, FineGrainBlocklet, PartitionSpec}
-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(segment: Segment, dataWritePath: String): AbstractDataMapWriter = {
-    new FGDataMapWriter(identifier, segment, dataWritePath, dataMapSchema)
-  }
-
-  /**
-   * Get the datamap for segmentid
-   */
-  override def getDataMaps(segment: Segment): java.util.List[AbstractFineGrainIndexDataMap] = {
-    val file = FileFactory
-      .getCarbonFile(CarbonTablePath.getSegmentPath(identifier.getTablePath, segment.getSegmentNo))
-
-    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(segment: Segment): java.util.List[DataMapDistributable] = {
-    val file = FileFactory.getCarbonFile(
-      CarbonTablePath.getSegmentPath(identifier.getTablePath, segment.getSegmentNo))
-
-    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(segment: Segment): 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[PartitionSpec]): 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 =>
-      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[Blocklet] = {
-    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
-      }
-      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,
-    segment: Segment, dataWriterPath: String, dataMapSchema: DataMapSchema)
-  extends AbstractDataMapWriter(identifier, segment, 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/1134431d/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
deleted file mode 100644
index 795ef6a..0000000
--- a/integration/spark-common-test/src/test/scala/org/apache/carbondata/spark/testsuite/datamap/IndexDataMapWriterSuite.scala
+++ /dev/null
@@ -1,217 +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.{DataMapDistributable, DataMapMeta, DataMapStoreManager, Segment}
-import org.apache.carbondata.core.datamap.dev.{AbstractDataMapWriter}
-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.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(segment: Segment): Unit = {}
-
-  override def clear(): Unit = {}
-
-  override def getDataMaps(distributable: DataMapDistributable): util.List[AbstractCoarseGrainIndexDataMap] = ???
-
-  override def getDataMaps(segment: Segment): util.List[AbstractCoarseGrainIndexDataMap] = ???
-
-  override def createWriter(segment: Segment, dataWritePath: String): AbstractDataMapWriter =
-    IndexDataMapWriterSuite.dataMapWriterC2Mock(identifier, segment, 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: Segment): 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, segment: Segment,
-      dataWritePath: String) =
-    new AbstractDataMapWriter(identifier, segment, 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/1134431d/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
new file mode 100644
index 0000000..eaa2ae7
--- /dev/null
+++ b/integration/spark-common-test/src/test/scala/org/apache/carbondata/spark/testsuite/datamap/TestDataMapCommand.scala
@@ -0,0 +1,285 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.carbondata.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' 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 datamap datamap_hiveMetaStoreTable_1 on table hiveMetaStoreTable_1")
+      checkExistence(sql("show datamap on table hiveMetaStoreTable_1"),
+        false,
+        "datamap_hiveMetaStoreTable_1")
+      assert(sql("show datamap on table hiveMetaStoreTable_1").collect().length == 0)
+      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")
+  }
+}


[12/20] carbondata git commit: [HOTFIX] Add dava doc for datamap interface

Posted by ja...@apache.org.
http://git-wip-us.apache.org/repos/asf/carbondata/blob/1134431d/core/src/main/java/org/apache/carbondata/core/indexstore/FineGrainBlocklet.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/carbondata/core/indexstore/FineGrainBlocklet.java b/core/src/main/java/org/apache/carbondata/core/indexstore/FineGrainBlocklet.java
deleted file mode 100644
index 229e5bf..0000000
--- a/core/src/main/java/org/apache/carbondata/core/indexstore/FineGrainBlocklet.java
+++ /dev/null
@@ -1,128 +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;
-
-import java.io.DataInput;
-import java.io.DataOutput;
-import java.io.IOException;
-import java.io.Serializable;
-import java.util.ArrayList;
-import java.util.BitSet;
-import java.util.List;
-
-import org.apache.carbondata.core.constants.CarbonV3DataFormatConstants;
-import org.apache.carbondata.core.metadata.schema.table.Writable;
-import org.apache.carbondata.core.util.BitSetGroup;
-
-/**
- * FineGrainBlocklet
- */
-public class FineGrainBlocklet extends Blocklet implements Serializable {
-
-  private List<Page> pages;
-
-  public FineGrainBlocklet(String blockId, String blockletId, List<Page> pages) {
-    super(blockId, blockletId);
-    this.pages = pages;
-  }
-
-  // For serialization purpose
-  public FineGrainBlocklet() {
-
-  }
-
-  public List<Page> getPages() {
-    return pages;
-  }
-
-  public static class Page implements Writable,Serializable {
-
-    private int pageId;
-
-    private int[] rowId;
-
-    public BitSet getBitSet() {
-      BitSet bitSet =
-          new BitSet(CarbonV3DataFormatConstants.NUMBER_OF_ROWS_PER_BLOCKLET_COLUMN_PAGE_DEFAULT);
-      for (int row : rowId) {
-        bitSet.set(row);
-      }
-      return bitSet;
-    }
-
-    @Override public void write(DataOutput out) throws IOException {
-      out.writeInt(pageId);
-      out.writeInt(rowId.length);
-      for (int i = 0; i < rowId.length; i++) {
-        out.writeInt(rowId[i]);
-      }
-    }
-
-    @Override public void readFields(DataInput in) throws IOException {
-      pageId = in.readInt();
-      int length = in.readInt();
-      rowId = new int[length];
-      for (int i = 0; i < length; i++) {
-        rowId[i] = in.readInt();
-      }
-    }
-
-    public void setPageId(int pageId) {
-      this.pageId = pageId;
-    }
-
-    public void setRowId(int[] rowId) {
-      this.rowId = rowId;
-    }
-  }
-
-  public BitSetGroup getBitSetGroup(int numberOfPages) {
-    BitSetGroup bitSetGroup = new BitSetGroup(numberOfPages);
-    for (int i = 0; i < pages.size(); i++) {
-      bitSetGroup.setBitSet(pages.get(i).getBitSet(), pages.get(i).pageId);
-    }
-    return bitSetGroup;
-  }
-
-  @Override public void write(DataOutput out) throws IOException {
-    super.write(out);
-    int size = pages.size();
-    out.writeInt(size);
-    for (Page page : pages) {
-      page.write(out);
-    }
-  }
-
-  @Override public void readFields(DataInput in) throws IOException {
-    super.readFields(in);
-    int size = in.readInt();
-    pages = new ArrayList<>(size);
-    for (int i = 0; i < size; i++) {
-      Page page = new Page();
-      page.readFields(in);
-      pages.add(page);
-    }
-  }
-
-  @Override public boolean equals(Object o) {
-    return super.equals(o);
-  }
-
-  @Override public int hashCode() {
-    return super.hashCode();
-  }
-}

http://git-wip-us.apache.org/repos/asf/carbondata/blob/1134431d/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
new file mode 100644
index 0000000..3ca9c5a
--- /dev/null
+++ b/core/src/main/java/org/apache/carbondata/core/indexstore/blockletindex/BlockletDataMap.java
@@ -0,0 +1,971 @@
+/*
+ * 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.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.CoarseGrainDataMap;
+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.PartitionSpec;
+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.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.CarbonUtil;
+import org.apache.carbondata.core.util.DataFileFooterConverter;
+import org.apache.carbondata.core.util.DataTypeUtil;
+import org.apache.carbondata.core.util.path.CarbonTablePath;
+
+import org.apache.commons.lang3.StringUtils;
+import org.apache.hadoop.fs.Path;
+import org.xerial.snappy.Snappy;
+
+/**
+ * Datamap implementation for blocklet.
+ */
+public class BlockletDataMap extends CoarseGrainDataMap 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 INDEX_PATH = 3;
+
+  private static int INDEX_FILE_NAME = 4;
+
+  private static int SEGMENTID = 5;
+
+  private UnsafeMemoryDMStore unsafeMemoryDMStore;
+
+  private UnsafeMemoryDMStore unsafeMemorySummaryDMStore;
+
+  private SegmentProperties segmentProperties;
+
+  private int[] columnCardinality;
+
+  @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());
+    Path path = new Path(blockletDataMapInfo.getFilePath());
+    byte[] filePath = path.getParent().toString().getBytes(CarbonCommonConstants.DEFAULT_CHARSET);
+    byte[] fileName = path.getName().toString().getBytes(CarbonCommonConstants.DEFAULT_CHARSET);
+    byte[] segmentId =
+        blockletDataMapInfo.getSegmentId().getBytes(CarbonCommonConstants.DEFAULT_CHARSET);
+    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, schemaBinary, filePath, fileName,
+            segmentId);
+      }
+      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,
+          schemaBinary,
+          filePath,
+          fileName,
+          segmentId);
+      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);
+    byte[][] maxValues = updateMaxValues(minMaxIndex.getMaxValues(), minMaxLen);
+    // update min max values in case of old store
+    byte[][] updatedMinValues =
+        CarbonUtil.updateMinMaxValues(fileFooter, maxValues, minValues, true);
+    byte[][] updatedMaxValues =
+        CarbonUtil.updateMinMaxValues(fileFooter, maxValues, minValues, false);
+    row.setRow(addMinMax(minMaxLen, schema[ordinal], updatedMinValues), ordinal);
+    // compute and set task level min values
+    addTaskMinMaxValues(summaryRow, minMaxLen,
+        unsafeMemorySummaryDMStore.getSchema()[taskMinMaxOrdinal], updatedMinValues,
+        TASK_MIN_VALUES_INDEX, true);
+    ordinal++;
+    taskMinMaxOrdinal++;
+    row.setRow(addMinMax(minMaxLen, schema[ordinal], updatedMaxValues), ordinal);
+    // compute and set task level max values
+    addTaskMinMaxValues(summaryRow, minMaxLen,
+        unsafeMemorySummaryDMStore.getSchema()[taskMinMaxOrdinal], updatedMaxValues,
+        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, byte[] schemaBinary,
+      byte[] filePath, byte[] fileName, byte[] segmentId) {
+    // 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);
+      }
+      summaryRow.setByteArray(filePath, INDEX_PATH);
+      summaryRow.setByteArray(fileName, INDEX_FILE_NAME);
+      summaryRow.setByteArray(segmentId, SEGMENTID);
+      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
+   * @throws MemoryException
+   */
+  private void createSummarySchema(SegmentProperties segmentProperties, byte[] schemaBinary,
+      byte[] filePath, byte[] fileName, byte[] segmentId)
+      throws MemoryException {
+    List<CarbonRowSchema> taskMinMaxSchemas = new ArrayList<>();
+    getMinMaxSchema(segmentProperties, taskMinMaxSchemas);
+    // for storing column schema
+    taskMinMaxSchemas.add(
+        new CarbonRowSchema.FixedCarbonRowSchema(DataTypes.BYTE_ARRAY, schemaBinary.length));
+    // for storing file path
+    taskMinMaxSchemas.add(
+        new CarbonRowSchema.FixedCarbonRowSchema(DataTypes.BYTE_ARRAY, filePath.length));
+    // for storing file name
+    taskMinMaxSchemas.add(
+        new CarbonRowSchema.FixedCarbonRowSchema(DataTypes.BYTE_ARRAY, fileName.length));
+    // for storing segmentid
+    taskMinMaxSchemas.add(
+        new CarbonRowSchema.FixedCarbonRowSchema(DataTypes.BYTE_ARRAY, segmentId.length));
+    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<>();
+    }
+    List<Blocklet> blocklets = new ArrayList<>();
+    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 {
+      // Remove B-tree jump logic as start and end key prepared is not
+      // correct for old store scenarios
+      int startIndex = 0;
+      int endIndex = unsafeMemoryDMStore.getRowCount();
+      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<PartitionSpec> partitions) {
+    if (unsafeMemoryDMStore.getRowCount() == 0) {
+      return new ArrayList<>();
+    }
+    // if it has partitioned datamap but there is no partitioned information stored, it means
+    // partitions are dropped so return empty list.
+    if (partitions != null) {
+      // First get the partitions which are stored inside datamap.
+      String[] fileDetails = getFileDetails();
+      // Check the exact match of partition information inside the stored partitions.
+      boolean found = false;
+      Path folderPath = new Path(fileDetails[0]);
+      for (PartitionSpec spec : partitions) {
+        if (folderPath.equals(spec.getLocation()) && isCorrectUUID(fileDetails, spec)) {
+          found = true;
+          break;
+        }
+      }
+      if (!found) {
+        return new ArrayList<>();
+      }
+    }
+    // Prune with filters if the partitions are existed in this datamap
+    return prune(filterExp, segmentProperties);
+  }
+
+  private boolean isCorrectUUID(String[] fileDetails, PartitionSpec spec) {
+    boolean needToScan = false;
+    if (spec.getUuid() != null) {
+      String[] split = spec.getUuid().split("_");
+      if (split[0].equals(fileDetails[2]) && CarbonTablePath.DataFileUtil
+          .getTimeStampFromFileName(fileDetails[1]).equals(split[1])) {
+        needToScan = true;
+      }
+    } else {
+      needToScan = true;
+    }
+    return needToScan;
+  }
+
+  /**
+   * 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;
+  }
+
+  private String[] getFileDetails() {
+    try {
+      String[] fileDetails = new String[3];
+      DataMapRow unsafeRow = unsafeMemorySummaryDMStore.getUnsafeRow(0);
+      fileDetails[0] =
+          new String(unsafeRow.getByteArray(INDEX_PATH), CarbonCommonConstants.DEFAULT_CHARSET);
+      fileDetails[1] = new String(unsafeRow.getByteArray(INDEX_FILE_NAME),
+          CarbonCommonConstants.DEFAULT_CHARSET);
+      fileDetails[2] = new String(unsafeRow.getByteArray(SEGMENTID),
+          CarbonCommonConstants.DEFAULT_CHARSET);
+      return fileDetails;
+    } catch (Exception e) {
+      throw new RuntimeException(e);
+    }
+  }
+
+
+  /**
+   * 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 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/1134431d/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
new file mode 100644
index 0000000..a383288
--- /dev/null
+++ b/core/src/main/java/org/apache/carbondata/core/indexstore/blockletindex/BlockletDataMapFactory.java
@@ -0,0 +1,285 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.carbondata.core.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.Segment;
+import org.apache.carbondata.core.datamap.dev.DataMap;
+import org.apache.carbondata.core.datamap.dev.DataMapWriter;
+import org.apache.carbondata.core.datamap.dev.cgdatamap.CoarseGrainDataMap;
+import org.apache.carbondata.core.datamap.dev.cgdatamap.CoarseGrainDataMapFactory;
+import org.apache.carbondata.core.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.PartitionSpec;
+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.SegmentFileStore;
+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 CoarseGrainDataMapFactory
+    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, CoarseGrainDataMap> cache;
+
+  @Override
+  public void init(AbsoluteTableIdentifier identifier, DataMapSchema dataMapSchema) {
+    this.identifier = identifier;
+    cache = CacheProvider.getInstance()
+        .createCache(CacheType.DRIVER_BLOCKLET_DATAMAP);
+  }
+
+  @Override
+  public DataMapWriter createWriter(Segment segment, String writeDirectoryPath) {
+    throw new UnsupportedOperationException("not implemented");
+  }
+
+  @Override
+  public List<CoarseGrainDataMap> getDataMaps(Segment segment) throws IOException {
+    List<TableBlockIndexUniqueIdentifier> tableBlockIndexUniqueIdentifiers =
+        getTableBlockIndexUniqueIdentifiers(segment);
+    return cache.getAll(tableBlockIndexUniqueIdentifiers);
+  }
+
+  private List<TableBlockIndexUniqueIdentifier> getTableBlockIndexUniqueIdentifiers(
+      Segment segment) throws IOException {
+    List<TableBlockIndexUniqueIdentifier> tableBlockIndexUniqueIdentifiers =
+        segmentMap.get(segment.getSegmentNo());
+    if (tableBlockIndexUniqueIdentifiers == null) {
+      tableBlockIndexUniqueIdentifiers = new ArrayList<>();
+      Map<String, String> indexFiles;
+      if (segment.getSegmentFileName() == null) {
+        String path =
+            CarbonTablePath.getSegmentPath(identifier.getTablePath(), segment.getSegmentNo());
+        indexFiles = new SegmentIndexFileStore().getIndexFilesFromSegment(path);
+      } else {
+        SegmentFileStore fileStore =
+            new SegmentFileStore(identifier.getTablePath(), segment.getSegmentFileName());
+        indexFiles = fileStore.getIndexFiles();
+      }
+      for (Map.Entry<String, String> indexFileEntry: indexFiles.entrySet()) {
+        Path indexFile = new Path(indexFileEntry.getKey());
+        tableBlockIndexUniqueIdentifiers.add(
+            new TableBlockIndexUniqueIdentifier(indexFile.getParent().toString(),
+                indexFile.getName(), indexFileEntry.getValue(), segment.getSegmentNo()));
+      }
+      segmentMap.put(segment.getSegmentNo(), 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, Segment segment)
+      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(segment);
+    // 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, Segment segment)
+      throws IOException {
+    if (blocklet instanceof ExtendedBlocklet) {
+      return (ExtendedBlocklet) blocklet;
+    }
+    List<TableBlockIndexUniqueIdentifier> identifiers =
+        getTableBlockIndexUniqueIdentifiers(segment);
+    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.getIndexFilePath().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(Segment segment) {
+    List<DataMapDistributable> distributables = new ArrayList<>();
+    try {
+      CarbonFile[] carbonIndexFiles;
+      if (segment.getSegmentFileName() == null) {
+        carbonIndexFiles = SegmentIndexFileStore.getCarbonIndexFiles(
+            CarbonTablePath.getSegmentPath(identifier.getTablePath(), segment.getSegmentNo()));
+      } else {
+        SegmentFileStore fileStore =
+            new SegmentFileStore(identifier.getTablePath(), segment.getSegmentFileName());
+        Map<String, String> indexFiles = fileStore.getIndexFiles();
+        carbonIndexFiles = new CarbonFile[indexFiles.size()];
+        int i = 0;
+        for (String indexFile : indexFiles.keySet()) {
+          carbonIndexFiles[i++] = FileFactory.getCarbonFile(indexFile);
+        }
+      }
+      for (int i = 0; i < carbonIndexFiles.length; i++) {
+        Path path = new Path(carbonIndexFiles[i].getPath());
+
+        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.toString());
+        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(Segment segment) {
+    List<TableBlockIndexUniqueIdentifier> blockIndexes = segmentMap.remove(segment.getSegmentNo());
+    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(new Segment(segmentId, null));
+    }
+  }
+
+  @Override
+  public List<CoarseGrainDataMap> getDataMaps(DataMapDistributable distributable)
+      throws IOException {
+    BlockletDataMapDistributable mapDistributable = (BlockletDataMapDistributable) distributable;
+    List<TableBlockIndexUniqueIdentifier> identifiers = new ArrayList<>();
+    Path indexPath = new Path(mapDistributable.getFilePath());
+    String segmentNo = mapDistributable.getSegment().getSegmentNo();
+    if (indexPath.getName().endsWith(CarbonTablePath.INDEX_FILE_EXT)) {
+      String parent = indexPath.getParent().toString();
+      identifiers
+          .add(new TableBlockIndexUniqueIdentifier(parent, indexPath.getName(), null, segmentNo));
+    } else if (indexPath.getName().endsWith(CarbonTablePath.MERGE_INDEX_FILE_EXT)) {
+      SegmentIndexFileStore fileStore = new SegmentIndexFileStore();
+      CarbonFile carbonFile = FileFactory.getCarbonFile(indexPath.toString());
+      String parentPath = carbonFile.getParentFile().getAbsolutePath();
+      List<String> indexFiles = fileStore.getIndexFilesFromMergeFile(carbonFile.getAbsolutePath());
+      for (String indexFile : indexFiles) {
+        identifiers.add(
+            new TableBlockIndexUniqueIdentifier(parentPath, indexFile, carbonFile.getName(),
+                segmentNo));
+      }
+    }
+    List<CoarseGrainDataMap> 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(Segment segment) throws IOException {
+    List<CoarseGrainDataMap> dataMaps = getDataMaps(segment);
+    assert (dataMaps.size() > 0);
+    CoarseGrainDataMap coarseGrainDataMap = dataMaps.get(0);
+    assert (coarseGrainDataMap instanceof BlockletDataMap);
+    BlockletDataMap dataMap = (BlockletDataMap) coarseGrainDataMap;
+    return dataMap.getSegmentProperties();
+  }
+
+  @Override public List<Blocklet> getAllBlocklets(Segment segment, List<PartitionSpec> partitions)
+      throws IOException {
+    List<Blocklet> blocklets = new ArrayList<>();
+    List<CoarseGrainDataMap> dataMaps = getDataMaps(segment);
+    for (CoarseGrainDataMap dataMap : dataMaps) {
+      blocklets.addAll(dataMap.prune(null, getSegmentProperties(segment), partitions));
+    }
+    return blocklets;
+  }
+}

http://git-wip-us.apache.org/repos/asf/carbondata/blob/1134431d/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 018c3f7..ebeb278 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
@@ -22,7 +22,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 BlockletIndexDataMap.
+ * It is the model object to keep the information to build or initialize BlockletDataMap.
  */
 public class BlockletDataMapModel extends DataMapModel {
 

http://git-wip-us.apache.org/repos/asf/carbondata/blob/1134431d/core/src/main/java/org/apache/carbondata/core/indexstore/blockletindex/BlockletDataRefNode.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/carbondata/core/indexstore/blockletindex/BlockletDataRefNode.java b/core/src/main/java/org/apache/carbondata/core/indexstore/blockletindex/BlockletDataRefNode.java
index 50862a7..a11ae8d 100644
--- a/core/src/main/java/org/apache/carbondata/core/indexstore/blockletindex/BlockletDataRefNode.java
+++ b/core/src/main/java/org/apache/carbondata/core/indexstore/blockletindex/BlockletDataRefNode.java
@@ -22,6 +22,7 @@ import java.util.List;
 import org.apache.carbondata.core.constants.CarbonV3DataFormatConstants;
 import org.apache.carbondata.core.constants.CarbonVersionConstants;
 import org.apache.carbondata.core.datamap.dev.BlockletSerializer;
+import org.apache.carbondata.core.datamap.dev.fgdatamap.FineGrainBlocklet;
 import org.apache.carbondata.core.datastore.DataRefNode;
 import org.apache.carbondata.core.datastore.FileReader;
 import org.apache.carbondata.core.datastore.block.TableBlockInfo;
@@ -31,7 +32,6 @@ import org.apache.carbondata.core.datastore.chunk.reader.CarbonDataReaderFactory
 import org.apache.carbondata.core.datastore.chunk.reader.DimensionColumnChunkReader;
 import org.apache.carbondata.core.datastore.chunk.reader.MeasureColumnChunkReader;
 import org.apache.carbondata.core.indexstore.BlockletDetailInfo;
-import org.apache.carbondata.core.indexstore.FineGrainBlocklet;
 import org.apache.carbondata.core.metadata.ColumnarFormatVersion;
 import org.apache.carbondata.core.metadata.blocklet.index.BlockletIndex;
 import org.apache.carbondata.core.util.BitSetGroup;


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

Posted by ja...@apache.org.
http://git-wip-us.apache.org/repos/asf/carbondata/blob/f8ded96e/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

http://git-wip-us.apache.org/repos/asf/carbondata/blob/f8ded96e/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 3bc4547..007ba2f 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
@@ -43,8 +43,7 @@ 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.PartitionSpec;
-import org.apache.carbondata.core.indexstore.blockletindex.BlockletDataMap;
-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;
@@ -756,7 +755,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/f8ded96e/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 b1962c1..f208c92 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
@@ -286,7 +286,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")
   }
 

http://git-wip-us.apache.org/repos/asf/carbondata/blob/f8ded96e/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 0f59949..6f78285 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
@@ -303,8 +303,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/f8ded96e/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 ec76b37..efe34c6 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
@@ -201,7 +201,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 12: USING and catch MalformedCarbonCommandException") {
@@ -216,7 +216,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 13: Only one granularity level can be defined 1") {

http://git-wip-us.apache.org/repos/asf/carbondata/blob/f8ded96e/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 041a63a..0000000
--- a/integration/spark-common-test/src/test/scala/org/apache/carbondata/spark/testsuite/datamap/CGDataMapTestCase.scala
+++ /dev/null
@@ -1,381 +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, Segment}
-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, PartitionSpec}
-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(segment: Segment, dataWritePath: String): AbstractDataMapWriter = {
-    new CGDataMapWriter(identifier, segment, dataWritePath, dataMapSchema)
-  }
-
-  /**
-   * Get the datamap for segmentid
-   */
-  override def getDataMaps(segment: Segment): java.util.List[AbstractCoarseGrainDataMap] = {
-    val file = FileFactory.getCarbonFile(
-      CarbonTablePath.getSegmentPath(identifier.getTablePath, segment.getSegmentNo))
-
-    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(segment: Segment): java.util.List[DataMapDistributable] = {
-    val file = FileFactory.getCarbonFile(
-      CarbonTablePath.getSegmentPath(identifier.getTablePath, segment.getSegmentNo))
-
-    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(segment: Segment): 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[PartitionSpec]): 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,
-    segment: Segment,
-    dataWritePath: String,
-    dataMapSchema: DataMapSchema)
-  extends AbstractDataMapWriter(identifier, segment, 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/f8ded96e/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..72a88ea
--- /dev/null
+++ b/integration/spark-common-test/src/test/scala/org/apache/carbondata/spark/testsuite/datamap/CGIndexDataMapTestCase.scala
@@ -0,0 +1,383 @@
+/*
+ * 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.{AbstractDataMapWriter, DataMapModel}
+import org.apache.carbondata.core.datamap.{DataMapDistributable, DataMapMeta, DataMapStoreManager, Segment}
+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, PartitionSpec}
+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(segment: Segment, dataWritePath: String): AbstractDataMapWriter = {
+    new CGDataMapWriter(identifier, segment, dataWritePath, dataMapSchema)
+  }
+
+  /**
+   * Get the datamap for segmentid
+   */
+  override def getDataMaps(segment: Segment): java.util.List[AbstractCoarseGrainDataMap] = {
+    val file = FileFactory.getCarbonFile(
+      CarbonTablePath.getSegmentPath(identifier.getTablePath, segment.getSegmentNo))
+
+    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(segment: Segment): java.util.List[DataMapDistributable] = {
+    val file = FileFactory.getCarbonFile(
+      CarbonTablePath.getSegmentPath(identifier.getTablePath, segment.getSegmentNo))
+
+    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(segment: Segment): 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[PartitionSpec]): 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,
+    segment: Segment,
+    dataWritePath: String,
+    dataMapSchema: DataMapSchema)
+  extends AbstractDataMapWriter(identifier, segment, 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/f8ded96e/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 00d13a9..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.{DataMapDistributable, DataMapMeta, DataMapStoreManager, Segment}
-import org.apache.carbondata.core.datamap.dev.{AbstractDataMapWriter, DataMap}
-import org.apache.carbondata.core.datamap.dev.cgdatamap.{AbstractCoarseGrainDataMap, AbstractCoarseGrainDataMapFactory}
-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.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(segment: Segment): Unit = {}
-
-  override def clear(): Unit = {}
-
-  override def getDataMaps(distributable: DataMapDistributable): util.List[AbstractCoarseGrainDataMap] = ???
-
-  override def getDataMaps(segment: Segment): util.List[AbstractCoarseGrainDataMap] = ???
-
-  override def createWriter(segment: Segment, dataWritePath: String): AbstractDataMapWriter =
-    DataMapWriterSuite.dataMapWriterC2Mock(identifier, segment, 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: Segment): 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, segment: Segment,
-      dataWritePath: String) =
-    new AbstractDataMapWriter(identifier, segment, 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/f8ded96e/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 90f0972..0000000
--- a/integration/spark-common-test/src/test/scala/org/apache/carbondata/spark/testsuite/datamap/FGDataMapTestCase.scala
+++ /dev/null
@@ -1,473 +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, Segment}
-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, FineGrainBlocklet, PartitionSpec}
-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(segment: Segment, dataWritePath: String): AbstractDataMapWriter = {
-    new FGDataMapWriter(identifier, segment, dataWritePath, dataMapSchema)
-  }
-
-  /**
-   * Get the datamap for segmentid
-   */
-  override def getDataMaps(segment: Segment): java.util.List[AbstractFineGrainDataMap] = {
-    val file = FileFactory
-      .getCarbonFile(CarbonTablePath.getSegmentPath(identifier.getTablePath, segment.getSegmentNo))
-
-    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(segment: Segment): java.util.List[DataMapDistributable] = {
-    val file = FileFactory.getCarbonFile(
-      CarbonTablePath.getSegmentPath(identifier.getTablePath, segment.getSegmentNo))
-
-    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(segment: Segment): 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[PartitionSpec]): 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 =>
-      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[Blocklet] = {
-    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
-      }
-      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,
-    segment: Segment, dataWriterPath: String, dataMapSchema: DataMapSchema)
-  extends AbstractDataMapWriter(identifier, segment, 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/f8ded96e/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..d92efe7
--- /dev/null
+++ b/integration/spark-common-test/src/test/scala/org/apache/carbondata/spark/testsuite/datamap/FGIndexDataMapTestCase.scala
@@ -0,0 +1,472 @@
+/*
+ * 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, DataMapStoreManager, Segment}
+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.{Blocklet, FineGrainBlocklet, PartitionSpec}
+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(segment: Segment, dataWritePath: String): AbstractDataMapWriter = {
+    new FGDataMapWriter(identifier, segment, dataWritePath, dataMapSchema)
+  }
+
+  /**
+   * Get the datamap for segmentid
+   */
+  override def getDataMaps(segment: Segment): java.util.List[AbstractFineGrainIndexDataMap] = {
+    val file = FileFactory
+      .getCarbonFile(CarbonTablePath.getSegmentPath(identifier.getTablePath, segment.getSegmentNo))
+
+    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(segment: Segment): java.util.List[DataMapDistributable] = {
+    val file = FileFactory.getCarbonFile(
+      CarbonTablePath.getSegmentPath(identifier.getTablePath, segment.getSegmentNo))
+
+    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(segment: Segment): 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[PartitionSpec]): 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 =>
+      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[Blocklet] = {
+    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
+      }
+      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,
+    segment: Segment, dataWriterPath: String, dataMapSchema: DataMapSchema)
+  extends AbstractDataMapWriter(identifier, segment, 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")
+  }
+}


[18/20] carbondata git commit: [CARBONDATA-2213][DataMap] Fixed wrong version for module datamap-example

Posted by ja...@apache.org.
[CARBONDATA-2213][DataMap] Fixed wrong version for module datamap-example

The version of Module ‘carbondata-datamap-example’ should be 1.4.0-snapshot instead of 1.3.0-snapshot, otherwise compilation will failed.

This closes #2011


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

Branch: refs/heads/datamap-rebase1
Commit: 65cb0d6125a541513fc90640c99d063f913326e7
Parents: 8049185
Author: xuchuanyin <xu...@hust.edu.cn>
Authored: Wed Feb 28 10:31:02 2018 +0800
Committer: Jacky Li <ja...@qq.com>
Committed: Sun Mar 4 22:52:05 2018 +0800

----------------------------------------------------------------------
 datamap/examples/pom.xml                            |  8 ++++----
 .../datamap/examples/MinMaxIndexDataMapFactory.java |  1 -
 .../datamap/lucene/LuceneCoarseGrainDataMap.java    |  3 ++-
 .../lucene/LuceneCoarseGrainDataMapFactory.java     |  9 ++++++---
 .../datamap/lucene/LuceneDataMapFactoryBase.java    | 16 +++++++++++-----
 .../datamap/lucene/LuceneDataMapWriter.java         |  5 +++--
 .../datamap/lucene/LuceneFineGrainDataMap.java      |  3 ++-
 .../lucene/LuceneFineGrainDataMapFactory.java       |  9 ++++++---
 8 files changed, 34 insertions(+), 20 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/carbondata/blob/65cb0d61/datamap/examples/pom.xml
----------------------------------------------------------------------
diff --git a/datamap/examples/pom.xml b/datamap/examples/pom.xml
index 8539a86..30e1522 100644
--- a/datamap/examples/pom.xml
+++ b/datamap/examples/pom.xml
@@ -22,10 +22,10 @@
   <modelVersion>4.0.0</modelVersion>
 
   <parent>
-      <groupId>org.apache.carbondata</groupId>
-      <artifactId>carbondata-parent</artifactId>
-      <version>1.4.0-SNAPSHOT</version>
-      <relativePath>../../pom.xml</relativePath>
+    <groupId>org.apache.carbondata</groupId>
+    <artifactId>carbondata-parent</artifactId>
+    <version>1.4.0-SNAPSHOT</version>
+    <relativePath>../../pom.xml</relativePath>
   </parent>
 
   <artifactId>carbondata-datamap-examples</artifactId>

http://git-wip-us.apache.org/repos/asf/carbondata/blob/65cb0d61/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
index 9fea55b..45dee2a 100644
--- 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
@@ -25,7 +25,6 @@ import java.util.List;
 import org.apache.carbondata.core.datamap.DataMapDistributable;
 import org.apache.carbondata.core.datamap.DataMapMeta;
 import org.apache.carbondata.core.datamap.Segment;
-import org.apache.carbondata.core.datamap.dev.AbstractDataMapWriter;
 import org.apache.carbondata.core.datamap.dev.DataMapModel;
 import org.apache.carbondata.core.datamap.dev.DataMapWriter;
 import org.apache.carbondata.core.datamap.dev.cgdatamap.CoarseGrainDataMap;

http://git-wip-us.apache.org/repos/asf/carbondata/blob/65cb0d61/datamap/lucene/src/main/java/org/apache/carbondata/datamap/lucene/LuceneCoarseGrainDataMap.java
----------------------------------------------------------------------
diff --git a/datamap/lucene/src/main/java/org/apache/carbondata/datamap/lucene/LuceneCoarseGrainDataMap.java b/datamap/lucene/src/main/java/org/apache/carbondata/datamap/lucene/LuceneCoarseGrainDataMap.java
index 0b7df86..580f18b 100644
--- a/datamap/lucene/src/main/java/org/apache/carbondata/datamap/lucene/LuceneCoarseGrainDataMap.java
+++ b/datamap/lucene/src/main/java/org/apache/carbondata/datamap/lucene/LuceneCoarseGrainDataMap.java
@@ -33,6 +33,7 @@ import org.apache.carbondata.core.datamap.dev.cgdatamap.CoarseGrainDataMap;
 import org.apache.carbondata.core.datastore.block.SegmentProperties;
 import org.apache.carbondata.core.datastore.impl.FileFactory;
 import org.apache.carbondata.core.indexstore.Blocklet;
+import org.apache.carbondata.core.indexstore.PartitionSpec;
 import org.apache.carbondata.core.memory.MemoryException;
 import org.apache.carbondata.core.scan.filter.resolver.FilterResolverIntf;
 
@@ -133,7 +134,7 @@ public class LuceneCoarseGrainDataMap extends CoarseGrainDataMap {
    */
   @Override
   public List<Blocklet> prune(FilterResolverIntf filterExp, SegmentProperties segmentProperties,
-      List<String> partitions) throws IOException {
+      List<PartitionSpec> partitions) throws IOException {
 
     // convert filter expr into lucene list query
     List<String> fields = new ArrayList<String>();

http://git-wip-us.apache.org/repos/asf/carbondata/blob/65cb0d61/datamap/lucene/src/main/java/org/apache/carbondata/datamap/lucene/LuceneCoarseGrainDataMapFactory.java
----------------------------------------------------------------------
diff --git a/datamap/lucene/src/main/java/org/apache/carbondata/datamap/lucene/LuceneCoarseGrainDataMapFactory.java b/datamap/lucene/src/main/java/org/apache/carbondata/datamap/lucene/LuceneCoarseGrainDataMapFactory.java
index 7f9cc1c..04160c0 100644
--- a/datamap/lucene/src/main/java/org/apache/carbondata/datamap/lucene/LuceneCoarseGrainDataMapFactory.java
+++ b/datamap/lucene/src/main/java/org/apache/carbondata/datamap/lucene/LuceneCoarseGrainDataMapFactory.java
@@ -26,6 +26,7 @@ import org.apache.carbondata.common.logging.LogService;
 import org.apache.carbondata.common.logging.LogServiceFactory;
 import org.apache.carbondata.core.datamap.DataMapDistributable;
 import org.apache.carbondata.core.datamap.DataMapLevel;
+import org.apache.carbondata.core.datamap.Segment;
 import org.apache.carbondata.core.datamap.dev.DataMapModel;
 import org.apache.carbondata.core.datamap.dev.cgdatamap.CoarseGrainDataMap;
 import org.apache.carbondata.core.memory.MemoryException;
@@ -41,13 +42,14 @@ public class LuceneCoarseGrainDataMapFactory extends LuceneDataMapFactoryBase<Co
   /**
    * Get the datamap for segmentid
    */
-  public List<CoarseGrainDataMap> getDataMaps(String segmentId) throws IOException {
+  @Override
+  public List<CoarseGrainDataMap> getDataMaps(Segment segment) throws IOException {
     List<CoarseGrainDataMap> lstDataMap = new ArrayList<>();
     CoarseGrainDataMap dataMap = new LuceneCoarseGrainDataMap(analyzer);
     try {
       dataMap.init(new DataMapModel(
           LuceneDataMapWriter.genDataMapStorePath(
-              tableIdentifier.getTablePath(), segmentId, dataMapName)));
+              tableIdentifier.getTablePath(), segment.getSegmentNo(), dataMapName)));
     } catch (MemoryException e) {
       LOGGER.error("failed to get lucene datamap , detail is {}" + e.getMessage());
       return lstDataMap;
@@ -59,9 +61,10 @@ public class LuceneCoarseGrainDataMapFactory extends LuceneDataMapFactoryBase<Co
   /**
    * Get datamaps for distributable object.
    */
+  @Override
   public List<CoarseGrainDataMap> getDataMaps(DataMapDistributable distributable)
       throws IOException {
-    return getDataMaps(distributable.getSegmentId());
+    return getDataMaps(distributable.getSegment());
   }
 
   @Override

http://git-wip-us.apache.org/repos/asf/carbondata/blob/65cb0d61/datamap/lucene/src/main/java/org/apache/carbondata/datamap/lucene/LuceneDataMapFactoryBase.java
----------------------------------------------------------------------
diff --git a/datamap/lucene/src/main/java/org/apache/carbondata/datamap/lucene/LuceneDataMapFactoryBase.java b/datamap/lucene/src/main/java/org/apache/carbondata/datamap/lucene/LuceneDataMapFactoryBase.java
index 5eb7054..88e70d5 100644
--- a/datamap/lucene/src/main/java/org/apache/carbondata/datamap/lucene/LuceneDataMapFactoryBase.java
+++ b/datamap/lucene/src/main/java/org/apache/carbondata/datamap/lucene/LuceneDataMapFactoryBase.java
@@ -27,6 +27,7 @@ import org.apache.carbondata.common.logging.LogService;
 import org.apache.carbondata.common.logging.LogServiceFactory;
 import org.apache.carbondata.core.datamap.DataMapDistributable;
 import org.apache.carbondata.core.datamap.DataMapMeta;
+import org.apache.carbondata.core.datamap.Segment;
 import org.apache.carbondata.core.datamap.dev.DataMap;
 import org.apache.carbondata.core.datamap.dev.DataMapFactory;
 import org.apache.carbondata.core.datamap.dev.DataMapWriter;
@@ -136,23 +137,26 @@ abstract class LuceneDataMapFactoryBase<T extends DataMap> implements DataMapFac
   /**
    * Return a new write for this datamap
    */
-  public DataMapWriter createWriter(String segmentId, String writeDirectoryPath) {
+  @Override
+  public DataMapWriter createWriter(Segment segment, String writeDirectoryPath) {
     LOGGER.info("lucene data write to " + writeDirectoryPath);
     return new LuceneDataMapWriter(
-        tableIdentifier, dataMapName, segmentId, writeDirectoryPath, true);
+        tableIdentifier, dataMapName, segment, writeDirectoryPath, true);
   }
 
   /**
    * Get all distributable objects of a segmentid
    */
-  public List<DataMapDistributable> toDistributable(String segmentId) {
+  @Override
+  public List<DataMapDistributable> toDistributable(Segment segment) {
     List<DataMapDistributable> lstDataMapDistribute = new ArrayList<DataMapDistributable>();
     DataMapDistributable luceneDataMapDistributable = new LuceneDataMapDistributable(
-        CarbonTablePath.getSegmentPath(tableIdentifier.getTablePath(), segmentId));
+        CarbonTablePath.getSegmentPath(tableIdentifier.getTablePath(), segment.getSegmentNo()));
     lstDataMapDistribute.add(luceneDataMapDistributable);
     return lstDataMapDistribute;
   }
 
+  @Override
   public void fireEvent(Event event) {
 
   }
@@ -160,13 +164,15 @@ abstract class LuceneDataMapFactoryBase<T extends DataMap> implements DataMapFac
   /**
    * Clears datamap of the segment
    */
-  public void clear(String segmentId) {
+  @Override
+  public void clear(Segment segment) {
 
   }
 
   /**
    * Clear all datamaps from memory
    */
+  @Override
   public void clear() {
 
   }

http://git-wip-us.apache.org/repos/asf/carbondata/blob/65cb0d61/datamap/lucene/src/main/java/org/apache/carbondata/datamap/lucene/LuceneDataMapWriter.java
----------------------------------------------------------------------
diff --git a/datamap/lucene/src/main/java/org/apache/carbondata/datamap/lucene/LuceneDataMapWriter.java b/datamap/lucene/src/main/java/org/apache/carbondata/datamap/lucene/LuceneDataMapWriter.java
index 849fc2e..86b2382 100644
--- a/datamap/lucene/src/main/java/org/apache/carbondata/datamap/lucene/LuceneDataMapWriter.java
+++ b/datamap/lucene/src/main/java/org/apache/carbondata/datamap/lucene/LuceneDataMapWriter.java
@@ -24,6 +24,7 @@ import java.io.UnsupportedEncodingException;
 import org.apache.carbondata.common.annotations.InterfaceAudience;
 import org.apache.carbondata.common.logging.LogService;
 import org.apache.carbondata.common.logging.LogServiceFactory;
+import org.apache.carbondata.core.datamap.Segment;
 import org.apache.carbondata.core.datamap.dev.DataMapWriter;
 import org.apache.carbondata.core.datastore.impl.FileFactory;
 import org.apache.carbondata.core.datastore.page.ColumnPage;
@@ -84,9 +85,9 @@ public class LuceneDataMapWriter extends DataMapWriter {
 
   private static final String ROWID_NAME = "rowId";
 
-  LuceneDataMapWriter(AbsoluteTableIdentifier identifier, String dataMapName, String segmentId,
+  LuceneDataMapWriter(AbsoluteTableIdentifier identifier, String dataMapName, Segment segment,
       String writeDirectoryPath, boolean isFineGrain) {
-    super(identifier, segmentId, writeDirectoryPath);
+    super(identifier, segment, writeDirectoryPath);
     this.dataMapName = dataMapName;
     this.isFineGrain = isFineGrain;
   }

http://git-wip-us.apache.org/repos/asf/carbondata/blob/65cb0d61/datamap/lucene/src/main/java/org/apache/carbondata/datamap/lucene/LuceneFineGrainDataMap.java
----------------------------------------------------------------------
diff --git a/datamap/lucene/src/main/java/org/apache/carbondata/datamap/lucene/LuceneFineGrainDataMap.java b/datamap/lucene/src/main/java/org/apache/carbondata/datamap/lucene/LuceneFineGrainDataMap.java
index c649545..faa4cbe 100644
--- a/datamap/lucene/src/main/java/org/apache/carbondata/datamap/lucene/LuceneFineGrainDataMap.java
+++ b/datamap/lucene/src/main/java/org/apache/carbondata/datamap/lucene/LuceneFineGrainDataMap.java
@@ -28,6 +28,7 @@ import org.apache.carbondata.core.datamap.dev.fgdatamap.FineGrainBlocklet;
 import org.apache.carbondata.core.datamap.dev.fgdatamap.FineGrainDataMap;
 import org.apache.carbondata.core.datastore.block.SegmentProperties;
 import org.apache.carbondata.core.datastore.impl.FileFactory;
+import org.apache.carbondata.core.indexstore.PartitionSpec;
 import org.apache.carbondata.core.memory.MemoryException;
 import org.apache.carbondata.core.scan.expression.Expression;
 import org.apache.carbondata.core.scan.filter.intf.ExpressionType;
@@ -144,7 +145,7 @@ public class LuceneFineGrainDataMap extends FineGrainDataMap {
    */
   @Override
   public List<FineGrainBlocklet> prune(FilterResolverIntf filterExp,
-      SegmentProperties segmentProperties, List<String> partitions) throws IOException {
+      SegmentProperties segmentProperties, List<PartitionSpec> partitions) throws IOException {
 
     // convert filter expr into lucene list query
     List<String> fields = new ArrayList<String>();

http://git-wip-us.apache.org/repos/asf/carbondata/blob/65cb0d61/datamap/lucene/src/main/java/org/apache/carbondata/datamap/lucene/LuceneFineGrainDataMapFactory.java
----------------------------------------------------------------------
diff --git a/datamap/lucene/src/main/java/org/apache/carbondata/datamap/lucene/LuceneFineGrainDataMapFactory.java b/datamap/lucene/src/main/java/org/apache/carbondata/datamap/lucene/LuceneFineGrainDataMapFactory.java
index e35d5bf..a9376bc 100644
--- a/datamap/lucene/src/main/java/org/apache/carbondata/datamap/lucene/LuceneFineGrainDataMapFactory.java
+++ b/datamap/lucene/src/main/java/org/apache/carbondata/datamap/lucene/LuceneFineGrainDataMapFactory.java
@@ -24,6 +24,7 @@ import java.util.List;
 import org.apache.carbondata.common.annotations.InterfaceAudience;
 import org.apache.carbondata.core.datamap.DataMapDistributable;
 import org.apache.carbondata.core.datamap.DataMapLevel;
+import org.apache.carbondata.core.datamap.Segment;
 import org.apache.carbondata.core.datamap.dev.DataMapModel;
 import org.apache.carbondata.core.datamap.dev.fgdatamap.FineGrainDataMap;
 import org.apache.carbondata.core.memory.MemoryException;
@@ -37,13 +38,14 @@ public class LuceneFineGrainDataMapFactory extends LuceneDataMapFactoryBase<Fine
   /**
    * Get the datamap for segmentid
    */
-  public List<FineGrainDataMap> getDataMaps(String segmentId) throws IOException {
+  @Override
+  public List<FineGrainDataMap> getDataMaps(Segment segment) throws IOException {
     List<FineGrainDataMap> lstDataMap = new ArrayList<>();
     FineGrainDataMap dataMap = new LuceneFineGrainDataMap(analyzer);
     try {
       dataMap.init(new DataMapModel(
           LuceneDataMapWriter.genDataMapStorePath(
-              tableIdentifier.getTablePath(), segmentId, dataMapName)));
+              tableIdentifier.getTablePath(), segment.getSegmentNo(), dataMapName)));
     } catch (MemoryException e) {
       LOGGER.error("failed to get lucene datamap , detail is {}" + e.getMessage());
       return lstDataMap;
@@ -55,9 +57,10 @@ public class LuceneFineGrainDataMapFactory extends LuceneDataMapFactoryBase<Fine
   /**
    * Get datamaps for distributable object.
    */
+  @Override
   public List<FineGrainDataMap> getDataMaps(DataMapDistributable distributable)
       throws IOException {
-    return getDataMaps(distributable.getSegmentId());
+    return getDataMaps(distributable.getSegment());
   }
 
   @Override


[09/20] carbondata git commit: [HOTFIX] Add dava doc for datamap interface

Posted by ja...@apache.org.
http://git-wip-us.apache.org/repos/asf/carbondata/blob/1134431d/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
deleted file mode 100644
index a05a8c2..0000000
--- a/integration/spark-common-test/src/test/scala/org/apache/carbondata/spark/testsuite/datamap/TestIndexDataMapCommand.scala
+++ /dev/null
@@ -1,285 +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 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 datamap datamap_hiveMetaStoreTable_1 on table hiveMetaStoreTable_1")
-      checkExistence(sql("show datamap on table hiveMetaStoreTable_1"),
-        false,
-        "datamap_hiveMetaStoreTable_1")
-      assert(sql("show datamap on table hiveMetaStoreTable_1").collect().length == 0)
-      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/1134431d/integration/spark-common-test/src/test/scala/org/apache/carbondata/spark/testsuite/sortcolumns/TestSortColumns.scala
----------------------------------------------------------------------
diff --git a/integration/spark-common-test/src/test/scala/org/apache/carbondata/spark/testsuite/sortcolumns/TestSortColumns.scala b/integration/spark-common-test/src/test/scala/org/apache/carbondata/spark/testsuite/sortcolumns/TestSortColumns.scala
index 17beace..13db652 100644
--- a/integration/spark-common-test/src/test/scala/org/apache/carbondata/spark/testsuite/sortcolumns/TestSortColumns.scala
+++ b/integration/spark-common-test/src/test/scala/org/apache/carbondata/spark/testsuite/sortcolumns/TestSortColumns.scala
@@ -29,6 +29,9 @@ import org.apache.carbondata.common.exceptions.sql.MalformedCarbonCommandExcepti
 class TestSortColumns extends QueryTest with BeforeAndAfterAll {
 
   override def beforeAll {
+    CarbonProperties.getInstance().addProperty(
+      CarbonCommonConstants.CARBON_TIMESTAMP_FORMAT, "dd-MM-yyyy")
+
     SparkUtil4Test.createTaskMockUp(sqlContext)
     dropTable
     CarbonProperties.getInstance()
@@ -380,6 +383,8 @@ class TestSortColumns extends QueryTest with BeforeAndAfterAll {
 
   override def afterAll = {
     dropTable
+    CarbonProperties.getInstance().addProperty(
+      CarbonCommonConstants.CARBON_TIMESTAMP_FORMAT, CarbonCommonConstants.CARBON_TIMESTAMP_DEFAULT_FORMAT)
   }
 
   def dropTable = {

http://git-wip-us.apache.org/repos/asf/carbondata/blob/1134431d/integration/spark-common-test/src/test/scala/org/apache/carbondata/spark/testsuite/sortcolumns/TestSortColumnsWithUnsafe.scala
----------------------------------------------------------------------
diff --git a/integration/spark-common-test/src/test/scala/org/apache/carbondata/spark/testsuite/sortcolumns/TestSortColumnsWithUnsafe.scala b/integration/spark-common-test/src/test/scala/org/apache/carbondata/spark/testsuite/sortcolumns/TestSortColumnsWithUnsafe.scala
index e98b701..2cfa4c2 100644
--- a/integration/spark-common-test/src/test/scala/org/apache/carbondata/spark/testsuite/sortcolumns/TestSortColumnsWithUnsafe.scala
+++ b/integration/spark-common-test/src/test/scala/org/apache/carbondata/spark/testsuite/sortcolumns/TestSortColumnsWithUnsafe.scala
@@ -16,21 +16,29 @@
  */
 package org.apache.carbondata.spark.testsuite.sortcolumns
 
-import org.scalatest.BeforeAndAfterAll
+import org.scalatest.{BeforeAndAfterAll, BeforeAndAfterEach}
+
 import org.apache.carbondata.core.constants.CarbonCommonConstants
 import org.apache.carbondata.core.util.CarbonProperties
 import org.apache.spark.sql.test.util.QueryTest
 
-class TestSortColumnsWithUnsafe extends QueryTest with BeforeAndAfterAll {
+class TestSortColumnsWithUnsafe extends QueryTest with BeforeAndAfterAll with BeforeAndAfterEach {
 
   override def beforeAll {
     CarbonProperties.getInstance()
       .addProperty(CarbonCommonConstants.ENABLE_UNSAFE_IN_QUERY_EXECUTION, "true")
+    CarbonProperties.getInstance().addProperty(
+      CarbonCommonConstants.CARBON_TIMESTAMP_FORMAT, "dd-MM-yyyy")
     dropTable
     sql("CREATE TABLE origintable1 (empno int, empname String, designation String, doj Timestamp, workgroupcategory int, workgroupcategoryname String, deptno int, deptname String, projectcode int, projectjoindate Timestamp, projectenddate Timestamp,attendance int,utilization int,salary int) STORED BY 'org.apache.carbondata.format'")
     sql(s"""LOAD DATA local inpath '$resourcesPath/data.csv' INTO TABLE origintable1 OPTIONS('DELIMITER'= ',', 'QUOTECHAR'= '\"')""")
   }
 
+  override def beforeEach(): Unit = {
+    CarbonProperties.getInstance().addProperty(
+      CarbonCommonConstants.CARBON_TIMESTAMP_FORMAT, "dd-MM-yyyy")
+  }
+
   test("create table with no dictionary sort_columns") {
     sql("CREATE TABLE sorttable1 (empno int, empname String, designation String, doj Timestamp, workgroupcategory int, workgroupcategoryname String, deptno int, deptname String, projectcode int, projectjoindate Timestamp, projectenddate Timestamp,attendance int,utilization int,salary int) STORED BY 'org.apache.carbondata.format' tblproperties('sort_columns'='empno')")
     sql(s"""LOAD DATA local inpath '$resourcesPath/data.csv' INTO TABLE sorttable1 OPTIONS('DELIMITER'= ',', 'QUOTECHAR'= '\"')""")
@@ -249,6 +257,9 @@ class TestSortColumnsWithUnsafe extends QueryTest with BeforeAndAfterAll {
     CarbonProperties.getInstance()
       .addProperty(CarbonCommonConstants.ENABLE_UNSAFE_IN_QUERY_EXECUTION,
         CarbonCommonConstants.ENABLE_UNSAFE_IN_QUERY_EXECUTION_DEFAULTVALUE)
+    CarbonProperties.getInstance().addProperty(
+      CarbonCommonConstants.CARBON_TIMESTAMP_FORMAT, CarbonCommonConstants.CARBON_TIMESTAMP_DEFAULT_FORMAT)
+
     dropTable
   }
 

http://git-wip-us.apache.org/repos/asf/carbondata/blob/1134431d/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
index a71e0d8..10955a3 100644
--- a/integration/spark2/src/main/java/org/apache/carbondata/datamap/DataMapProvider.java
+++ b/integration/spark2/src/main/java/org/apache/carbondata/datamap/DataMapProvider.java
@@ -18,7 +18,6 @@
 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;
@@ -34,7 +33,7 @@ import org.apache.spark.sql.SparkSession;
  * <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>
+ *   <li> DataMap: index type of DataMap to accelerate filter query </li>
  * </ol>
  *
  * <p>
@@ -47,15 +46,14 @@ import org.apache.spark.sql.SparkSession;
  *   <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>
+ *   <li> class name of {@link org.apache.carbondata.core.datamap.dev.DataMapFactory}
+ * implementation: Developer can implement new type of DataMap by extending
+ * {@link org.apache.carbondata.core.datamap.dev.DataMapFactory} </li>
  * </ol>
  *
  * @since 1.4.0
  */
-@InterfaceAudience.Developer("DataMap")
-@InterfaceStability.Unstable
+@InterfaceAudience.Internal
 public interface DataMapProvider {
 
   /**

http://git-wip-us.apache.org/repos/asf/carbondata/blob/1134431d/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
index e11e522..2a6a70a 100644
--- a/integration/spark2/src/main/java/org/apache/carbondata/datamap/IndexDataMapProvider.java
+++ b/integration/spark2/src/main/java/org/apache/carbondata/datamap/IndexDataMapProvider.java
@@ -22,7 +22,7 @@ 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.datamap.dev.DataMapFactory;
 import org.apache.carbondata.core.metadata.schema.table.CarbonTable;
 import org.apache.carbondata.core.metadata.schema.table.DataMapSchema;
 import org.apache.carbondata.format.TableInfo;
@@ -38,7 +38,7 @@ public class IndexDataMapProvider implements DataMapProvider {
   @Override
   public void initMeta(CarbonTable mainTable, DataMapSchema dataMapSchema, String ctasSqlStatement,
       SparkSession sparkSession) throws MalformedDataMapCommandException {
-    IndexDataMapFactory dataMapFactory = createIndexDataMapFactory(dataMapSchema);
+    DataMapFactory dataMapFactory = createIndexDataMapFactory(dataMapSchema);
     DataMapStoreManager.getInstance().registerDataMap(
         mainTable.getAbsoluteTableIdentifier(), dataMapSchema, dataMapFactory);
     originalTableInfo = PreAggregateUtil.updateMainTable(mainTable, dataMapSchema, sparkSession);
@@ -73,13 +73,13 @@ public class IndexDataMapProvider implements DataMapProvider {
     throw new UnsupportedOperationException();
   }
 
-  private IndexDataMapFactory createIndexDataMapFactory(DataMapSchema dataMapSchema)
+  private DataMapFactory createIndexDataMapFactory(DataMapSchema dataMapSchema)
       throws MalformedDataMapCommandException {
-    IndexDataMapFactory dataMapFactory;
+    DataMapFactory dataMapFactory;
     try {
       // try to create DataMapProvider instance by taking providerName as class name
-      Class<? extends IndexDataMapFactory> providerClass =
-          (Class<? extends IndexDataMapFactory>) Class.forName(dataMapSchema.getClassName());
+      Class<? extends DataMapFactory> providerClass =
+          (Class<? extends DataMapFactory>) Class.forName(dataMapSchema.getClassName());
       dataMapFactory = providerClass.newInstance();
     } catch (ClassNotFoundException e) {
       // try to create DataMapProvider instance by taking providerName as short name
@@ -91,14 +91,14 @@ public class IndexDataMapProvider implements DataMapProvider {
     return dataMapFactory;
   }
 
-  private IndexDataMapFactory getDataMapFactoryByShortName(String providerName)
+  private DataMapFactory getDataMapFactoryByShortName(String providerName)
       throws MalformedDataMapCommandException {
-    IndexDataMapFactory dataMapFactory;
+    DataMapFactory dataMapFactory;
     String className = DataMapRegistry.getDataMapClassName(providerName);
     if (className != null) {
       try {
-        Class<? extends IndexDataMapFactory> datamapClass =
-            (Class<? extends IndexDataMapFactory>) Class.forName(providerName);
+        Class<? extends DataMapFactory> datamapClass =
+            (Class<? extends DataMapFactory>) Class.forName(providerName);
         dataMapFactory = datamapClass.newInstance();
       } catch (ClassNotFoundException ex) {
         throw new MalformedDataMapCommandException(

http://git-wip-us.apache.org/repos/asf/carbondata/blob/1134431d/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 be8d4d7..22a273b 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,14 +30,14 @@ import org.apache.carbondata.core.datamap.DataMapMeta;
 import org.apache.carbondata.core.datamap.DataMapStoreManager;
 import org.apache.carbondata.core.datamap.Segment;
 import org.apache.carbondata.core.datamap.TableDataMap;
-import org.apache.carbondata.core.datamap.dev.AbstractDataMapWriter;
-import org.apache.carbondata.core.datamap.dev.IndexDataMapFactory;
+import org.apache.carbondata.core.datamap.dev.DataMapFactory;
+import org.apache.carbondata.core.datamap.dev.DataMapWriter;
 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 IndexDataMap for one table
+ * It is for writing DataMap for one table
  */
 public class DataMapWriterListener {
 
@@ -45,26 +45,26 @@ public class DataMapWriterListener {
       DataMapWriterListener.class.getCanonicalName());
 
   // list indexed column name -> list of data map writer
-  private Map<List<String>, List<AbstractDataMapWriter>> registry = new ConcurrentHashMap<>();
+  private Map<List<String>, List<DataMapWriter>> registry = new ConcurrentHashMap<>();
 
   /**
    * register all datamap writer for specified table and segment
    */
   public void registerAllWriter(CarbonTable carbonTable, String segmentId,
       String dataWritePath) {
-    List<TableDataMap> tableDataMaps = DataMapStoreManager.getInstance().getAllDataMap(carbonTable);
-    if (tableDataMaps != null) {
-      for (TableDataMap tableDataMap : tableDataMaps) {
-        IndexDataMapFactory factory = tableDataMap.getIndexDataMapFactory();
+    List<TableDataMap> tableIndices = DataMapStoreManager.getInstance().getAllDataMap(carbonTable);
+    if (tableIndices != null) {
+      for (TableDataMap tableDataMap : tableIndices) {
+        DataMapFactory factory = tableDataMap.getDataMapFactory();
         register(factory, segmentId, dataWritePath);
       }
     }
   }
 
   /**
-   * Register a AbstractDataMapWriter
+   * Register a DataMapWriter
    */
-  private void register(IndexDataMapFactory factory, String segmentId, String dataWritePath) {
+  private void register(DataMapFactory factory, String segmentId, String dataWritePath) {
     assert (factory != null);
     assert (segmentId != null);
     DataMapMeta meta = factory.getMeta();
@@ -73,9 +73,8 @@ public class DataMapWriterListener {
       return;
     }
     List<String> columns = factory.getMeta().getIndexedColumns();
-    List<AbstractDataMapWriter> writers = registry.get(columns);
-    AbstractDataMapWriter writer = factory.createWriter(
-        new Segment(segmentId, null), dataWritePath);
+    List<DataMapWriter> writers = registry.get(columns);
+    DataMapWriter writer = factory.createWriter(new Segment(segmentId, null), dataWritePath);
     if (writers != null) {
       writers.add(writer);
     } else {
@@ -83,36 +82,36 @@ public class DataMapWriterListener {
       writers.add(writer);
       registry.put(columns, writers);
     }
-    LOG.info("AbstractDataMapWriter " + writer + " added");
+    LOG.info("DataMapWriter " + writer + " added");
   }
 
   public void onBlockStart(String blockId, String blockPath) {
-    for (List<AbstractDataMapWriter> writers : registry.values()) {
-      for (AbstractDataMapWriter writer : writers) {
+    for (List<DataMapWriter> writers : registry.values()) {
+      for (DataMapWriter writer : writers) {
         writer.onBlockStart(blockId);
       }
     }
   }
 
   public void onBlockEnd(String blockId) {
-    for (List<AbstractDataMapWriter> writers : registry.values()) {
-      for (AbstractDataMapWriter writer : writers) {
+    for (List<DataMapWriter> writers : registry.values()) {
+      for (DataMapWriter writer : writers) {
         writer.onBlockEnd(blockId);
       }
     }
   }
 
   public void onBlockletStart(int blockletId) {
-    for (List<AbstractDataMapWriter> writers : registry.values()) {
-      for (AbstractDataMapWriter writer : writers) {
+    for (List<DataMapWriter> writers : registry.values()) {
+      for (DataMapWriter writer : writers) {
         writer.onBlockletStart(blockletId);
       }
     }
   }
 
   public void onBlockletEnd(int blockletId) {
-    for (List<AbstractDataMapWriter> writers : registry.values()) {
-      for (AbstractDataMapWriter writer : writers) {
+    for (List<DataMapWriter> writers : registry.values()) {
+      for (DataMapWriter writer : writers) {
         writer.onBlockletEnd(blockletId);
       }
     }
@@ -125,15 +124,15 @@ public class DataMapWriterListener {
    * @param tablePage  page data
    */
   public void onPageAdded(int blockletId, int pageId, TablePage tablePage) {
-    Set<Map.Entry<List<String>, List<AbstractDataMapWriter>>> entries = registry.entrySet();
-    for (Map.Entry<List<String>, List<AbstractDataMapWriter>> entry : entries) {
+    Set<Map.Entry<List<String>, List<DataMapWriter>>> entries = registry.entrySet();
+    for (Map.Entry<List<String>, List<DataMapWriter>> entry : entries) {
       List<String> indexedColumns = entry.getKey();
       ColumnPage[] pages = new ColumnPage[indexedColumns.size()];
       for (int i = 0; i < indexedColumns.size(); i++) {
         pages[i] = tablePage.getColumnPage(indexedColumns.get(i));
       }
-      List<AbstractDataMapWriter> writers = entry.getValue();
-      for (AbstractDataMapWriter writer : writers) {
+      List<DataMapWriter> writers = entry.getValue();
+      for (DataMapWriter writer : writers) {
         writer.onPageAdded(blockletId, pageId, pages);
       }
     }
@@ -143,8 +142,8 @@ public class DataMapWriterListener {
    * Finish all datamap writers
    */
   public void finish() throws IOException {
-    for (List<AbstractDataMapWriter> writers : registry.values()) {
-      for (AbstractDataMapWriter writer : writers) {
+    for (List<DataMapWriter> writers : registry.values()) {
+      for (DataMapWriter writer : writers) {
         writer.finish();
       }
     }


[13/20] carbondata git commit: [HOTFIX] Add dava doc for datamap interface

Posted by ja...@apache.org.
[HOTFIX] Add dava doc for datamap interface

1. Rename some of the datamap interface
2. Add more java doc for all public class of datamap interface

This closes #1998


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

Branch: refs/heads/datamap-rebase1
Commit: 1134431dcfa22fdfc6ff0e9d26688009cf940326
Parents: f8ded96
Author: Jacky Li <ja...@qq.com>
Authored: Mon Feb 26 10:04:51 2018 +0800
Committer: Jacky Li <ja...@qq.com>
Committed: Sun Mar 4 22:43:48 2018 +0800

----------------------------------------------------------------------
 .../carbondata/core/datamap/DataMapChooser.java |   8 +-
 .../core/datamap/DataMapDistributable.java      |   2 +
 .../carbondata/core/datamap/DataMapLevel.java   |  37 +
 .../carbondata/core/datamap/DataMapMeta.java    |   7 +
 .../core/datamap/DataMapRegistry.java           |  21 +
 .../core/datamap/DataMapStoreManager.java       |  86 +-
 .../carbondata/core/datamap/DataMapType.java    |  21 -
 .../carbondata/core/datamap/TableDataMap.java   |  54 +-
 .../core/datamap/dev/AbstractDataMapWriter.java | 111 ---
 .../core/datamap/dev/BlockletSerializer.java    |   8 +-
 .../carbondata/core/datamap/dev/DataMap.java    |  58 ++
 .../core/datamap/dev/DataMapFactory.java        |  86 ++
 .../core/datamap/dev/DataMapWriter.java         | 117 +++
 .../core/datamap/dev/IndexDataMap.java          |  56 --
 .../core/datamap/dev/IndexDataMapFactory.java   |  86 --
 .../AbstractCoarseGrainIndexDataMap.java        |  24 -
 .../AbstractCoarseGrainIndexDataMapFactory.java |  34 -
 .../dev/cgdatamap/CoarseGrainDataMap.java       |  31 +
 .../cgdatamap/CoarseGrainDataMapFactory.java    |  38 +
 .../datamap/dev/expr/AndDataMapExprWrapper.java |   4 +-
 .../datamap/dev/expr/DataMapExprWrapper.java    |   4 +-
 .../dev/expr/DataMapExprWrapperImpl.java        |   6 +-
 .../datamap/dev/expr/OrDataMapExprWrapper.java  |   4 +-
 .../AbstractFineGrainIndexDataMap.java          |  24 -
 .../AbstractFineGrainIndexDataMapFactory.java   |  38 -
 .../dev/fgdatamap/FineGrainBlocklet.java        | 134 +++
 .../datamap/dev/fgdatamap/FineGrainDataMap.java |  30 +
 .../dev/fgdatamap/FineGrainDataMapFactory.java  |  44 +
 .../indexstore/BlockletDataMapIndexStore.java   |  30 +-
 .../core/indexstore/FineGrainBlocklet.java      | 128 ---
 .../blockletindex/BlockletDataMap.java          | 971 +++++++++++++++++++
 .../blockletindex/BlockletDataMapFactory.java   | 285 ++++++
 .../blockletindex/BlockletDataMapModel.java     |   2 +-
 .../blockletindex/BlockletDataRefNode.java      |   2 +-
 .../blockletindex/BlockletIndexDataMap.java     | 971 -------------------
 .../BlockletIndexDataMapFactory.java            | 285 ------
 .../core/metadata/schema/table/CarbonTable.java |   2 +-
 .../blockletindex/TestBlockletDataMap.java      |  59 ++
 .../blockletindex/TestBlockletIndexDataMap.java |  59 --
 .../datamap/examples/MinMaxDataWriter.java      |   4 +-
 .../datamap/examples/MinMaxIndexDataMap.java    |   4 +-
 .../examples/MinMaxIndexDataMapFactory.java     |  15 +-
 .../hadoop/api/CarbonTableInputFormat.java      |   8 +-
 .../testsuite/datamap/CGDataMapTestCase.scala   | 381 ++++++++
 .../datamap/CGIndexDataMapTestCase.scala        | 383 --------
 .../testsuite/datamap/DataMapWriterSuite.scala  | 216 +++++
 .../testsuite/datamap/FGDataMapTestCase.scala   | 473 +++++++++
 .../datamap/FGIndexDataMapTestCase.scala        | 472 ---------
 .../datamap/IndexDataMapWriterSuite.scala       | 217 -----
 .../testsuite/datamap/TestDataMapCommand.scala  | 285 ++++++
 .../datamap/TestIndexDataMapCommand.scala       | 285 ------
 .../testsuite/sortcolumns/TestSortColumns.scala |   5 +
 .../sortcolumns/TestSortColumnsWithUnsafe.scala |  15 +-
 .../carbondata/datamap/DataMapProvider.java     |  12 +-
 .../datamap/IndexDataMapProvider.java           |  20 +-
 .../datamap/DataMapWriterListener.java          |  55 +-
 56 files changed, 3463 insertions(+), 3354 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/carbondata/blob/1134431d/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 41e9b56..94b48c6 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
@@ -74,12 +74,12 @@ public class DataMapChooser {
       Expression expression = resolverIntf.getFilterExpression();
       // First check for FG datamaps if any exist
       List<TableDataMap> allDataMapFG =
-          DataMapStoreManager.getInstance().getAllDataMap(carbonTable, DataMapType.FG);
+          DataMapStoreManager.getInstance().getAllDataMap(carbonTable, DataMapLevel.FG);
       ExpressionTuple tuple = selectDataMap(expression, allDataMapFG);
       if (tuple.dataMapExprWrapper == null) {
         // Check for CG datamap
         List<TableDataMap> allDataMapCG =
-            DataMapStoreManager.getInstance().getAllDataMap(carbonTable, DataMapType.CG);
+            DataMapStoreManager.getInstance().getAllDataMap(carbonTable, DataMapLevel.CG);
         tuple = selectDataMap(expression, allDataMapCG);
       }
       if (tuple.dataMapExprWrapper != null) {
@@ -212,10 +212,10 @@ public class DataMapChooser {
       List<ColumnExpression> columnExpressions, Set<ExpressionType> expressionTypes) {
     List<DataMapTuple> tuples = new ArrayList<>();
     for (TableDataMap dataMap : allDataMap) {
-      if (contains(dataMap.getIndexDataMapFactory().getMeta(), columnExpressions, expressionTypes))
+      if (contains(dataMap.getDataMapFactory().getMeta(), columnExpressions, expressionTypes))
       {
         tuples.add(
-            new DataMapTuple(dataMap.getIndexDataMapFactory().getMeta().getIndexedColumns().size(),
+            new DataMapTuple(dataMap.getDataMapFactory().getMeta().getIndexedColumns().size(),
                 dataMap));
       }
     }

http://git-wip-us.apache.org/repos/asf/carbondata/blob/1134431d/core/src/main/java/org/apache/carbondata/core/datamap/DataMapDistributable.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/carbondata/core/datamap/DataMapDistributable.java b/core/src/main/java/org/apache/carbondata/core/datamap/DataMapDistributable.java
index 473d8d4..5cd8c40 100644
--- a/core/src/main/java/org/apache/carbondata/core/datamap/DataMapDistributable.java
+++ b/core/src/main/java/org/apache/carbondata/core/datamap/DataMapDistributable.java
@@ -19,6 +19,7 @@ package org.apache.carbondata.core.datamap;
 import java.io.IOException;
 import java.io.Serializable;
 
+import org.apache.carbondata.common.annotations.InterfaceAudience;
 import org.apache.carbondata.core.datastore.block.Distributable;
 import org.apache.carbondata.core.metadata.schema.table.DataMapSchema;
 
@@ -27,6 +28,7 @@ import org.apache.hadoop.mapreduce.InputSplit;
 /**
  * Distributable class for datamap.
  */
+@InterfaceAudience.Internal
 public abstract class DataMapDistributable extends InputSplit
     implements Distributable, Serializable {
 

http://git-wip-us.apache.org/repos/asf/carbondata/blob/1134431d/core/src/main/java/org/apache/carbondata/core/datamap/DataMapLevel.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/carbondata/core/datamap/DataMapLevel.java b/core/src/main/java/org/apache/carbondata/core/datamap/DataMapLevel.java
new file mode 100644
index 0000000..2c9672a
--- /dev/null
+++ b/core/src/main/java/org/apache/carbondata/core/datamap/DataMapLevel.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 org.apache.carbondata.common.annotations.InterfaceAudience;
+import org.apache.carbondata.common.annotations.InterfaceStability;
+
+/**
+ * Index level of the datamap
+ */
+@InterfaceAudience.Developer("DataMap")
+@InterfaceStability.Evolving
+public enum DataMapLevel {
+  /**
+   * Coarse Grain Index, index is of blocklet level
+   */
+  CG,
+
+  /**
+   * Fine Grain Index, index is of row level
+   */
+  FG;
+}

http://git-wip-us.apache.org/repos/asf/carbondata/blob/1134431d/core/src/main/java/org/apache/carbondata/core/datamap/DataMapMeta.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/carbondata/core/datamap/DataMapMeta.java b/core/src/main/java/org/apache/carbondata/core/datamap/DataMapMeta.java
index dd15ccb..396c5db 100644
--- a/core/src/main/java/org/apache/carbondata/core/datamap/DataMapMeta.java
+++ b/core/src/main/java/org/apache/carbondata/core/datamap/DataMapMeta.java
@@ -19,8 +19,15 @@ package org.apache.carbondata.core.datamap;
 
 import java.util.List;
 
+import org.apache.carbondata.common.annotations.InterfaceAudience;
+import org.apache.carbondata.common.annotations.InterfaceStability;
 import org.apache.carbondata.core.scan.filter.intf.ExpressionType;
 
+/**
+ * Metadata of the datamap, set by DataMap developer
+ */
+@InterfaceAudience.Developer("DataMap")
+@InterfaceStability.Evolving
 public class DataMapMeta {
 
   private List<String> indexedColumns;

http://git-wip-us.apache.org/repos/asf/carbondata/blob/1134431d/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
index 03c0c3e..1b6782a 100644
--- a/core/src/main/java/org/apache/carbondata/core/datamap/DataMapRegistry.java
+++ b/core/src/main/java/org/apache/carbondata/core/datamap/DataMapRegistry.java
@@ -21,6 +21,27 @@ import java.util.Map;
 import java.util.Objects;
 import java.util.concurrent.ConcurrentHashMap;
 
+import org.apache.carbondata.common.annotations.InterfaceAudience;
+import org.apache.carbondata.common.annotations.InterfaceStability;
+
+/**
+ * Developer can register a datamap implementation with a short name.
+ * After registration, user can use short name to create the datamap, like
+ * <p>
+ * {@code
+ *  CREATE DATAMAP dm ON TABLE table
+ *  USING 'short-name-of-the-datamap'
+ * }
+ * otherwise, user should use the class name of the datamap implementation to create the datamap
+ * (subclass of {@link org.apache.carbondata.core.datamap.dev.DataMapFactory})
+ * <p>
+ * {@code
+ *  CREATE DATAMAP dm ON TABLE table
+ *  USING 'class-name-of-the-datamap'
+ * }
+ */
+@InterfaceAudience.Developer("DataMap")
+@InterfaceStability.Evolving
 public class DataMapRegistry {
   private static Map<String, String> shortNameToClassName = new ConcurrentHashMap<>();
 

http://git-wip-us.apache.org/repos/asf/carbondata/blob/1134431d/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 ed4e18c..e57a841 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
@@ -22,14 +22,15 @@ import java.util.List;
 import java.util.Map;
 import java.util.concurrent.ConcurrentHashMap;
 
+import org.apache.carbondata.common.annotations.InterfaceAudience;
 import org.apache.carbondata.common.exceptions.MetadataProcessException;
 import org.apache.carbondata.common.exceptions.sql.MalformedDataMapCommandException;
 import org.apache.carbondata.common.logging.LogService;
 import org.apache.carbondata.common.logging.LogServiceFactory;
-import org.apache.carbondata.core.datamap.dev.IndexDataMapFactory;
+import org.apache.carbondata.core.datamap.dev.DataMapFactory;
 import org.apache.carbondata.core.indexstore.BlockletDetailsFetcher;
 import org.apache.carbondata.core.indexstore.SegmentPropertiesFetcher;
-import org.apache.carbondata.core.indexstore.blockletindex.BlockletIndexDataMapFactory;
+import org.apache.carbondata.core.indexstore.blockletindex.BlockletDataMapFactory;
 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;
@@ -41,6 +42,7 @@ import org.apache.carbondata.core.statusmanager.SegmentUpdateStatusManager;
 /**
  * It maintains all the DataMaps in it.
  */
+@InterfaceAudience.Internal
 public final class DataMapStoreManager {
 
   private static DataMapStoreManager instance = new DataMapStoreManager();
@@ -63,12 +65,12 @@ public final class DataMapStoreManager {
    * It gives all datamaps of type @mapType except the default datamap.
    *
    */
-  public List<TableDataMap> getAllDataMap(CarbonTable carbonTable, DataMapType mapType) {
+  public List<TableDataMap> getAllDataMap(CarbonTable carbonTable, DataMapLevel mapType) {
     List<TableDataMap> dataMaps = new ArrayList<>();
-    List<TableDataMap> tableDataMaps = getAllDataMap(carbonTable);
-    if (tableDataMaps != null) {
-      for (TableDataMap dataMap : tableDataMaps) {
-        if (mapType == dataMap.getIndexDataMapFactory().getDataMapType()) {
+    List<TableDataMap> tableIndices = getAllDataMap(carbonTable);
+    if (tableIndices != null) {
+      for (TableDataMap dataMap : tableIndices) {
+        if (mapType == dataMap.getDataMapFactory().getDataMapType()) {
           dataMaps.add(dataMap);
         }
       }
@@ -96,13 +98,13 @@ public final class DataMapStoreManager {
   }
 
   /**
-   * It gives the default datamap of the table. Default datamap of any table is BlockletIndexDataMap
+   * It gives the default datamap of the table. Default datamap of any table is BlockletDataMap
    *
    * @param identifier
    * @return
    */
   public TableDataMap getDefaultDataMap(AbsoluteTableIdentifier identifier) {
-    return getDataMap(identifier, BlockletIndexDataMapFactory.DATA_MAP_SCHEMA);
+    return getDataMap(identifier, BlockletDataMapFactory.DATA_MAP_SCHEMA);
   }
 
   /**
@@ -110,16 +112,16 @@ public final class DataMapStoreManager {
    */
   public TableDataMap getDataMap(AbsoluteTableIdentifier identifier, DataMapSchema dataMapSchema) {
     String table = identifier.getCarbonTableIdentifier().getTableUniqueName();
-    List<TableDataMap> tableDataMaps = allDataMaps.get(table);
+    List<TableDataMap> tableIndices = allDataMaps.get(table);
     TableDataMap dataMap = null;
-    if (tableDataMaps != null) {
-      dataMap = getTableDataMap(dataMapSchema.getDataMapName(), tableDataMaps);
+    if (tableIndices != null) {
+      dataMap = getTableDataMap(dataMapSchema.getDataMapName(), tableIndices);
     }
     if (dataMap == null) {
       synchronized (table.intern()) {
-        tableDataMaps = allDataMaps.get(table);
-        if (tableDataMaps != null) {
-          dataMap = getTableDataMap(dataMapSchema.getDataMapName(), tableDataMaps);
+        tableIndices = allDataMaps.get(table);
+        if (tableIndices != null) {
+          dataMap = getTableDataMap(dataMapSchema.getDataMapName(), tableIndices);
         }
         if (dataMap == null) {
           try {
@@ -143,12 +145,12 @@ public final class DataMapStoreManager {
    */
   private TableDataMap createAndRegisterDataMap(AbsoluteTableIdentifier identifier,
       DataMapSchema dataMapSchema) throws MalformedDataMapCommandException {
-    IndexDataMapFactory indexDataMapFactory;
+    DataMapFactory dataMapFactory;
     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();
+      // 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 = factoryClass.newInstance();
     } catch (ClassNotFoundException e) {
       throw new MalformedDataMapCommandException(
           "DataMap '" + dataMapSchema.getClassName() + "' not found");
@@ -156,39 +158,39 @@ public final class DataMapStoreManager {
       throw new MetadataProcessException(
           "failed to create DataMap '" + dataMapSchema.getClassName() + "'", e);
     }
-    return registerDataMap(identifier, dataMapSchema, indexDataMapFactory);
+    return registerDataMap(identifier, dataMapSchema, dataMapFactory);
   }
 
   public TableDataMap registerDataMap(AbsoluteTableIdentifier identifier,
-      DataMapSchema dataMapSchema,  IndexDataMapFactory indexDataMapFactory) {
+      DataMapSchema dataMapSchema,  DataMapFactory dataMapFactory) {
     String table = identifier.getCarbonTableIdentifier().getTableUniqueName();
     // Just update the segmentRefreshMap with the table if not added.
     getTableSegmentRefresher(identifier);
-    List<TableDataMap> tableDataMaps = allDataMaps.get(table);
-    if (tableDataMaps == null) {
-      tableDataMaps = new ArrayList<>();
+    List<TableDataMap> tableIndices = allDataMaps.get(table);
+    if (tableIndices == null) {
+      tableIndices = new ArrayList<>();
     }
 
-    indexDataMapFactory.init(identifier, dataMapSchema);
+    dataMapFactory.init(identifier, dataMapSchema);
     BlockletDetailsFetcher blockletDetailsFetcher;
     SegmentPropertiesFetcher segmentPropertiesFetcher = null;
-    if (indexDataMapFactory instanceof BlockletDetailsFetcher) {
-      blockletDetailsFetcher = (BlockletDetailsFetcher) indexDataMapFactory;
+    if (dataMapFactory instanceof BlockletDetailsFetcher) {
+      blockletDetailsFetcher = (BlockletDetailsFetcher) dataMapFactory;
     } else {
       blockletDetailsFetcher = getBlockletDetailsFetcher(identifier);
     }
     segmentPropertiesFetcher = (SegmentPropertiesFetcher) blockletDetailsFetcher;
-    TableDataMap dataMap = new TableDataMap(identifier, dataMapSchema, indexDataMapFactory,
+    TableDataMap dataMap = new TableDataMap(identifier, dataMapSchema, dataMapFactory,
         blockletDetailsFetcher, segmentPropertiesFetcher);
 
-    tableDataMaps.add(dataMap);
-    allDataMaps.put(table, tableDataMaps);
+    tableIndices.add(dataMap);
+    allDataMaps.put(table, tableIndices);
     return dataMap;
   }
 
-  private TableDataMap getTableDataMap(String dataMapName, List<TableDataMap> tableDataMaps) {
+  private TableDataMap getTableDataMap(String dataMapName, List<TableDataMap> tableIndices) {
     TableDataMap dataMap = null;
-    for (TableDataMap tableDataMap : tableDataMaps) {
+    for (TableDataMap tableDataMap : tableIndices) {
       if (tableDataMap.getDataMapSchema().getDataMapName().equals(dataMapName)) {
         dataMap = tableDataMap;
         break;
@@ -218,10 +220,10 @@ public final class DataMapStoreManager {
    */
   public void clearDataMaps(AbsoluteTableIdentifier identifier) {
     String tableUniqueName = identifier.getCarbonTableIdentifier().getTableUniqueName();
-    List<TableDataMap> tableDataMaps = allDataMaps.get(tableUniqueName);
+    List<TableDataMap> tableIndices = allDataMaps.get(tableUniqueName);
     segmentRefreshMap.remove(identifier.uniqueName());
-    if (tableDataMaps != null) {
-      for (TableDataMap tableDataMap : tableDataMaps) {
+    if (tableIndices != null) {
+      for (TableDataMap tableDataMap : tableIndices) {
         if (tableDataMap != null) {
           tableDataMap.clear();
           break;
@@ -237,15 +239,15 @@ public final class DataMapStoreManager {
    * @param identifier Table identifier
    */
   public void clearDataMap(AbsoluteTableIdentifier identifier, String dataMapName) {
-    List<TableDataMap> tableDataMaps =
+    List<TableDataMap> tableIndices =
         allDataMaps.get(identifier.getCarbonTableIdentifier().getTableUniqueName());
-    if (tableDataMaps != null) {
+    if (tableIndices != null) {
       int i = 0;
-      for (TableDataMap tableDataMap : tableDataMaps) {
+      for (TableDataMap tableDataMap : tableIndices) {
         if (tableDataMap != null && dataMapName
             .equalsIgnoreCase(tableDataMap.getDataMapSchema().getDataMapName())) {
           tableDataMap.clear();
-          tableDataMaps.remove(i);
+          tableIndices.remove(i);
           break;
         }
         i++;
@@ -260,8 +262,8 @@ public final class DataMapStoreManager {
    * @return
    */
   private BlockletDetailsFetcher getBlockletDetailsFetcher(AbsoluteTableIdentifier identifier) {
-    TableDataMap blockletMap = getDataMap(identifier, BlockletIndexDataMapFactory.DATA_MAP_SCHEMA);
-    return (BlockletDetailsFetcher) blockletMap.getIndexDataMapFactory();
+    TableDataMap blockletMap = getDataMap(identifier, BlockletDataMapFactory.DATA_MAP_SCHEMA);
+    return (BlockletDetailsFetcher) blockletMap.getDataMapFactory();
   }
 
   /**

http://git-wip-us.apache.org/repos/asf/carbondata/blob/1134431d/core/src/main/java/org/apache/carbondata/core/datamap/DataMapType.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/carbondata/core/datamap/DataMapType.java b/core/src/main/java/org/apache/carbondata/core/datamap/DataMapType.java
deleted file mode 100644
index bf812b3..0000000
--- a/core/src/main/java/org/apache/carbondata/core/datamap/DataMapType.java
+++ /dev/null
@@ -1,21 +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;
-
-public enum DataMapType {
-  CG,FG;
-}

http://git-wip-us.apache.org/repos/asf/carbondata/blob/1134431d/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 b9a50d2..81571ce 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
@@ -20,16 +20,17 @@ import java.io.IOException;
 import java.util.ArrayList;
 import java.util.List;
 
+import org.apache.carbondata.common.annotations.InterfaceAudience;
 import org.apache.carbondata.core.constants.CarbonCommonConstants;
 import org.apache.carbondata.core.datamap.dev.BlockletSerializer;
-import org.apache.carbondata.core.datamap.dev.IndexDataMap;
-import org.apache.carbondata.core.datamap.dev.IndexDataMapFactory;
+import org.apache.carbondata.core.datamap.dev.DataMap;
+import org.apache.carbondata.core.datamap.dev.DataMapFactory;
+import org.apache.carbondata.core.datamap.dev.fgdatamap.FineGrainBlocklet;
 import org.apache.carbondata.core.datastore.block.SegmentProperties;
 import org.apache.carbondata.core.datastore.impl.FileFactory;
 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.FineGrainBlocklet;
 import org.apache.carbondata.core.indexstore.PartitionSpec;
 import org.apache.carbondata.core.indexstore.SegmentPropertiesFetcher;
 import org.apache.carbondata.core.metadata.AbsoluteTableIdentifier;
@@ -40,16 +41,21 @@ import org.apache.carbondata.events.OperationContext;
 import org.apache.carbondata.events.OperationEventListener;
 
 /**
- * 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.
+ * Index at the table level, user can add any number of DataMap for one table, by
+ * {@code
+ *   CREATE DATAMAP dm ON TABLE table
+ *   USING 'class name of DataMapFactory implementation'
+ * }
+ * Depends on the filter condition it can prune the data (blocklet or row level).
  */
+@InterfaceAudience.Internal
 public final class TableDataMap extends OperationEventListener {
 
   private AbsoluteTableIdentifier identifier;
 
   private DataMapSchema dataMapSchema;
 
-  private IndexDataMapFactory indexDataMapFactory;
+  private DataMapFactory dataMapFactory;
 
   private BlockletDetailsFetcher blockletDetailsFetcher;
 
@@ -58,12 +64,12 @@ public final class TableDataMap extends OperationEventListener {
   /**
    * It is called to initialize and load the required table datamap metadata.
    */
-  public TableDataMap(AbsoluteTableIdentifier identifier, DataMapSchema dataMapSchema,
-      IndexDataMapFactory indexDataMapFactory, BlockletDetailsFetcher blockletDetailsFetcher,
+  TableDataMap(AbsoluteTableIdentifier identifier, DataMapSchema dataMapSchema,
+      DataMapFactory dataMapFactory, BlockletDetailsFetcher blockletDetailsFetcher,
       SegmentPropertiesFetcher segmentPropertiesFetcher) {
     this.identifier = identifier;
     this.dataMapSchema = dataMapSchema;
-    this.indexDataMapFactory = indexDataMapFactory;
+    this.dataMapFactory = dataMapFactory;
     this.blockletDetailsFetcher = blockletDetailsFetcher;
     this.segmentPropertiesFetcher = segmentPropertiesFetcher;
   }
@@ -85,9 +91,9 @@ public final class TableDataMap extends OperationEventListener {
       if (filterExp == null) {
         pruneBlocklets = blockletDetailsFetcher.getAllBlocklets(segment, partitions);
       } else {
-        List<IndexDataMap> dataMaps = indexDataMapFactory.getDataMaps(segment);
+        List<DataMap> dataMaps = dataMapFactory.getDataMaps(segment);
         segmentProperties = segmentPropertiesFetcher.getSegmentProperties(segment);
-        for (IndexDataMap dataMap : dataMaps) {
+        for (DataMap dataMap : dataMaps) {
           pruneBlocklets.addAll(dataMap.prune(filterExp, segmentProperties, partitions));
         }
       }
@@ -115,7 +121,7 @@ public final class TableDataMap extends OperationEventListener {
   public List<DataMapDistributable> toDistributable(List<Segment> segments) throws IOException {
     List<DataMapDistributable> distributables = new ArrayList<>();
     for (Segment segment : segments) {
-      List<DataMapDistributable> list = indexDataMapFactory.toDistributable(segment);
+      List<DataMapDistributable> list = dataMapFactory.toDistributable(segment);
       for (DataMapDistributable distributable: list) {
         distributable.setDataMapSchema(dataMapSchema);
         distributable.setSegment(segment);
@@ -138,10 +144,10 @@ public final class TableDataMap extends OperationEventListener {
       FilterResolverIntf filterExp, List<PartitionSpec> partitions) throws IOException {
     List<ExtendedBlocklet> detailedBlocklets = new ArrayList<>();
     List<Blocklet> blocklets = new ArrayList<>();
-    List<IndexDataMap> indexDataMaps = indexDataMapFactory.getDataMaps(distributable);
-    for (IndexDataMap indexDataMap : indexDataMaps) {
+    List<DataMap> dataMaps = dataMapFactory.getDataMaps(distributable);
+    for (DataMap dataMap : dataMaps) {
       blocklets.addAll(
-          indexDataMap.prune(
+          dataMap.prune(
               filterExp,
               segmentPropertiesFetcher.getSegmentProperties(distributable.getSegment()),
               partitions));
@@ -150,13 +156,13 @@ public final class TableDataMap extends OperationEventListener {
     String writePath =
         identifier.getTablePath() + CarbonCommonConstants.FILE_SEPARATOR + dataMapSchema
             .getDataMapName();
-    if (indexDataMapFactory.getDataMapType() == DataMapType.FG) {
+    if (dataMapFactory.getDataMapType() == DataMapLevel.FG) {
       FileFactory.mkdirs(writePath, FileFactory.getFileType(writePath));
     }
     for (Blocklet blocklet : blocklets) {
       ExtendedBlocklet detailedBlocklet =
           blockletDetailsFetcher.getExtendedBlocklet(blocklet, distributable.getSegment());
-      if (indexDataMapFactory.getDataMapType() == DataMapType.FG) {
+      if (dataMapFactory.getDataMapType() == DataMapLevel.FG) {
         String blockletwritePath =
             writePath + CarbonCommonConstants.FILE_SEPARATOR + System.nanoTime();
         detailedBlocklet.setDataMapWriterPath(blockletwritePath);
@@ -174,7 +180,7 @@ public final class TableDataMap extends OperationEventListener {
    */
   public void clear(List<Segment> segments) {
     for (Segment segment: segments) {
-      indexDataMapFactory.clear(segment);
+      dataMapFactory.clear(segment);
     }
   }
 
@@ -182,19 +188,19 @@ public final class TableDataMap extends OperationEventListener {
    * Clears all datamap
    */
   public void clear() {
-    indexDataMapFactory.clear();
+    dataMapFactory.clear();
   }
 
   public DataMapSchema getDataMapSchema() {
     return dataMapSchema;
   }
 
-  public IndexDataMapFactory getIndexDataMapFactory() {
-    return indexDataMapFactory;
+  public DataMapFactory getDataMapFactory() {
+    return dataMapFactory;
   }
 
   @Override public void onEvent(Event event, OperationContext opContext) throws Exception {
-    indexDataMapFactory.fireEvent(event);
+    dataMapFactory.fireEvent(event);
   }
 
   /**
@@ -209,8 +215,8 @@ public final class TableDataMap extends OperationEventListener {
       throws IOException {
     List<Segment> prunedSegments = new ArrayList<>(CarbonCommonConstants.DEFAULT_COLLECTION_SIZE);
     for (Segment segment : segments) {
-      List<IndexDataMap> dataMaps = indexDataMapFactory.getDataMaps(segment);
-      for (IndexDataMap dataMap : dataMaps) {
+      List<DataMap> dataMaps = dataMapFactory.getDataMaps(segment);
+      for (DataMap dataMap : dataMaps) {
         if (dataMap.isScanRequired(filterExp)) {
           // If any one task in a given segment contains the data that means the segment need to
           // be scanned and we need to validate further data maps in the same segment

http://git-wip-us.apache.org/repos/asf/carbondata/blob/1134431d/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
deleted file mode 100644
index c6961c7..0000000
--- a/core/src/main/java/org/apache/carbondata/core/datamap/dev/AbstractDataMapWriter.java
+++ /dev/null
@@ -1,111 +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 org.apache.carbondata.core.datamap.Segment;
-import org.apache.carbondata.core.datastore.impl.FileFactory;
-import org.apache.carbondata.core.datastore.page.ColumnPage;
-import org.apache.carbondata.core.metadata.AbsoluteTableIdentifier;
-import org.apache.carbondata.core.util.CarbonUtil;
-import org.apache.carbondata.core.util.path.CarbonTablePath;
-
-/**
- * Data Map writer
- */
-public abstract class AbstractDataMapWriter {
-
-  protected AbsoluteTableIdentifier identifier;
-
-  protected String segmentId;
-
-  protected String writeDirectoryPath;
-
-  public AbstractDataMapWriter(AbsoluteTableIdentifier identifier, Segment segment,
-      String writeDirectoryPath) {
-    this.identifier = identifier;
-    this.segmentId = segment.getSegmentNo();
-    this.writeDirectoryPath = writeDirectoryPath;
-  }
-
-  /**
-   * Start of new block notification.
-   *
-   * @param blockId file name of the carbondata file
-   */
-  public abstract void onBlockStart(String blockId);
-
-  /**
-   * End of block notification
-   */
-  public abstract void onBlockEnd(String blockId);
-
-  /**
-   * Start of new blocklet notification.
-   *
-   * @param blockletId sequence number of blocklet in the block
-   */
-  public abstract void onBlockletStart(int blockletId);
-
-  /**
-   * End of blocklet notification
-   *
-   * @param blockletId sequence number of blocklet in the block
-   */
-  public abstract void onBlockletEnd(int blockletId);
-
-  /**
-   * 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.
-   */
-  public abstract void onPageAdded(int blockletId, int pageId, ColumnPage[] pages);
-
-  /**
-   * This is called during closing of writer.So after this call no more data will be sent to this
-   * class.
-   */
-  public abstract void finish() throws IOException;
-
-  /**
-   * It copies the file from temp folder to actual folder
-   *
-   * @param dataMapFile
-   * @throws IOException
-   */
-  protected void commitFile(String dataMapFile) throws IOException {
-    if (!dataMapFile.startsWith(writeDirectoryPath)) {
-      throw new UnsupportedOperationException(
-          "Datamap file " + dataMapFile + " is not written in provided directory path "
-              + writeDirectoryPath);
-    }
-    String dataMapFileName =
-        dataMapFile.substring(writeDirectoryPath.length(), dataMapFile.length());
-    String carbonFilePath = dataMapFileName.substring(0, dataMapFileName.lastIndexOf("/"));
-    String segmentPath = CarbonTablePath.getSegmentPath(identifier.getTablePath(), segmentId);
-    if (carbonFilePath.length() > 0) {
-      carbonFilePath = segmentPath + carbonFilePath;
-      FileFactory.mkdirs(carbonFilePath, FileFactory.getFileType(carbonFilePath));
-    } else {
-      carbonFilePath = segmentPath;
-    }
-    CarbonUtil.copyCarbonDataFileToCarbonStorePath(dataMapFile, carbonFilePath, 0);
-  }
-
-}

http://git-wip-us.apache.org/repos/asf/carbondata/blob/1134431d/core/src/main/java/org/apache/carbondata/core/datamap/dev/BlockletSerializer.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/carbondata/core/datamap/dev/BlockletSerializer.java b/core/src/main/java/org/apache/carbondata/core/datamap/dev/BlockletSerializer.java
index 3d4c717..bd5f994 100644
--- a/core/src/main/java/org/apache/carbondata/core/datamap/dev/BlockletSerializer.java
+++ b/core/src/main/java/org/apache/carbondata/core/datamap/dev/BlockletSerializer.java
@@ -20,9 +20,15 @@ import java.io.DataInputStream;
 import java.io.DataOutputStream;
 import java.io.IOException;
 
+import org.apache.carbondata.common.annotations.InterfaceAudience;
+import org.apache.carbondata.core.datamap.dev.fgdatamap.FineGrainBlocklet;
 import org.apache.carbondata.core.datastore.impl.FileFactory;
-import org.apache.carbondata.core.indexstore.FineGrainBlocklet;
 
+/**
+ * A serializer/deserializer for {@link FineGrainBlocklet}, it is used after prune the data
+ * by {@link org.apache.carbondata.core.datamap.dev.fgdatamap.FineGrainDataMap}
+ */
+@InterfaceAudience.Internal
 public class BlockletSerializer {
 
   /**

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

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

http://git-wip-us.apache.org/repos/asf/carbondata/blob/1134431d/core/src/main/java/org/apache/carbondata/core/datamap/dev/DataMapWriter.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/carbondata/core/datamap/dev/DataMapWriter.java b/core/src/main/java/org/apache/carbondata/core/datamap/dev/DataMapWriter.java
new file mode 100644
index 0000000..4911f7b
--- /dev/null
+++ b/core/src/main/java/org/apache/carbondata/core/datamap/dev/DataMapWriter.java
@@ -0,0 +1,117 @@
+/*
+ * 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 org.apache.carbondata.common.annotations.InterfaceAudience;
+import org.apache.carbondata.common.annotations.InterfaceStability;
+import org.apache.carbondata.core.datamap.Segment;
+import org.apache.carbondata.core.datastore.impl.FileFactory;
+import org.apache.carbondata.core.datastore.page.ColumnPage;
+import org.apache.carbondata.core.metadata.AbsoluteTableIdentifier;
+import org.apache.carbondata.core.util.CarbonUtil;
+import org.apache.carbondata.core.util.path.CarbonTablePath;
+
+/**
+ * Writer interface for datamap.
+ * Developer should implement this interface to write index files.
+ * Writer will be called for every new block/blocklet/page is created when data load is executing.
+ */
+@InterfaceAudience.Developer("DataMap")
+@InterfaceStability.Evolving
+public abstract class DataMapWriter {
+
+  protected AbsoluteTableIdentifier identifier;
+
+  protected String segmentId;
+
+  protected String writeDirectoryPath;
+
+  public DataMapWriter(AbsoluteTableIdentifier identifier, Segment segment,
+      String writeDirectoryPath) {
+    this.identifier = identifier;
+    this.segmentId = segment.getSegmentNo();
+    this.writeDirectoryPath = writeDirectoryPath;
+  }
+
+  /**
+   * Start of new block notification.
+   *
+   * @param blockId file name of the carbondata file
+   */
+  public abstract void onBlockStart(String blockId);
+
+  /**
+   * End of block notification
+   */
+  public abstract void onBlockEnd(String blockId);
+
+  /**
+   * Start of new blocklet notification.
+   *
+   * @param blockletId sequence number of blocklet in the block
+   */
+  public abstract void onBlockletStart(int blockletId);
+
+  /**
+   * End of blocklet notification
+   *
+   * @param blockletId sequence number of blocklet in the block
+   */
+  public abstract void onBlockletEnd(int blockletId);
+
+  /**
+   * 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.
+   */
+  public abstract void onPageAdded(int blockletId, int pageId, ColumnPage[] pages);
+
+  /**
+   * This is called during closing of writer.So after this call no more data will be sent to this
+   * class.
+   */
+  public abstract void finish() throws IOException;
+
+  /**
+   * It commits the index file by copying the file from temp folder to actual folder
+   *
+   * @param dataMapFile file path of index file
+   * @throws IOException if IO fails
+   */
+  protected void commitFile(String dataMapFile) throws IOException {
+    if (!dataMapFile.startsWith(writeDirectoryPath)) {
+      throw new UnsupportedOperationException(
+          "Datamap file " + dataMapFile + " is not written in provided directory path "
+              + writeDirectoryPath);
+    }
+    String dataMapFileName =
+        dataMapFile.substring(writeDirectoryPath.length(), dataMapFile.length());
+    String carbonFilePath = dataMapFileName.substring(0, dataMapFileName.lastIndexOf("/"));
+    String segmentPath = CarbonTablePath.getSegmentPath(identifier.getTablePath(), segmentId);
+    if (carbonFilePath.length() > 0) {
+      carbonFilePath = segmentPath + carbonFilePath;
+      FileFactory.mkdirs(carbonFilePath, FileFactory.getFileType(carbonFilePath));
+    } else {
+      carbonFilePath = segmentPath;
+    }
+    CarbonUtil.copyCarbonDataFileToCarbonStorePath(dataMapFile, carbonFilePath, 0);
+  }
+
+}

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

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

http://git-wip-us.apache.org/repos/asf/carbondata/blob/1134431d/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
deleted file mode 100644
index df9d4e8..0000000
--- a/core/src/main/java/org/apache/carbondata/core/datamap/dev/cgdatamap/AbstractCoarseGrainIndexDataMap.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.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/1134431d/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
deleted file mode 100644
index 037c32e..0000000
--- a/core/src/main/java/org/apache/carbondata/core/datamap/dev/cgdatamap/AbstractCoarseGrainIndexDataMapFactory.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.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/1134431d/core/src/main/java/org/apache/carbondata/core/datamap/dev/cgdatamap/CoarseGrainDataMap.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/carbondata/core/datamap/dev/cgdatamap/CoarseGrainDataMap.java b/core/src/main/java/org/apache/carbondata/core/datamap/dev/cgdatamap/CoarseGrainDataMap.java
new file mode 100644
index 0000000..62a1d1b
--- /dev/null
+++ b/core/src/main/java/org/apache/carbondata/core/datamap/dev/cgdatamap/CoarseGrainDataMap.java
@@ -0,0 +1,31 @@
+/*
+ * 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.common.annotations.InterfaceAudience;
+import org.apache.carbondata.common.annotations.InterfaceStability;
+import org.apache.carbondata.core.datamap.dev.DataMap;
+import org.apache.carbondata.core.indexstore.Blocklet;
+
+/**
+ * DataMap for Coarse Grain level, see {@link org.apache.carbondata.core.datamap.DataMapLevel#CG}
+ */
+@InterfaceAudience.Developer("DataMap")
+@InterfaceStability.Evolving
+public abstract class CoarseGrainDataMap implements DataMap<Blocklet> {
+
+}

http://git-wip-us.apache.org/repos/asf/carbondata/blob/1134431d/core/src/main/java/org/apache/carbondata/core/datamap/dev/cgdatamap/CoarseGrainDataMapFactory.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/carbondata/core/datamap/dev/cgdatamap/CoarseGrainDataMapFactory.java b/core/src/main/java/org/apache/carbondata/core/datamap/dev/cgdatamap/CoarseGrainDataMapFactory.java
new file mode 100644
index 0000000..4d20cdb
--- /dev/null
+++ b/core/src/main/java/org/apache/carbondata/core/datamap/dev/cgdatamap/CoarseGrainDataMapFactory.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.cgdatamap;
+
+import org.apache.carbondata.common.annotations.InterfaceAudience;
+import org.apache.carbondata.common.annotations.InterfaceStability;
+import org.apache.carbondata.core.datamap.DataMapLevel;
+import org.apache.carbondata.core.datamap.dev.DataMapFactory;
+
+/**
+ *  Factory for {@link CoarseGrainDataMap}
+ *  1. Any filter query which hits the table with datamap will call prune method of CGdatamap.
+ *  2. The prune method of CGDatamap return list Blocklet , these blocklets contain the
+ *     information of block and blocklet.
+ *  3. Based on the splits scanrdd schedule the tasks.
+ */
+@InterfaceAudience.Developer("DataMap")
+@InterfaceStability.Evolving
+public abstract class CoarseGrainDataMapFactory implements DataMapFactory<CoarseGrainDataMap> {
+
+  @Override public DataMapLevel getDataMapType() {
+    return DataMapLevel.CG;
+  }
+}

http://git-wip-us.apache.org/repos/asf/carbondata/blob/1134431d/core/src/main/java/org/apache/carbondata/core/datamap/dev/expr/AndDataMapExprWrapper.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/carbondata/core/datamap/dev/expr/AndDataMapExprWrapper.java b/core/src/main/java/org/apache/carbondata/core/datamap/dev/expr/AndDataMapExprWrapper.java
index 2abe253..12b60b4 100644
--- a/core/src/main/java/org/apache/carbondata/core/datamap/dev/expr/AndDataMapExprWrapper.java
+++ b/core/src/main/java/org/apache/carbondata/core/datamap/dev/expr/AndDataMapExprWrapper.java
@@ -20,8 +20,8 @@ import java.io.IOException;
 import java.util.ArrayList;
 import java.util.List;
 
-import org.apache.carbondata.core.datamap.DataMapType;
 import org.apache.carbondata.core.datamap.Segment;
+import org.apache.carbondata.core.datamap.DataMapLevel;
 import org.apache.carbondata.core.indexstore.ExtendedBlocklet;
 import org.apache.carbondata.core.indexstore.PartitionSpec;
 import org.apache.carbondata.core.scan.filter.resolver.FilterResolverIntf;
@@ -93,7 +93,7 @@ public class AndDataMapExprWrapper implements DataMapExprWrapper {
     return wrappers;
   }
 
-  @Override public DataMapType getDataMapType() {
+  @Override public DataMapLevel getDataMapType() {
     return left.getDataMapType();
   }
 }

http://git-wip-us.apache.org/repos/asf/carbondata/blob/1134431d/core/src/main/java/org/apache/carbondata/core/datamap/dev/expr/DataMapExprWrapper.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/carbondata/core/datamap/dev/expr/DataMapExprWrapper.java b/core/src/main/java/org/apache/carbondata/core/datamap/dev/expr/DataMapExprWrapper.java
index 2baba6f..ddb19e9 100644
--- a/core/src/main/java/org/apache/carbondata/core/datamap/dev/expr/DataMapExprWrapper.java
+++ b/core/src/main/java/org/apache/carbondata/core/datamap/dev/expr/DataMapExprWrapper.java
@@ -20,8 +20,8 @@ import java.io.IOException;
 import java.io.Serializable;
 import java.util.List;
 
-import org.apache.carbondata.core.datamap.DataMapType;
 import org.apache.carbondata.core.datamap.Segment;
+import org.apache.carbondata.core.datamap.DataMapLevel;
 import org.apache.carbondata.core.indexstore.ExtendedBlocklet;
 import org.apache.carbondata.core.indexstore.PartitionSpec;
 import org.apache.carbondata.core.scan.filter.resolver.FilterResolverIntf;
@@ -74,6 +74,6 @@ public interface DataMapExprWrapper extends Serializable {
    * Get the datamap type.
    * @return
    */
-  DataMapType getDataMapType();
+  DataMapLevel getDataMapType();
 
 }

http://git-wip-us.apache.org/repos/asf/carbondata/blob/1134431d/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 312fc11..d4be416 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
@@ -22,8 +22,8 @@ import java.util.List;
 import java.util.UUID;
 
 import org.apache.carbondata.core.datamap.DataMapDistributable;
-import org.apache.carbondata.core.datamap.DataMapType;
 import org.apache.carbondata.core.datamap.Segment;
+import org.apache.carbondata.core.datamap.DataMapLevel;
 import org.apache.carbondata.core.datamap.TableDataMap;
 import org.apache.carbondata.core.indexstore.ExtendedBlocklet;
 import org.apache.carbondata.core.indexstore.PartitionSpec;
@@ -82,7 +82,7 @@ public class DataMapExprWrapperImpl implements DataMapExprWrapper {
     return wrappers;
   }
 
-  @Override public DataMapType getDataMapType() {
-    return dataMap.getIndexDataMapFactory().getDataMapType();
+  @Override public DataMapLevel getDataMapType() {
+    return dataMap.getDataMapFactory().getDataMapType();
   }
 }

http://git-wip-us.apache.org/repos/asf/carbondata/blob/1134431d/core/src/main/java/org/apache/carbondata/core/datamap/dev/expr/OrDataMapExprWrapper.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/carbondata/core/datamap/dev/expr/OrDataMapExprWrapper.java b/core/src/main/java/org/apache/carbondata/core/datamap/dev/expr/OrDataMapExprWrapper.java
index 3c63202..a8bfcad 100644
--- a/core/src/main/java/org/apache/carbondata/core/datamap/dev/expr/OrDataMapExprWrapper.java
+++ b/core/src/main/java/org/apache/carbondata/core/datamap/dev/expr/OrDataMapExprWrapper.java
@@ -22,8 +22,8 @@ import java.util.HashSet;
 import java.util.List;
 import java.util.Set;
 
-import org.apache.carbondata.core.datamap.DataMapType;
 import org.apache.carbondata.core.datamap.Segment;
+import org.apache.carbondata.core.datamap.DataMapLevel;
 import org.apache.carbondata.core.indexstore.ExtendedBlocklet;
 import org.apache.carbondata.core.indexstore.PartitionSpec;
 import org.apache.carbondata.core.scan.filter.resolver.FilterResolverIntf;
@@ -90,7 +90,7 @@ public class OrDataMapExprWrapper implements DataMapExprWrapper {
   }
 
 
-  @Override public DataMapType getDataMapType() {
+  @Override public DataMapLevel getDataMapType() {
     return left.getDataMapType();
   }
 }

http://git-wip-us.apache.org/repos/asf/carbondata/blob/1134431d/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
deleted file mode 100644
index ea536b9..0000000
--- a/core/src/main/java/org/apache/carbondata/core/datamap/dev/fgdatamap/AbstractFineGrainIndexDataMap.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.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/1134431d/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
deleted file mode 100644
index 762c233..0000000
--- a/core/src/main/java/org/apache/carbondata/core/datamap/dev/fgdatamap/AbstractFineGrainIndexDataMapFactory.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.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/1134431d/core/src/main/java/org/apache/carbondata/core/datamap/dev/fgdatamap/FineGrainBlocklet.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/carbondata/core/datamap/dev/fgdatamap/FineGrainBlocklet.java b/core/src/main/java/org/apache/carbondata/core/datamap/dev/fgdatamap/FineGrainBlocklet.java
new file mode 100644
index 0000000..b42500f
--- /dev/null
+++ b/core/src/main/java/org/apache/carbondata/core/datamap/dev/fgdatamap/FineGrainBlocklet.java
@@ -0,0 +1,134 @@
+/*
+ * 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 java.io.DataInput;
+import java.io.DataOutput;
+import java.io.IOException;
+import java.io.Serializable;
+import java.util.ArrayList;
+import java.util.BitSet;
+import java.util.List;
+
+import org.apache.carbondata.common.annotations.InterfaceAudience;
+import org.apache.carbondata.common.annotations.InterfaceStability;
+import org.apache.carbondata.core.constants.CarbonV3DataFormatConstants;
+
+import org.apache.carbondata.core.indexstore.Blocklet;
+import org.apache.carbondata.core.metadata.schema.table.Writable;
+import org.apache.carbondata.core.util.BitSetGroup;
+
+/**
+ * Used for returning matched rows after pruned by {@link FineGrainDataMap}
+ */
+@InterfaceAudience.Developer("DataMap")
+@InterfaceStability.Evolving
+public class FineGrainBlocklet extends Blocklet implements Serializable {
+
+  private List<Page> pages;
+
+  public FineGrainBlocklet(String blockId, String blockletId, List<Page> pages) {
+    super(blockId, blockletId);
+    this.pages = pages;
+  }
+
+  // For serialization purpose
+  public FineGrainBlocklet() {
+
+  }
+
+  public List<Page> getPages() {
+    return pages;
+  }
+
+  public static class Page implements Writable,Serializable {
+
+    private int pageId;
+
+    private int[] rowId;
+
+    public BitSet getBitSet() {
+      BitSet bitSet =
+          new BitSet(CarbonV3DataFormatConstants.NUMBER_OF_ROWS_PER_BLOCKLET_COLUMN_PAGE_DEFAULT);
+      for (int row : rowId) {
+        bitSet.set(row);
+      }
+      return bitSet;
+    }
+
+    @Override public void write(DataOutput out) throws IOException {
+      out.writeInt(pageId);
+      out.writeInt(rowId.length);
+      for (int i = 0; i < rowId.length; i++) {
+        out.writeInt(rowId[i]);
+      }
+    }
+
+    @Override public void readFields(DataInput in) throws IOException {
+      pageId = in.readInt();
+      int length = in.readInt();
+      rowId = new int[length];
+      for (int i = 0; i < length; i++) {
+        rowId[i] = in.readInt();
+      }
+    }
+
+    public void setPageId(int pageId) {
+      this.pageId = pageId;
+    }
+
+    public void setRowId(int[] rowId) {
+      this.rowId = rowId;
+    }
+  }
+
+  public BitSetGroup getBitSetGroup(int numberOfPages) {
+    BitSetGroup bitSetGroup = new BitSetGroup(numberOfPages);
+    for (int i = 0; i < pages.size(); i++) {
+      bitSetGroup.setBitSet(pages.get(i).getBitSet(), pages.get(i).pageId);
+    }
+    return bitSetGroup;
+  }
+
+  @Override public void write(DataOutput out) throws IOException {
+    super.write(out);
+    int size = pages.size();
+    out.writeInt(size);
+    for (Page page : pages) {
+      page.write(out);
+    }
+  }
+
+  @Override public void readFields(DataInput in) throws IOException {
+    super.readFields(in);
+    int size = in.readInt();
+    pages = new ArrayList<>(size);
+    for (int i = 0; i < size; i++) {
+      Page page = new Page();
+      page.readFields(in);
+      pages.add(page);
+    }
+  }
+
+  @Override public boolean equals(Object o) {
+    return super.equals(o);
+  }
+
+  @Override public int hashCode() {
+    return super.hashCode();
+  }
+}

http://git-wip-us.apache.org/repos/asf/carbondata/blob/1134431d/core/src/main/java/org/apache/carbondata/core/datamap/dev/fgdatamap/FineGrainDataMap.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/carbondata/core/datamap/dev/fgdatamap/FineGrainDataMap.java b/core/src/main/java/org/apache/carbondata/core/datamap/dev/fgdatamap/FineGrainDataMap.java
new file mode 100644
index 0000000..18389b2
--- /dev/null
+++ b/core/src/main/java/org/apache/carbondata/core/datamap/dev/fgdatamap/FineGrainDataMap.java
@@ -0,0 +1,30 @@
+/*
+ * 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.common.annotations.InterfaceAudience;
+import org.apache.carbondata.common.annotations.InterfaceStability;
+import org.apache.carbondata.core.datamap.dev.DataMap;
+
+/**
+ * DataMap for Fine Grain level, see {@link org.apache.carbondata.core.datamap.DataMapLevel#FG}
+ */
+@InterfaceAudience.Developer("DataMap")
+@InterfaceStability.Evolving
+public abstract class FineGrainDataMap implements DataMap<FineGrainBlocklet> {
+
+}

http://git-wip-us.apache.org/repos/asf/carbondata/blob/1134431d/core/src/main/java/org/apache/carbondata/core/datamap/dev/fgdatamap/FineGrainDataMapFactory.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/carbondata/core/datamap/dev/fgdatamap/FineGrainDataMapFactory.java b/core/src/main/java/org/apache/carbondata/core/datamap/dev/fgdatamap/FineGrainDataMapFactory.java
new file mode 100644
index 0000000..0c9aaed
--- /dev/null
+++ b/core/src/main/java/org/apache/carbondata/core/datamap/dev/fgdatamap/FineGrainDataMapFactory.java
@@ -0,0 +1,44 @@
+/*
+ * 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.common.annotations.InterfaceAudience;
+import org.apache.carbondata.common.annotations.InterfaceStability;
+import org.apache.carbondata.core.datamap.DataMapLevel;
+import org.apache.carbondata.core.datamap.dev.DataMapFactory;
+
+/**
+ *  Factory for {@link FineGrainDataMap}
+ *
+ *  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.
+ */
+@InterfaceAudience.Developer("DataMap")
+@InterfaceStability.Evolving
+public abstract class FineGrainDataMapFactory
+    implements DataMapFactory<FineGrainDataMap> {
+
+  @Override public DataMapLevel getDataMapType() {
+    return DataMapLevel.FG;
+  }
+}

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


[20/20] carbondata git commit: [REBASE] Fix style after rebasing master

Posted by ja...@apache.org.
[REBASE] Fix style after rebasing master


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

Branch: refs/heads/datamap-rebase1
Commit: f9139930ec8ffe60a049e6dac2adf37f262dca59
Parents: 62e33e5
Author: Jacky Li <ja...@qq.com>
Authored: Sun Mar 4 22:57:02 2018 +0800
Committer: Jacky Li <ja...@qq.com>
Committed: Sun Mar 4 22:57:02 2018 +0800

----------------------------------------------------------------------
 .../carbondata/core/datamap/dev/expr/AndDataMapExprWrapper.java | 2 +-
 .../carbondata/core/datamap/dev/expr/DataMapExprWrapper.java    | 2 +-
 .../core/datamap/dev/expr/DataMapExprWrapperImpl.java           | 2 +-
 .../carbondata/core/datamap/dev/expr/OrDataMapExprWrapper.java  | 2 +-
 .../apache/carbondata/hadoop/api/CarbonTableInputFormat.java    | 3 ++-
 .../command/preaaggregate/PreAggregateTableHelper.scala         | 5 -----
 .../scala/org/apache/spark/sql/optimizer/CarbonFilters.scala    | 4 +---
 7 files changed, 7 insertions(+), 13 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/carbondata/blob/f9139930/core/src/main/java/org/apache/carbondata/core/datamap/dev/expr/AndDataMapExprWrapper.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/carbondata/core/datamap/dev/expr/AndDataMapExprWrapper.java b/core/src/main/java/org/apache/carbondata/core/datamap/dev/expr/AndDataMapExprWrapper.java
index 12b60b4..74469d7 100644
--- a/core/src/main/java/org/apache/carbondata/core/datamap/dev/expr/AndDataMapExprWrapper.java
+++ b/core/src/main/java/org/apache/carbondata/core/datamap/dev/expr/AndDataMapExprWrapper.java
@@ -20,8 +20,8 @@ import java.io.IOException;
 import java.util.ArrayList;
 import java.util.List;
 
-import org.apache.carbondata.core.datamap.Segment;
 import org.apache.carbondata.core.datamap.DataMapLevel;
+import org.apache.carbondata.core.datamap.Segment;
 import org.apache.carbondata.core.indexstore.ExtendedBlocklet;
 import org.apache.carbondata.core.indexstore.PartitionSpec;
 import org.apache.carbondata.core.scan.filter.resolver.FilterResolverIntf;

http://git-wip-us.apache.org/repos/asf/carbondata/blob/f9139930/core/src/main/java/org/apache/carbondata/core/datamap/dev/expr/DataMapExprWrapper.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/carbondata/core/datamap/dev/expr/DataMapExprWrapper.java b/core/src/main/java/org/apache/carbondata/core/datamap/dev/expr/DataMapExprWrapper.java
index ddb19e9..14cfc33 100644
--- a/core/src/main/java/org/apache/carbondata/core/datamap/dev/expr/DataMapExprWrapper.java
+++ b/core/src/main/java/org/apache/carbondata/core/datamap/dev/expr/DataMapExprWrapper.java
@@ -20,8 +20,8 @@ import java.io.IOException;
 import java.io.Serializable;
 import java.util.List;
 
-import org.apache.carbondata.core.datamap.Segment;
 import org.apache.carbondata.core.datamap.DataMapLevel;
+import org.apache.carbondata.core.datamap.Segment;
 import org.apache.carbondata.core.indexstore.ExtendedBlocklet;
 import org.apache.carbondata.core.indexstore.PartitionSpec;
 import org.apache.carbondata.core.scan.filter.resolver.FilterResolverIntf;

http://git-wip-us.apache.org/repos/asf/carbondata/blob/f9139930/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 d4be416..c6b011c 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
@@ -22,8 +22,8 @@ import java.util.List;
 import java.util.UUID;
 
 import org.apache.carbondata.core.datamap.DataMapDistributable;
-import org.apache.carbondata.core.datamap.Segment;
 import org.apache.carbondata.core.datamap.DataMapLevel;
+import org.apache.carbondata.core.datamap.Segment;
 import org.apache.carbondata.core.datamap.TableDataMap;
 import org.apache.carbondata.core.indexstore.ExtendedBlocklet;
 import org.apache.carbondata.core.indexstore.PartitionSpec;

http://git-wip-us.apache.org/repos/asf/carbondata/blob/f9139930/core/src/main/java/org/apache/carbondata/core/datamap/dev/expr/OrDataMapExprWrapper.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/carbondata/core/datamap/dev/expr/OrDataMapExprWrapper.java b/core/src/main/java/org/apache/carbondata/core/datamap/dev/expr/OrDataMapExprWrapper.java
index a8bfcad..37cd5dd 100644
--- a/core/src/main/java/org/apache/carbondata/core/datamap/dev/expr/OrDataMapExprWrapper.java
+++ b/core/src/main/java/org/apache/carbondata/core/datamap/dev/expr/OrDataMapExprWrapper.java
@@ -22,8 +22,8 @@ import java.util.HashSet;
 import java.util.List;
 import java.util.Set;
 
-import org.apache.carbondata.core.datamap.Segment;
 import org.apache.carbondata.core.datamap.DataMapLevel;
+import org.apache.carbondata.core.datamap.Segment;
 import org.apache.carbondata.core.indexstore.ExtendedBlocklet;
 import org.apache.carbondata.core.indexstore.PartitionSpec;
 import org.apache.carbondata.core.scan.filter.resolver.FilterResolverIntf;

http://git-wip-us.apache.org/repos/asf/carbondata/blob/f9139930/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 81b5fde..bcc487e 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
@@ -929,7 +929,8 @@ public class CarbonTableInputFormat<T> extends FileInputFormat<Void, T> {
     TableDataMap blockletMap = DataMapStoreManager.getInstance().getDefaultDataMap(identifier);
     LoadMetadataDetails[] loadMetadataDetails = SegmentStatusManager
         .readTableStatusFile(CarbonTablePath.getTableStatusFilePath(identifier.getTablePath()));
-    SegmentUpdateStatusManager updateStatusManager = new SegmentUpdateStatusManager(identifier, loadMetadataDetails);
+    SegmentUpdateStatusManager updateStatusManager = new SegmentUpdateStatusManager(
+        identifier, loadMetadataDetails);
     SegmentStatusManager.ValidAndInvalidSegmentsInfo allSegments =
         new SegmentStatusManager(identifier).getValidAndInvalidSegments(loadMetadataDetails);
     Map<String, Long> blockRowCountMapping = new HashMap<>();

http://git-wip-us.apache.org/repos/asf/carbondata/blob/f9139930/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
index 4c0e637..1f1e1e6 100644
--- 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
@@ -26,16 +26,11 @@ 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.optimizer.CarbonFilters
 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
 import org.apache.carbondata.core.metadata.schema.table.CarbonTable
-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}
 
 /**

http://git-wip-us.apache.org/repos/asf/carbondata/blob/f9139930/integration/spark2/src/main/scala/org/apache/spark/sql/optimizer/CarbonFilters.scala
----------------------------------------------------------------------
diff --git a/integration/spark2/src/main/scala/org/apache/spark/sql/optimizer/CarbonFilters.scala b/integration/spark2/src/main/scala/org/apache/spark/sql/optimizer/CarbonFilters.scala
index 2213fdb..3344449 100644
--- a/integration/spark2/src/main/scala/org/apache/spark/sql/optimizer/CarbonFilters.scala
+++ b/integration/spark2/src/main/scala/org/apache/spark/sql/optimizer/CarbonFilters.scala
@@ -40,11 +40,9 @@ import org.apache.carbondata.core.metadata.schema.table.CarbonTable
 import org.apache.carbondata.core.scan.expression.{ColumnExpression => CarbonColumnExpression, Expression => CarbonExpression, LiteralExpression => CarbonLiteralExpression, MatchExpression}
 import org.apache.carbondata.core.scan.expression.conditional._
 import org.apache.carbondata.core.scan.expression.logical.{AndExpression, FalseExpression, OrExpression}
-import org.apache.carbondata.core.util.{CarbonProperties, ThreadLocalSessionInfo}
 import org.apache.carbondata.core.scan.filter.intf.ExpressionType
-import org.apache.carbondata.core.util.ThreadLocalSessionInfo
-import org.apache.carbondata.core.util.path.CarbonTablePath
 import org.apache.carbondata.core.util.CarbonProperties
+import org.apache.carbondata.core.util.ThreadLocalSessionInfo
 import org.apache.carbondata.datamap.TextMatch
 import org.apache.carbondata.spark.CarbonAliasDecoderRelation
 import org.apache.carbondata.spark.util.CarbonScalaUtil


[11/20] carbondata git commit: [HOTFIX] Add dava doc for datamap interface

Posted by ja...@apache.org.
http://git-wip-us.apache.org/repos/asf/carbondata/blob/1134431d/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
deleted file mode 100644
index 34e11ac..0000000
--- a/core/src/main/java/org/apache/carbondata/core/indexstore/blockletindex/BlockletIndexDataMap.java
+++ /dev/null
@@ -1,971 +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.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.PartitionSpec;
-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.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.CarbonUtil;
-import org.apache.carbondata.core.util.DataFileFooterConverter;
-import org.apache.carbondata.core.util.DataTypeUtil;
-import org.apache.carbondata.core.util.path.CarbonTablePath;
-
-import org.apache.commons.lang3.StringUtils;
-import org.apache.hadoop.fs.Path;
-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 INDEX_PATH = 3;
-
-  private static int INDEX_FILE_NAME = 4;
-
-  private static int SEGMENTID = 5;
-
-  private UnsafeMemoryDMStore unsafeMemoryDMStore;
-
-  private UnsafeMemoryDMStore unsafeMemorySummaryDMStore;
-
-  private SegmentProperties segmentProperties;
-
-  private int[] columnCardinality;
-
-  @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());
-    Path path = new Path(blockletDataMapInfo.getFilePath());
-    byte[] filePath = path.getParent().toString().getBytes(CarbonCommonConstants.DEFAULT_CHARSET);
-    byte[] fileName = path.getName().toString().getBytes(CarbonCommonConstants.DEFAULT_CHARSET);
-    byte[] segmentId =
-        blockletDataMapInfo.getSegmentId().getBytes(CarbonCommonConstants.DEFAULT_CHARSET);
-    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, schemaBinary, filePath, fileName,
-            segmentId);
-      }
-      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,
-          schemaBinary,
-          filePath,
-          fileName,
-          segmentId);
-      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);
-    byte[][] maxValues = updateMaxValues(minMaxIndex.getMaxValues(), minMaxLen);
-    // update min max values in case of old store
-    byte[][] updatedMinValues =
-        CarbonUtil.updateMinMaxValues(fileFooter, maxValues, minValues, true);
-    byte[][] updatedMaxValues =
-        CarbonUtil.updateMinMaxValues(fileFooter, maxValues, minValues, false);
-    row.setRow(addMinMax(minMaxLen, schema[ordinal], updatedMinValues), ordinal);
-    // compute and set task level min values
-    addTaskMinMaxValues(summaryRow, minMaxLen,
-        unsafeMemorySummaryDMStore.getSchema()[taskMinMaxOrdinal], updatedMinValues,
-        TASK_MIN_VALUES_INDEX, true);
-    ordinal++;
-    taskMinMaxOrdinal++;
-    row.setRow(addMinMax(minMaxLen, schema[ordinal], updatedMaxValues), ordinal);
-    // compute and set task level max values
-    addTaskMinMaxValues(summaryRow, minMaxLen,
-        unsafeMemorySummaryDMStore.getSchema()[taskMinMaxOrdinal], updatedMaxValues,
-        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, byte[] schemaBinary,
-      byte[] filePath, byte[] fileName, byte[] segmentId) {
-    // 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);
-      }
-      summaryRow.setByteArray(filePath, INDEX_PATH);
-      summaryRow.setByteArray(fileName, INDEX_FILE_NAME);
-      summaryRow.setByteArray(segmentId, SEGMENTID);
-      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
-   * @throws MemoryException
-   */
-  private void createSummarySchema(SegmentProperties segmentProperties, byte[] schemaBinary,
-      byte[] filePath, byte[] fileName, byte[] segmentId)
-      throws MemoryException {
-    List<CarbonRowSchema> taskMinMaxSchemas = new ArrayList<>();
-    getMinMaxSchema(segmentProperties, taskMinMaxSchemas);
-    // for storing column schema
-    taskMinMaxSchemas.add(
-        new CarbonRowSchema.FixedCarbonRowSchema(DataTypes.BYTE_ARRAY, schemaBinary.length));
-    // for storing file path
-    taskMinMaxSchemas.add(
-        new CarbonRowSchema.FixedCarbonRowSchema(DataTypes.BYTE_ARRAY, filePath.length));
-    // for storing file name
-    taskMinMaxSchemas.add(
-        new CarbonRowSchema.FixedCarbonRowSchema(DataTypes.BYTE_ARRAY, fileName.length));
-    // for storing segmentid
-    taskMinMaxSchemas.add(
-        new CarbonRowSchema.FixedCarbonRowSchema(DataTypes.BYTE_ARRAY, segmentId.length));
-    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<>();
-    }
-    List<Blocklet> blocklets = new ArrayList<>();
-    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 {
-      // Remove B-tree jump logic as start and end key prepared is not
-      // correct for old store scenarios
-      int startIndex = 0;
-      int endIndex = unsafeMemoryDMStore.getRowCount();
-      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<PartitionSpec> partitions) {
-    if (unsafeMemoryDMStore.getRowCount() == 0) {
-      return new ArrayList<>();
-    }
-    // if it has partitioned datamap but there is no partitioned information stored, it means
-    // partitions are dropped so return empty list.
-    if (partitions != null) {
-      // First get the partitions which are stored inside datamap.
-      String[] fileDetails = getFileDetails();
-      // Check the exact match of partition information inside the stored partitions.
-      boolean found = false;
-      Path folderPath = new Path(fileDetails[0]);
-      for (PartitionSpec spec : partitions) {
-        if (folderPath.equals(spec.getLocation()) && isCorrectUUID(fileDetails, spec)) {
-          found = true;
-          break;
-        }
-      }
-      if (!found) {
-        return new ArrayList<>();
-      }
-    }
-    // Prune with filters if the partitions are existed in this datamap
-    return prune(filterExp, segmentProperties);
-  }
-
-  private boolean isCorrectUUID(String[] fileDetails, PartitionSpec spec) {
-    boolean needToScan = false;
-    if (spec.getUuid() != null) {
-      String[] split = spec.getUuid().split("_");
-      if (split[0].equals(fileDetails[2]) && CarbonTablePath.DataFileUtil
-          .getTimeStampFromFileName(fileDetails[1]).equals(split[1])) {
-        needToScan = true;
-      }
-    } else {
-      needToScan = true;
-    }
-    return needToScan;
-  }
-
-  /**
-   * 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;
-  }
-
-  private String[] getFileDetails() {
-    try {
-      String[] fileDetails = new String[3];
-      DataMapRow unsafeRow = unsafeMemorySummaryDMStore.getUnsafeRow(0);
-      fileDetails[0] =
-          new String(unsafeRow.getByteArray(INDEX_PATH), CarbonCommonConstants.DEFAULT_CHARSET);
-      fileDetails[1] = new String(unsafeRow.getByteArray(INDEX_FILE_NAME),
-          CarbonCommonConstants.DEFAULT_CHARSET);
-      fileDetails[2] = new String(unsafeRow.getByteArray(SEGMENTID),
-          CarbonCommonConstants.DEFAULT_CHARSET);
-      return fileDetails;
-    } catch (Exception e) {
-      throw new RuntimeException(e);
-    }
-  }
-
-
-  /**
-   * 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 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/1134431d/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
deleted file mode 100644
index f1b97a4..0000000
--- a/core/src/main/java/org/apache/carbondata/core/indexstore/blockletindex/BlockletIndexDataMapFactory.java
+++ /dev/null
@@ -1,285 +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.Segment;
-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.PartitionSpec;
-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.SegmentFileStore;
-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(Segment segment, String writeDirectoryPath) {
-    throw new UnsupportedOperationException("not implemented");
-  }
-
-  @Override
-  public List<AbstractCoarseGrainIndexDataMap> getDataMaps(Segment segment) throws IOException {
-    List<TableBlockIndexUniqueIdentifier> tableBlockIndexUniqueIdentifiers =
-        getTableBlockIndexUniqueIdentifiers(segment);
-    return cache.getAll(tableBlockIndexUniqueIdentifiers);
-  }
-
-  private List<TableBlockIndexUniqueIdentifier> getTableBlockIndexUniqueIdentifiers(
-      Segment segment) throws IOException {
-    List<TableBlockIndexUniqueIdentifier> tableBlockIndexUniqueIdentifiers =
-        segmentMap.get(segment.getSegmentNo());
-    if (tableBlockIndexUniqueIdentifiers == null) {
-      tableBlockIndexUniqueIdentifiers = new ArrayList<>();
-      Map<String, String> indexFiles;
-      if (segment.getSegmentFileName() == null) {
-        String path =
-            CarbonTablePath.getSegmentPath(identifier.getTablePath(), segment.getSegmentNo());
-        indexFiles = new SegmentIndexFileStore().getIndexFilesFromSegment(path);
-      } else {
-        SegmentFileStore fileStore =
-            new SegmentFileStore(identifier.getTablePath(), segment.getSegmentFileName());
-        indexFiles = fileStore.getIndexFiles();
-      }
-      for (Map.Entry<String, String> indexFileEntry: indexFiles.entrySet()) {
-        Path indexFile = new Path(indexFileEntry.getKey());
-        tableBlockIndexUniqueIdentifiers.add(
-            new TableBlockIndexUniqueIdentifier(indexFile.getParent().toString(),
-                indexFile.getName(), indexFileEntry.getValue(), segment.getSegmentNo()));
-      }
-      segmentMap.put(segment.getSegmentNo(), 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, Segment segment)
-      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(segment);
-    // 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, Segment segment)
-      throws IOException {
-    if (blocklet instanceof ExtendedBlocklet) {
-      return (ExtendedBlocklet) blocklet;
-    }
-    List<TableBlockIndexUniqueIdentifier> identifiers =
-        getTableBlockIndexUniqueIdentifiers(segment);
-    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.getIndexFilePath().equals(carbonIndexFileName)) {
-        IndexDataMap dataMap = cache.get(identifier);
-        return ((BlockletIndexDataMap) dataMap).getDetailedBlocklet(blocklet.getBlockletId());
-      }
-    }
-    throw new IOException("Blocklet with blockid " + blocklet.getBlockletId() + " not found ");
-  }
-
-
-  @Override
-  public List<DataMapDistributable> toDistributable(Segment segment) {
-    List<DataMapDistributable> distributables = new ArrayList<>();
-    try {
-      CarbonFile[] carbonIndexFiles;
-      if (segment.getSegmentFileName() == null) {
-        carbonIndexFiles = SegmentIndexFileStore.getCarbonIndexFiles(
-            CarbonTablePath.getSegmentPath(identifier.getTablePath(), segment.getSegmentNo()));
-      } else {
-        SegmentFileStore fileStore =
-            new SegmentFileStore(identifier.getTablePath(), segment.getSegmentFileName());
-        Map<String, String> indexFiles = fileStore.getIndexFiles();
-        carbonIndexFiles = new CarbonFile[indexFiles.size()];
-        int i = 0;
-        for (String indexFile : indexFiles.keySet()) {
-          carbonIndexFiles[i++] = FileFactory.getCarbonFile(indexFile);
-        }
-      }
-      for (int i = 0; i < carbonIndexFiles.length; i++) {
-        Path path = new Path(carbonIndexFiles[i].getPath());
-
-        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.toString());
-        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(Segment segment) {
-    List<TableBlockIndexUniqueIdentifier> blockIndexes = segmentMap.remove(segment.getSegmentNo());
-    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(new Segment(segmentId, null));
-    }
-  }
-
-  @Override
-  public List<AbstractCoarseGrainIndexDataMap> getDataMaps(DataMapDistributable distributable)
-      throws IOException {
-    BlockletDataMapDistributable mapDistributable = (BlockletDataMapDistributable) distributable;
-    List<TableBlockIndexUniqueIdentifier> identifiers = new ArrayList<>();
-    Path indexPath = new Path(mapDistributable.getFilePath());
-    String segmentNo = mapDistributable.getSegment().getSegmentNo();
-    if (indexPath.getName().endsWith(CarbonTablePath.INDEX_FILE_EXT)) {
-      String parent = indexPath.getParent().toString();
-      identifiers
-          .add(new TableBlockIndexUniqueIdentifier(parent, indexPath.getName(), null, segmentNo));
-    } else if (indexPath.getName().endsWith(CarbonTablePath.MERGE_INDEX_FILE_EXT)) {
-      SegmentIndexFileStore fileStore = new SegmentIndexFileStore();
-      CarbonFile carbonFile = FileFactory.getCarbonFile(indexPath.toString());
-      String parentPath = carbonFile.getParentFile().getAbsolutePath();
-      List<String> indexFiles = fileStore.getIndexFilesFromMergeFile(carbonFile.getAbsolutePath());
-      for (String indexFile : indexFiles) {
-        identifiers.add(
-            new TableBlockIndexUniqueIdentifier(parentPath, indexFile, carbonFile.getName(),
-                segmentNo));
-      }
-    }
-    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(Segment segment) throws IOException {
-    List<AbstractCoarseGrainIndexDataMap> dataMaps = getDataMaps(segment);
-    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(Segment segment, List<PartitionSpec> partitions)
-      throws IOException {
-    List<Blocklet> blocklets = new ArrayList<>();
-    List<AbstractCoarseGrainIndexDataMap> dataMaps = getDataMaps(segment);
-    for (AbstractCoarseGrainIndexDataMap dataMap : dataMaps) {
-      blocklets.addAll(dataMap.prune(null, getSegmentProperties(segment), partitions));
-    }
-    return blocklets;
-  }
-}

http://git-wip-us.apache.org/repos/asf/carbondata/blob/1134431d/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 721845d..f81f1dd 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
@@ -776,7 +776,7 @@ public class CarbonTable implements Serializable {
   }
 
   /**
-   * whether this table has aggregation IndexDataMap or not
+   * whether this table has aggregation DataMap or not
    */
   public boolean hasAggregationDataMap() {
     List<DataMapSchema> dataMapSchemaList = tableInfo.getDataMapSchemaList();

http://git-wip-us.apache.org/repos/asf/carbondata/blob/1134431d/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
new file mode 100644
index 0000000..fa142aa
--- /dev/null
+++ b/core/src/test/java/org/apache/carbondata/core/indexstore/blockletindex/TestBlockletDataMap.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 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/1134431d/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
deleted file mode 100644
index 16048db..0000000
--- a/core/src/test/java/org/apache/carbondata/core/indexstore/blockletindex/TestBlockletIndexDataMap.java
+++ /dev/null
@@ -1,59 +0,0 @@
-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/1134431d/datamap/examples/src/minmaxdatamap/main/java/org/apache/carbondata/datamap/examples/MinMaxDataWriter.java
----------------------------------------------------------------------
diff --git a/datamap/examples/src/minmaxdatamap/main/java/org/apache/carbondata/datamap/examples/MinMaxDataWriter.java b/datamap/examples/src/minmaxdatamap/main/java/org/apache/carbondata/datamap/examples/MinMaxDataWriter.java
index 17c8332..e116825 100644
--- a/datamap/examples/src/minmaxdatamap/main/java/org/apache/carbondata/datamap/examples/MinMaxDataWriter.java
+++ b/datamap/examples/src/minmaxdatamap/main/java/org/apache/carbondata/datamap/examples/MinMaxDataWriter.java
@@ -29,7 +29,7 @@ import java.util.Map;
 import org.apache.carbondata.common.logging.LogService;
 import org.apache.carbondata.common.logging.LogServiceFactory;
 import org.apache.carbondata.core.datamap.Segment;
-import org.apache.carbondata.core.datamap.dev.AbstractDataMapWriter;
+import org.apache.carbondata.core.datamap.dev.DataMapWriter;
 import org.apache.carbondata.core.datastore.impl.FileFactory;
 import org.apache.carbondata.core.datastore.page.ColumnPage;
 import org.apache.carbondata.core.metadata.AbsoluteTableIdentifier;
@@ -39,7 +39,7 @@ import org.apache.carbondata.core.util.CarbonUtil;
 
 import com.google.gson.Gson;
 
-public class MinMaxDataWriter extends AbstractDataMapWriter {
+public class MinMaxDataWriter extends DataMapWriter {
 
   private static final LogService LOGGER =
       LogServiceFactory.getLogService(TableInfo.class.getName());

http://git-wip-us.apache.org/repos/asf/carbondata/blob/1134431d/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
index 8569081..bad22b2 100644
--- 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
@@ -28,7 +28,7 @@ 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.datamap.dev.cgdatamap.CoarseGrainDataMap;
 import org.apache.carbondata.core.datastore.block.SegmentProperties;
 import org.apache.carbondata.core.datastore.filesystem.CarbonFile;
 import org.apache.carbondata.core.datastore.filesystem.CarbonFileFilter;
@@ -49,7 +49,7 @@ import com.google.gson.Gson;
 /**
  * Datamap implementation for min max blocklet.
  */
-public class MinMaxIndexDataMap extends AbstractCoarseGrainIndexDataMap {
+public class MinMaxIndexDataMap extends CoarseGrainDataMap {
 
   private static final LogService LOGGER =
       LogServiceFactory.getLogService(MinMaxIndexDataMap.class.getName());

http://git-wip-us.apache.org/repos/asf/carbondata/blob/1134431d/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
index 9624766..9fea55b 100644
--- 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
@@ -27,8 +27,9 @@ import org.apache.carbondata.core.datamap.DataMapMeta;
 import org.apache.carbondata.core.datamap.Segment;
 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.datamap.dev.DataMapWriter;
+import org.apache.carbondata.core.datamap.dev.cgdatamap.CoarseGrainDataMap;
+import org.apache.carbondata.core.datamap.dev.cgdatamap.CoarseGrainDataMapFactory;
 import org.apache.carbondata.core.memory.MemoryException;
 import org.apache.carbondata.core.metadata.AbsoluteTableIdentifier;
 import org.apache.carbondata.core.metadata.schema.table.DataMapSchema;
@@ -39,7 +40,7 @@ import org.apache.carbondata.events.Event;
 /**
  * Min Max DataMap Factory
  */
-public class MinMaxIndexDataMapFactory extends AbstractCoarseGrainIndexDataMapFactory {
+public class MinMaxIndexDataMapFactory extends CoarseGrainDataMapFactory {
 
   private AbsoluteTableIdentifier identifier;
 
@@ -53,7 +54,7 @@ public class MinMaxIndexDataMapFactory extends AbstractCoarseGrainIndexDataMapFa
    * @param segment
    * @return
    */
-  @Override public AbstractDataMapWriter createWriter(Segment segment, String writeDirectoryPath) {
+  @Override public DataMapWriter createWriter(Segment segment, String writeDirectoryPath) {
     return new MinMaxDataWriter(identifier, segment, writeDirectoryPath);
   }
 
@@ -64,9 +65,9 @@ public class MinMaxIndexDataMapFactory extends AbstractCoarseGrainIndexDataMapFa
    * @return
    * @throws IOException
    */
-  @Override public List<AbstractCoarseGrainIndexDataMap> getDataMaps(Segment segment)
+  @Override public List<CoarseGrainDataMap> getDataMaps(Segment segment)
       throws IOException {
-    List<AbstractCoarseGrainIndexDataMap> dataMapList = new ArrayList<>();
+    List<CoarseGrainDataMap> dataMapList = new ArrayList<>();
     // Form a dataMap of Type MinMaxIndexDataMap.
     MinMaxIndexDataMap dataMap = new MinMaxIndexDataMap();
     try {
@@ -101,7 +102,7 @@ public class MinMaxIndexDataMapFactory extends AbstractCoarseGrainIndexDataMapFa
   @Override public void clear() {
   }
 
-  @Override public List<AbstractCoarseGrainIndexDataMap> getDataMaps(DataMapDistributable distributable)
+  @Override public List<CoarseGrainDataMap> getDataMaps(DataMapDistributable distributable)
       throws IOException {
     return null;
   }

http://git-wip-us.apache.org/repos/asf/carbondata/blob/1134431d/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 007ba2f..81b5fde 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
@@ -33,8 +33,8 @@ import java.util.Map;
 
 import org.apache.carbondata.core.constants.CarbonCommonConstants;
 import org.apache.carbondata.core.datamap.DataMapChooser;
+import org.apache.carbondata.core.datamap.DataMapLevel;
 import org.apache.carbondata.core.datamap.DataMapStoreManager;
-import org.apache.carbondata.core.datamap.DataMapType;
 import org.apache.carbondata.core.datamap.Segment;
 import org.apache.carbondata.core.datamap.TableDataMap;
 import org.apache.carbondata.core.datamap.dev.expr.DataMapExprWrapper;
@@ -43,7 +43,7 @@ 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.PartitionSpec;
-import org.apache.carbondata.core.indexstore.blockletindex.BlockletIndexDataMapFactory;
+import org.apache.carbondata.core.indexstore.blockletindex.BlockletDataMapFactory;
 import org.apache.carbondata.core.metadata.AbsoluteTableIdentifier;
 import org.apache.carbondata.core.metadata.ColumnarFormatVersion;
 import org.apache.carbondata.core.metadata.schema.PartitionInfo;
@@ -751,11 +751,11 @@ public class CarbonTableInputFormat<T> extends FileInputFormat<Void, T> {
     DataMapJob dataMapJob = getDataMapJob(job.getConfiguration());
     List<PartitionSpec> partitionsToPrune = getPartitionsToPrune(job.getConfiguration());
     List<ExtendedBlocklet> prunedBlocklets;
-    if (distributedCG || dataMapExprWrapper.getDataMapType() == DataMapType.FG) {
+    if (distributedCG || dataMapExprWrapper.getDataMapType() == DataMapLevel.FG) {
       DistributableDataMapFormat datamapDstr =
           new DistributableDataMapFormat(absoluteTableIdentifier, dataMapExprWrapper,
               segmentIds, partitionsToPrune,
-              BlockletIndexDataMapFactory.class.getName());
+              BlockletDataMapFactory.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/1134431d/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
new file mode 100644
index 0000000..3cba3ff
--- /dev/null
+++ b/integration/spark-common-test/src/test/scala/org/apache/carbondata/spark/testsuite/datamap/CGDataMapTestCase.scala
@@ -0,0 +1,381 @@
+/*
+ * 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, Segment}
+import org.apache.carbondata.core.datamap.dev.{DataMapModel, DataMapWriter}
+import org.apache.carbondata.core.datamap.dev.cgdatamap.{CoarseGrainDataMap, CoarseGrainDataMapFactory}
+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, PartitionSpec}
+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 CoarseGrainDataMapFactory {
+  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(segment: Segment, dataWritePath: String): DataMapWriter = {
+    new CGDataMapWriter(identifier, segment, dataWritePath, dataMapSchema)
+  }
+
+  /**
+   * Get the datamap for segmentid
+   */
+  override def getDataMaps(segment: Segment): java.util.List[CoarseGrainDataMap] = {
+    val file = FileFactory.getCarbonFile(
+      CarbonTablePath.getSegmentPath(identifier.getTablePath, segment.getSegmentNo))
+
+    val files = file.listFiles(new CarbonFileFilter {
+      override def accept(file: CarbonFile): Boolean = file.getName.endsWith(".datamap")
+    })
+    files.map {f =>
+      val dataMap: CoarseGrainDataMap = new CGDataMap()
+      dataMap.init(new DataMapModel(f.getCanonicalPath))
+      dataMap
+    }.toList.asJava
+  }
+
+
+  /**
+   * Get datamaps for distributable object.
+   */
+  override def getDataMaps(
+      distributable: DataMapDistributable): java.util.List[CoarseGrainDataMap] = {
+    val mapDistributable = distributable.asInstanceOf[BlockletDataMapDistributable]
+    val dataMap: CoarseGrainDataMap = 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(segment: Segment): java.util.List[DataMapDistributable] = {
+    val file = FileFactory.getCarbonFile(
+      CarbonTablePath.getSegmentPath(identifier.getTablePath, segment.getSegmentNo))
+
+    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(segment: Segment): 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 CoarseGrainDataMap {
+
+  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[PartitionSpec]): 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,
+    segment: Segment,
+    dataWritePath: String,
+    dataMapSchema: DataMapSchema)
+  extends DataMapWriter(identifier, segment, 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")
+  }
+}


[14/20] carbondata git commit: [HOTFIX] Fix timestamp issue in TestSortColumnsWithUnsafe

Posted by ja...@apache.org.
[HOTFIX] Fix timestamp issue in TestSortColumnsWithUnsafe

Fix timestamp issue in TestSortColumnsWithUnsafe

This closes #2001


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

Branch: refs/heads/datamap-rebase1
Commit: 96ee82b336b178c7a0d5a6d8aa8fd14afc7bc27e
Parents: 1134431
Author: Jacky Li <ja...@qq.com>
Authored: Tue Feb 27 13:06:02 2018 +0800
Committer: Jacky Li <ja...@qq.com>
Committed: Sun Mar 4 22:43:49 2018 +0800

----------------------------------------------------------------------
 .../testsuite/sortcolumns/TestSortColumns.scala | 14 +++++------
 .../sortcolumns/TestSortColumnsWithUnsafe.scala | 25 +++++++-------------
 2 files changed, 15 insertions(+), 24 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/carbondata/blob/96ee82b3/integration/spark-common-test/src/test/scala/org/apache/carbondata/spark/testsuite/sortcolumns/TestSortColumns.scala
----------------------------------------------------------------------
diff --git a/integration/spark-common-test/src/test/scala/org/apache/carbondata/spark/testsuite/sortcolumns/TestSortColumns.scala b/integration/spark-common-test/src/test/scala/org/apache/carbondata/spark/testsuite/sortcolumns/TestSortColumns.scala
index 13db652..adf8423 100644
--- a/integration/spark-common-test/src/test/scala/org/apache/carbondata/spark/testsuite/sortcolumns/TestSortColumns.scala
+++ b/integration/spark-common-test/src/test/scala/org/apache/carbondata/spark/testsuite/sortcolumns/TestSortColumns.scala
@@ -37,7 +37,7 @@ class TestSortColumns extends QueryTest with BeforeAndAfterAll {
     CarbonProperties.getInstance()
       .addProperty(CarbonCommonConstants.CARBON_TIMESTAMP_FORMAT, "dd-MM-yyyy")
     sql("CREATE TABLE origintable1 (empno int, empname String, designation String, doj Timestamp, workgroupcategory int, workgroupcategoryname String, deptno int, deptname String, projectcode int, projectjoindate Timestamp, projectenddate Timestamp,attendance int,utilization int,salary int) STORED BY 'org.apache.carbondata.format'")
-    sql(s"""LOAD DATA local inpath '$resourcesPath/data.csv' INTO TABLE origintable1 OPTIONS('DELIMITER'= ',', 'QUOTECHAR'= '\"')""")
+    sql(s"""LOAD DATA local inpath '$resourcesPath/data.csv' INTO TABLE origintable1 OPTIONS('DELIMITER'= ',', 'QUOTECHAR'= '\"', 'TIMESTAMPFORMAT'='dd-MM-yyyy')""")
 
     sql("CREATE TABLE tableOne(id int, name string, city string, age int) STORED BY 'org.apache.carbondata.format'")
     sql("CREATE TABLE tableTwo(id int, age int) STORED BY 'org.apache.carbondata.format'")
@@ -244,7 +244,7 @@ class TestSortColumns extends QueryTest with BeforeAndAfterAll {
       sql(s"""LOAD DATA local inpath '$resourcesPath/data.csv' INTO TABLE origintable1 OPTIONS('DELIMITER'= ',', 'QUOTECHAR'= '\"')""")
       setLoadingProperties("false", "false", "false")
       sql("CREATE TABLE unsortedtable_heap_safe (empno int, empname String, designation String, doj Timestamp, workgroupcategory int, workgroupcategoryname String, deptno int, deptname String, projectcode int, projectjoindate Timestamp, projectenddate Timestamp,attendance int,utilization int,salary int) STORED BY 'org.apache.carbondata.format' tblproperties('sort_columns'='')")
-      sql(s"""LOAD DATA local inpath '$resourcesPath/data.csv' INTO TABLE unsortedtable_heap_safe OPTIONS('DELIMITER'= ',', 'QUOTECHAR'= '\"')""")
+      sql(s"""LOAD DATA local inpath '$resourcesPath/data.csv' INTO TABLE unsortedtable_heap_safe OPTIONS('DELIMITER'= ',', 'QUOTECHAR'= '\"', 'TIMESTAMPFORMAT'='dd-MM-yyyy')""")
       checkAnswer(sql("select * from unsortedtable_heap_safe where empno = 11"), sql("select * from origintable1 where empno = 11"))
       checkAnswer(sql("select * from unsortedtable_heap_safe order by empno"), sql("select * from origintable1 order by empno"))
     } finally {
@@ -259,7 +259,7 @@ class TestSortColumns extends QueryTest with BeforeAndAfterAll {
       sql(s"""LOAD DATA local inpath '$resourcesPath/data.csv' INTO TABLE origintable1 OPTIONS('DELIMITER'= ',', 'QUOTECHAR'= '\"')""")
       setLoadingProperties("false", "true", "false")
       sql("CREATE TABLE unsortedtable_heap_unsafe (empno int, empname String, designation String, doj Timestamp, workgroupcategory int, workgroupcategoryname String, deptno int, deptname String, projectcode int, projectjoindate Timestamp, projectenddate Timestamp,attendance int,utilization int,salary int) STORED BY 'org.apache.carbondata.format' tblproperties('sort_columns'='')")
-      sql(s"""LOAD DATA local inpath '$resourcesPath/data.csv' INTO TABLE unsortedtable_heap_unsafe OPTIONS('DELIMITER'= ',', 'QUOTECHAR'= '\"')""")
+      sql(s"""LOAD DATA local inpath '$resourcesPath/data.csv' INTO TABLE unsortedtable_heap_unsafe OPTIONS('DELIMITER'= ',', 'QUOTECHAR'= '\"', 'TIMESTAMPFORMAT'='dd-MM-yyyy')""")
       checkAnswer(sql("select * from unsortedtable_heap_unsafe where empno = 11"), sql("select * from origintable1 where empno = 11"))
       checkAnswer(sql("select * from unsortedtable_heap_unsafe order by empno"), sql("select * from origintable1 order by empno"))
     } finally {
@@ -274,7 +274,7 @@ class TestSortColumns extends QueryTest with BeforeAndAfterAll {
       sql(s"""LOAD DATA local inpath '$resourcesPath/data.csv' INTO TABLE origintable1 OPTIONS('DELIMITER'= ',', 'QUOTECHAR'= '\"')""")
       setLoadingProperties("false", "false", "true")
       sql("CREATE TABLE unsortedtable_heap_inmemory (empno int, empname String, designation String, doj Timestamp, workgroupcategory int, workgroupcategoryname String, deptno int, deptname String, projectcode int, projectjoindate Timestamp, projectenddate Timestamp,attendance int,utilization int,salary int) STORED BY 'org.apache.carbondata.format' tblproperties('sort_columns'='')")
-      sql(s"""LOAD DATA local inpath '$resourcesPath/data.csv' INTO TABLE unsortedtable_heap_inmemory OPTIONS('DELIMITER'= ',', 'QUOTECHAR'= '\"')""")
+      sql(s"""LOAD DATA local inpath '$resourcesPath/data.csv' INTO TABLE unsortedtable_heap_inmemory OPTIONS('DELIMITER'= ',', 'QUOTECHAR'= '\"', 'TIMESTAMPFORMAT'='dd-MM-yyyy')""")
       checkAnswer(sql("select * from unsortedtable_heap_inmemory where empno = 11"), sql("select * from origintable1 where empno = 11"))
       checkAnswer(sql("select * from unsortedtable_heap_inmemory order by empno"), sql("select * from origintable1 order by empno"))
     } finally {
@@ -289,7 +289,7 @@ class TestSortColumns extends QueryTest with BeforeAndAfterAll {
       sql(s"""LOAD DATA local inpath '$resourcesPath/data.csv' INTO TABLE origintable1 OPTIONS('DELIMITER'= ',', 'QUOTECHAR'= '\"')""")
       setLoadingProperties("true", "false", "false")
       sql("CREATE TABLE unsortedtable_offheap_safe (empno int, empname String, designation String, doj Timestamp, workgroupcategory int, workgroupcategoryname String, deptno int, deptname String, projectcode int, projectjoindate Timestamp, projectenddate Timestamp,attendance int,utilization int,salary int) STORED BY 'org.apache.carbondata.format' tblproperties('sort_columns'='')")
-      sql(s"""LOAD DATA local inpath '$resourcesPath/data.csv' INTO TABLE unsortedtable_offheap_safe OPTIONS('DELIMITER'= ',', 'QUOTECHAR'= '\"')""")
+      sql(s"""LOAD DATA local inpath '$resourcesPath/data.csv' INTO TABLE unsortedtable_offheap_safe OPTIONS('DELIMITER'= ',', 'QUOTECHAR'= '\"', 'TIMESTAMPFORMAT'='dd-MM-yyyy')""")
       checkAnswer(sql("select * from unsortedtable_offheap_safe where empno = 11"), sql("select * from origintable1 where empno = 11"))
       checkAnswer(sql("select * from unsortedtable_offheap_safe order by empno"), sql("select * from origintable1 order by empno"))
     } finally {
@@ -304,7 +304,7 @@ class TestSortColumns extends QueryTest with BeforeAndAfterAll {
       sql(s"""LOAD DATA local inpath '$resourcesPath/data.csv' INTO TABLE origintable1 OPTIONS('DELIMITER'= ',', 'QUOTECHAR'= '\"')""")
       setLoadingProperties("true", "true", "false")
       sql("CREATE TABLE unsortedtable_offheap_unsafe (empno int, empname String, designation String, doj Timestamp, workgroupcategory int, workgroupcategoryname String, deptno int, deptname String, projectcode int, projectjoindate Timestamp, projectenddate Timestamp,attendance int,utilization int,salary int) STORED BY 'org.apache.carbondata.format' tblproperties('sort_columns'='')")
-      sql(s"""LOAD DATA local inpath '$resourcesPath/data.csv' INTO TABLE unsortedtable_offheap_unsafe OPTIONS('DELIMITER'= ',', 'QUOTECHAR'= '\"')""")
+      sql(s"""LOAD DATA local inpath '$resourcesPath/data.csv' INTO TABLE unsortedtable_offheap_unsafe OPTIONS('DELIMITER'= ',', 'QUOTECHAR'= '\"', 'TIMESTAMPFORMAT'='dd-MM-yyyy')""")
       checkAnswer(sql("select * from unsortedtable_offheap_unsafe where empno = 11"), sql("select * from origintable1 where empno = 11"))
       checkAnswer(sql("select * from unsortedtable_offheap_unsafe order by empno"), sql("select * from origintable1 order by empno"))
     } finally {
@@ -319,7 +319,7 @@ class TestSortColumns extends QueryTest with BeforeAndAfterAll {
       sql(s"""LOAD DATA local inpath '$resourcesPath/data.csv' INTO TABLE origintable1 OPTIONS('DELIMITER'= ',', 'QUOTECHAR'= '\"')""")
       setLoadingProperties("true", "false", "true")
       sql("CREATE TABLE unsortedtable_offheap_inmemory (empno int, empname String, designation String, doj Timestamp, workgroupcategory int, workgroupcategoryname String, deptno int, deptname String, projectcode int, projectjoindate Timestamp, projectenddate Timestamp,attendance int,utilization int,salary int) STORED BY 'org.apache.carbondata.format' tblproperties('sort_columns'='')")
-      sql(s"""LOAD DATA local inpath '$resourcesPath/data.csv' INTO TABLE unsortedtable_offheap_inmemory OPTIONS('DELIMITER'= ',', 'QUOTECHAR'= '\"')""")
+      sql(s"""LOAD DATA local inpath '$resourcesPath/data.csv' INTO TABLE unsortedtable_offheap_inmemory OPTIONS('DELIMITER'= ',', 'QUOTECHAR'= '\"', 'TIMESTAMPFORMAT'='dd-MM-yyyy')""")
       checkAnswer(sql("select * from unsortedtable_offheap_inmemory where empno = 11"), sql("select * from origintable1 where empno = 11"))
       checkAnswer(sql("select * from unsortedtable_offheap_inmemory where empno < 15 order by empno"), sql("select * from origintable1 where empno < 15 order by empno"))
       checkAnswer(sql("select * from unsortedtable_offheap_inmemory where empno <= 15 order by empno"), sql("select * from origintable1 where empno <= 15 order by empno"))

http://git-wip-us.apache.org/repos/asf/carbondata/blob/96ee82b3/integration/spark-common-test/src/test/scala/org/apache/carbondata/spark/testsuite/sortcolumns/TestSortColumnsWithUnsafe.scala
----------------------------------------------------------------------
diff --git a/integration/spark-common-test/src/test/scala/org/apache/carbondata/spark/testsuite/sortcolumns/TestSortColumnsWithUnsafe.scala b/integration/spark-common-test/src/test/scala/org/apache/carbondata/spark/testsuite/sortcolumns/TestSortColumnsWithUnsafe.scala
index 2cfa4c2..16e434b 100644
--- a/integration/spark-common-test/src/test/scala/org/apache/carbondata/spark/testsuite/sortcolumns/TestSortColumnsWithUnsafe.scala
+++ b/integration/spark-common-test/src/test/scala/org/apache/carbondata/spark/testsuite/sortcolumns/TestSortColumnsWithUnsafe.scala
@@ -27,16 +27,9 @@ class TestSortColumnsWithUnsafe extends QueryTest with BeforeAndAfterAll with Be
   override def beforeAll {
     CarbonProperties.getInstance()
       .addProperty(CarbonCommonConstants.ENABLE_UNSAFE_IN_QUERY_EXECUTION, "true")
-    CarbonProperties.getInstance().addProperty(
-      CarbonCommonConstants.CARBON_TIMESTAMP_FORMAT, "dd-MM-yyyy")
     dropTable
     sql("CREATE TABLE origintable1 (empno int, empname String, designation String, doj Timestamp, workgroupcategory int, workgroupcategoryname String, deptno int, deptname String, projectcode int, projectjoindate Timestamp, projectenddate Timestamp,attendance int,utilization int,salary int) STORED BY 'org.apache.carbondata.format'")
-    sql(s"""LOAD DATA local inpath '$resourcesPath/data.csv' INTO TABLE origintable1 OPTIONS('DELIMITER'= ',', 'QUOTECHAR'= '\"')""")
-  }
-
-  override def beforeEach(): Unit = {
-    CarbonProperties.getInstance().addProperty(
-      CarbonCommonConstants.CARBON_TIMESTAMP_FORMAT, "dd-MM-yyyy")
+    sql(s"""LOAD DATA local inpath '$resourcesPath/data.csv' INTO TABLE origintable1 OPTIONS('DELIMITER'= ',', 'QUOTECHAR'= '\"', 'TIMESTAMPFORMAT'='dd-MM-yyyy')""")
   }
 
   test("create table with no dictionary sort_columns") {
@@ -153,7 +146,7 @@ class TestSortColumnsWithUnsafe extends QueryTest with BeforeAndAfterAll with Be
 
   test("filter on sort_columns include no-dictionary, direct-dictionary and dictioanry") {
     sql("CREATE TABLE sorttable6 (empno int, empname String, designation String, doj Timestamp, workgroupcategory int, workgroupcategoryname String, deptno int, deptname String, projectcode int, projectjoindate Timestamp, projectenddate Timestamp,attendance int,utilization int,salary int) STORED BY 'org.apache.carbondata.format' tblproperties('sort_columns'='workgroupcategory, doj, empname', 'dictionary_include'='empname')")
-    sql(s"""LOAD DATA local inpath '$resourcesPath/data.csv' INTO TABLE sorttable6 OPTIONS('DELIMITER'= ',', 'QUOTECHAR'= '\"')""")
+    sql(s"""LOAD DATA local inpath '$resourcesPath/data.csv' INTO TABLE sorttable6 OPTIONS('DELIMITER'= ',', 'QUOTECHAR'= '\"', 'TIMESTAMPFORMAT'='dd-MM-yyyy')""")
     // no dictionary
     checkAnswer(sql("select * from sorttable6 where workgroupcategory = 1"), sql("select * from origintable1 where workgroupcategory = 1 order by doj"))
     // direct dictionary
@@ -166,7 +159,7 @@ class TestSortColumnsWithUnsafe extends QueryTest with BeforeAndAfterAll with Be
     try {
       setLoadingProperties("false", "false", "false")
       sql("CREATE TABLE unsortedtable_heap_safe (empno int, empname String, designation String, doj Timestamp, workgroupcategory int, workgroupcategoryname String, deptno int, deptname String, projectcode int, projectjoindate Timestamp, projectenddate Timestamp,attendance int,utilization int,salary int) STORED BY 'org.apache.carbondata.format' tblproperties('sort_columns'='')")
-      sql(s"""LOAD DATA local inpath '$resourcesPath/data.csv' INTO TABLE unsortedtable_heap_safe OPTIONS('DELIMITER'= ',', 'QUOTECHAR'= '\"')""")
+      sql(s"""LOAD DATA local inpath '$resourcesPath/data.csv' INTO TABLE unsortedtable_heap_safe OPTIONS('DELIMITER'= ',', 'QUOTECHAR'= '\"', 'TIMESTAMPFORMAT'='dd-MM-yyyy')""")
       checkAnswer(sql("select * from unsortedtable_heap_safe where empno = 11"), sql("select * from origintable1 where empno = 11"))
       checkAnswer(sql("select * from unsortedtable_heap_safe order by empno"), sql("select * from origintable1 order by empno"))
     } finally {
@@ -178,7 +171,7 @@ class TestSortColumnsWithUnsafe extends QueryTest with BeforeAndAfterAll with Be
     try {
       setLoadingProperties("false", "true", "false")
       sql("CREATE TABLE unsortedtable_heap_unsafe (empno int, empname String, designation String, doj Timestamp, workgroupcategory int, workgroupcategoryname String, deptno int, deptname String, projectcode int, projectjoindate Timestamp, projectenddate Timestamp,attendance int,utilization int,salary int) STORED BY 'org.apache.carbondata.format' tblproperties('sort_columns'='')")
-      sql(s"""LOAD DATA local inpath '$resourcesPath/data.csv' INTO TABLE unsortedtable_heap_unsafe OPTIONS('DELIMITER'= ',', 'QUOTECHAR'= '\"')""")
+      sql(s"""LOAD DATA local inpath '$resourcesPath/data.csv' INTO TABLE unsortedtable_heap_unsafe OPTIONS('DELIMITER'= ',', 'QUOTECHAR'= '\"', 'TIMESTAMPFORMAT'='dd-MM-yyyy')""")
       checkAnswer(sql("select * from unsortedtable_heap_unsafe where empno = 11"), sql("select * from origintable1 where empno = 11"))
       checkAnswer(sql("select * from unsortedtable_heap_unsafe order by empno"), sql("select * from origintable1 order by empno"))
     } finally {
@@ -190,7 +183,7 @@ class TestSortColumnsWithUnsafe extends QueryTest with BeforeAndAfterAll with Be
     try {
       setLoadingProperties("false", "false", "true")
       sql("CREATE TABLE unsortedtable_heap_inmemory (empno int, empname String, designation String, doj Timestamp, workgroupcategory int, workgroupcategoryname String, deptno int, deptname String, projectcode int, projectjoindate Timestamp, projectenddate Timestamp,attendance int,utilization int,salary int) STORED BY 'org.apache.carbondata.format' tblproperties('sort_columns'='')")
-      sql(s"""LOAD DATA local inpath '$resourcesPath/data.csv' INTO TABLE unsortedtable_heap_inmemory OPTIONS('DELIMITER'= ',', 'QUOTECHAR'= '\"')""")
+      sql(s"""LOAD DATA local inpath '$resourcesPath/data.csv' INTO TABLE unsortedtable_heap_inmemory OPTIONS('DELIMITER'= ',', 'QUOTECHAR'= '\"', 'TIMESTAMPFORMAT'='dd-MM-yyyy')""")
       checkAnswer(sql("select * from unsortedtable_heap_inmemory where empno = 11"), sql("select * from origintable1 where empno = 11"))
       checkAnswer(sql("select * from unsortedtable_heap_inmemory order by empno"), sql("select * from origintable1 order by empno"))
     } finally {
@@ -202,7 +195,7 @@ class TestSortColumnsWithUnsafe extends QueryTest with BeforeAndAfterAll with Be
     try {
       setLoadingProperties("true", "false", "false")
       sql("CREATE TABLE unsortedtable_offheap_safe (empno int, empname String, designation String, doj Timestamp, workgroupcategory int, workgroupcategoryname String, deptno int, deptname String, projectcode int, projectjoindate Timestamp, projectenddate Timestamp,attendance int,utilization int,salary int) STORED BY 'org.apache.carbondata.format' tblproperties('sort_columns'='')")
-      sql(s"""LOAD DATA local inpath '$resourcesPath/data.csv' INTO TABLE unsortedtable_offheap_safe OPTIONS('DELIMITER'= ',', 'QUOTECHAR'= '\"')""")
+      sql(s"""LOAD DATA local inpath '$resourcesPath/data.csv' INTO TABLE unsortedtable_offheap_safe OPTIONS('DELIMITER'= ',', 'QUOTECHAR'= '\"', 'TIMESTAMPFORMAT'='dd-MM-yyyy')""")
       checkAnswer(sql("select * from unsortedtable_offheap_safe where empno = 11"), sql("select * from origintable1 where empno = 11"))
       checkAnswer(sql("select * from unsortedtable_offheap_safe order by empno"), sql("select * from origintable1 order by empno"))
     } finally {
@@ -214,7 +207,7 @@ class TestSortColumnsWithUnsafe extends QueryTest with BeforeAndAfterAll with Be
     try {
       setLoadingProperties("true", "true", "false")
       sql("CREATE TABLE unsortedtable_offheap_unsafe (empno int, empname String, designation String, doj Timestamp, workgroupcategory int, workgroupcategoryname String, deptno int, deptname String, projectcode int, projectjoindate Timestamp, projectenddate Timestamp,attendance int,utilization int,salary int) STORED BY 'org.apache.carbondata.format' tblproperties('sort_columns'='')")
-      sql(s"""LOAD DATA local inpath '$resourcesPath/data.csv' INTO TABLE unsortedtable_offheap_unsafe OPTIONS('DELIMITER'= ',', 'QUOTECHAR'= '\"')""")
+      sql(s"""LOAD DATA local inpath '$resourcesPath/data.csv' INTO TABLE unsortedtable_offheap_unsafe OPTIONS('DELIMITER'= ',', 'QUOTECHAR'= '\"', 'TIMESTAMPFORMAT'='dd-MM-yyyy')""")
       checkAnswer(sql("select * from unsortedtable_offheap_unsafe where empno = 11"), sql("select * from origintable1 where empno = 11"))
       checkAnswer(sql("select * from unsortedtable_offheap_unsafe order by empno"), sql("select * from origintable1 order by empno"))
     } finally {
@@ -226,7 +219,7 @@ class TestSortColumnsWithUnsafe extends QueryTest with BeforeAndAfterAll with Be
     try {
       setLoadingProperties("true", "false", "true")
       sql("CREATE TABLE unsortedtable_offheap_inmemory (empno int, empname String, designation String, doj Timestamp, workgroupcategory int, workgroupcategoryname String, deptno int, deptname String, projectcode int, projectjoindate Timestamp, projectenddate Timestamp,attendance int,utilization int,salary int) STORED BY 'org.apache.carbondata.format' tblproperties('sort_columns'='')")
-      sql(s"""LOAD DATA local inpath '$resourcesPath/data.csv' INTO TABLE unsortedtable_offheap_inmemory OPTIONS('DELIMITER'= ',', 'QUOTECHAR'= '\"')""")
+      sql(s"""LOAD DATA local inpath '$resourcesPath/data.csv' INTO TABLE unsortedtable_offheap_inmemory OPTIONS('DELIMITER'= ',', 'QUOTECHAR'= '\"', 'TIMESTAMPFORMAT'='dd-MM-yyyy')""")
       checkAnswer(sql("select * from unsortedtable_offheap_inmemory where empno = 11"), sql("select * from origintable1 where empno = 11"))
       checkAnswer(sql("select * from unsortedtable_offheap_inmemory where empno < 15 order by empno"), sql("select * from origintable1 where empno < 15 order by empno"))
       checkAnswer(sql("select * from unsortedtable_offheap_inmemory where empno <= 15 order by empno"), sql("select * from origintable1 where empno <= 15 order by empno"))
@@ -257,8 +250,6 @@ class TestSortColumnsWithUnsafe extends QueryTest with BeforeAndAfterAll with Be
     CarbonProperties.getInstance()
       .addProperty(CarbonCommonConstants.ENABLE_UNSAFE_IN_QUERY_EXECUTION,
         CarbonCommonConstants.ENABLE_UNSAFE_IN_QUERY_EXECUTION_DEFAULTVALUE)
-    CarbonProperties.getInstance().addProperty(
-      CarbonCommonConstants.CARBON_TIMESTAMP_FORMAT, CarbonCommonConstants.CARBON_TIMESTAMP_DEFAULT_FORMAT)
 
     dropTable
   }


[15/20] carbondata git commit: [CARBONDATA-2206] support lucene index datamap

Posted by ja...@apache.org.
http://git-wip-us.apache.org/repos/asf/carbondata/blob/c0133aac/processing/src/main/java/org/apache/carbondata/processing/store/writer/v3/CarbonFactDataWriterImplV3.java
----------------------------------------------------------------------
diff --git a/processing/src/main/java/org/apache/carbondata/processing/store/writer/v3/CarbonFactDataWriterImplV3.java b/processing/src/main/java/org/apache/carbondata/processing/store/writer/v3/CarbonFactDataWriterImplV3.java
index 80d8154..1c9ccc8 100644
--- a/processing/src/main/java/org/apache/carbondata/processing/store/writer/v3/CarbonFactDataWriterImplV3.java
+++ b/processing/src/main/java/org/apache/carbondata/processing/store/writer/v3/CarbonFactDataWriterImplV3.java
@@ -110,7 +110,7 @@ public class CarbonFactDataWriterImplV3 extends AbstractFactDataWriter {
    * @param tablePage
    */
   @Override public void writeTablePage(TablePage tablePage)
-      throws CarbonDataWriterException {
+      throws CarbonDataWriterException,IOException {
     // condition for writting all the pages
     if (!tablePage.isLastPage()) {
       boolean isAdded = false;
@@ -148,7 +148,7 @@ public class CarbonFactDataWriterImplV3 extends AbstractFactDataWriter {
     }
   }
 
-  private void addPageData(TablePage tablePage) {
+  private void addPageData(TablePage tablePage) throws IOException {
     blockletDataHolder.addPage(tablePage);
     if (listener != null) {
       if (pageId == 0) {


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

Posted by ja...@apache.org.
http://git-wip-us.apache.org/repos/asf/carbondata/blob/f8ded96e/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..4c0e637
--- /dev/null
+++ b/integration/spark2/src/main/scala/org/apache/spark/sql/execution/command/preaaggregate/PreAggregateTableHelper.scala
@@ -0,0 +1,195 @@
+/*
+ * 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.optimizer.CarbonFilters
+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
+import org.apache.carbondata.core.metadata.schema.table.CarbonTable
+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: Option[String] = None,
+    ifNotExistsSet: Boolean = false) {
+
+  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.asScala.getOrElse("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 = ifNotExistsSet,
+      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.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), isVisible = false).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,
+      dataMapClassName,
+      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) {
+      PreAggregateUtil.createTimeSeriesSelectQueryFromMain(
+        childSchema.getChildSchema,
+        parentTable.getTableName,
+        parentTable.getDatabaseName)
+    } else {
+      queryString
+    }
+    val dataFrame = sparkSession.sql(new CarbonSpark2SqlParser().addPreAggLoadFunction(
+      updatedLoadQuery)).drop("preAggLoad")
+    loadCommand = PreAggregateUtil.createLoadCommandForChild(
+      childSchema.getChildSchema.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) {
+      // 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(
+        TableIdentifier(parentTable.getTableName, Some(parentTable.getDatabaseName)),
+        segmentToLoad = "*",
+        validateSegments = true,
+        loadCommand,
+        isOverwrite = false,
+        sparkSession)
+    }
+    Seq.empty
+  }
+}
+
+

http://git-wip-us.apache.org/repos/asf/carbondata/blob/f8ded96e/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 dd42b50..b52d0e7 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
 
@@ -407,11 +407,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)
@@ -424,7 +424,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,
@@ -436,12 +436,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)
-      LOGGER.info(s"Parent table updated is successful for table" +
-                  s" $dbName.${childSchema.getRelationIdentifier.toString}")
+      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")
@@ -450,7 +449,6 @@ object PreAggregateUtil {
       // release lock after command execution completion
       releaseLocks(locks)
     }
-    Seq.empty
   }
 
   /**
@@ -526,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/f8ded96e/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 064ba18..4962c3a 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/f8ded96e/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 4ef0df9..c8f7ac7 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(
       }
       // read the latest schema file
       val tableInfo: org.apache.carbondata.format.TableInfo =
-        metastore.getThriftTableInfo(carbonTable)(sparkSession)
+        metastore.getThriftTableInfo(carbonTable)
       // maintain the added column for schema evolution history
       var addColumnSchema: org.apache.carbondata.format.ColumnSchema = null
       var deletedColumnSchema: org.apache.carbondata.format.ColumnSchema = null

http://git-wip-us.apache.org/repos/asf/carbondata/blob/f8ded96e/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 7bbefd7..a64bdb9 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
@@ -100,7 +100,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/f8ded96e/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 2503fc3..05c0059 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/f8ded96e/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 2b35416..6fd0820 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.trim.equalsIgnoreCase(defaultValue)) {
-        return (granularity.toString.toLowerCase, dmProperties.get(granularity.getName).get)
+      if (dmProperties.containsKey(granularity.getName) &&
+        dmProperties.get(granularity.getName).trim.equalsIgnoreCase(defaultValue)) {
+        return (granularity.toString.toLowerCase, dmProperties.get(granularity.getName))
       }
     }
 
@@ -194,8 +194,8 @@ object TimeSeriesUtil {
    * @param timeSeriesColumn
    *                         timeseries column name
    */
-  def updateTimeColumnSelect(fieldMapping: scala.collection.mutable
-  .LinkedHashMap[Field, DataMapField],
+  def updateTimeColumnSelect(
+      fieldMapping: scala.collection.mutable.LinkedHashMap[Field, DataMapField],
       timeSeriesColumn: String,
       timeSeriesFunction: String) : Any = {
     val isTimeSeriesColumnExits = fieldMapping

http://git-wip-us.apache.org/repos/asf/carbondata/blob/f8ded96e/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/f8ded96e/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 5e242b7..96ef473 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/f8ded96e/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 fc5bdac..3f1ebbc 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,8 +143,9 @@ trait CarbonMetaStore {
 
   def listAllTables(sparkSession: SparkSession): Seq[CarbonTable]
 
-  def getThriftTableInfo(carbonTable: CarbonTable)(sparkSession: SparkSession):
-  org.apache.carbondata.format.TableInfo
+  def getThriftTableInfo(
+      carbonTable: CarbonTable
+  ): org.apache.carbondata.format.TableInfo
 
   def getTableFromMetadataCache(database: String, tableName: String): Option[CarbonTable]
 

http://git-wip-us.apache.org/repos/asf/carbondata/blob/f8ded96e/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/f8ded96e/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 f514074..be8d4d7 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
@@ -31,13 +31,13 @@ import org.apache.carbondata.core.datamap.DataMapStoreManager;
 import org.apache.carbondata.core.datamap.Segment;
 import org.apache.carbondata.core.datamap.TableDataMap;
 import org.apache.carbondata.core.datamap.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 {
 
@@ -55,7 +55,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);
       }
     }
@@ -64,7 +64,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();


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

Posted by ja...@apache.org.
http://git-wip-us.apache.org/repos/asf/carbondata/blob/f8ded96e/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..795ef6a
--- /dev/null
+++ b/integration/spark-common-test/src/test/scala/org/apache/carbondata/spark/testsuite/datamap/IndexDataMapWriterSuite.scala
@@ -0,0 +1,217 @@
+/*
+ * 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.{DataMapDistributable, DataMapMeta, DataMapStoreManager, Segment}
+import org.apache.carbondata.core.datamap.dev.{AbstractDataMapWriter}
+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.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(segment: Segment): Unit = {}
+
+  override def clear(): Unit = {}
+
+  override def getDataMaps(distributable: DataMapDistributable): util.List[AbstractCoarseGrainIndexDataMap] = ???
+
+  override def getDataMaps(segment: Segment): util.List[AbstractCoarseGrainIndexDataMap] = ???
+
+  override def createWriter(segment: Segment, dataWritePath: String): AbstractDataMapWriter =
+    IndexDataMapWriterSuite.dataMapWriterC2Mock(identifier, segment, 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: Segment): 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, segment: Segment,
+      dataWritePath: String) =
+    new AbstractDataMapWriter(identifier, segment, 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/f8ded96e/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 717af6f..0000000
--- a/integration/spark-common-test/src/test/scala/org/apache/carbondata/spark/testsuite/datamap/TestDataMapCommand.scala
+++ /dev/null
@@ -1,288 +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 datamap datamap_hiveMetaStoreTable_1 on table hiveMetaStoreTable_1")
-      checkExistence(sql("show datamap on table hiveMetaStoreTable_1"),
-        false,
-        "datamap_hiveMetaStoreTable_1")
-      assert(sql("show datamap on table hiveMetaStoreTable_1").collect().length == 0)
-      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/f8ded96e/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..a05a8c2
--- /dev/null
+++ b/integration/spark-common-test/src/test/scala/org/apache/carbondata/spark/testsuite/datamap/TestIndexDataMapCommand.scala
@@ -0,0 +1,285 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.carbondata.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 datamap datamap_hiveMetaStoreTable_1 on table hiveMetaStoreTable_1")
+      checkExistence(sql("show datamap on table hiveMetaStoreTable_1"),
+        false,
+        "datamap_hiveMetaStoreTable_1")
+      assert(sql("show datamap on table hiveMetaStoreTable_1").collect().length == 0)
+      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/f8ded96e/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/f8ded96e/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/f8ded96e/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/f8ded96e/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/f8ded96e/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..dc53900
--- /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, false);
+    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/f8ded96e/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..a66f26a
--- /dev/null
+++ b/integration/spark2/src/main/java/org/apache/carbondata/datamap/TimeseriesDataMapProvider.java
@@ -0,0 +1,50 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.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.Some;
+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, new Some(details._1()), false);
+    helper.initMeta(sparkSession);
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/carbondata/blob/f8ded96e/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 39e73ee..9315208 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
@@ -94,7 +94,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, carbonSessionInfo, storePath)

http://git-wip-us.apache.org/repos/asf/carbondata/blob/f8ded96e/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 c6d86b3..8532e9d 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
@@ -19,20 +19,13 @@ package org.apache.spark.sql.execution.command.datamap
 import scala.collection.JavaConverters._
 
 import org.apache.spark.sql._
-import org.apache.spark.sql.catalyst.analysis.TableAlreadyExistsException
 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.metadata.schema.datamap.DataMapProvider
-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
@@ -47,59 +40,25 @@ case class CarbonCreateDataMapCommand(
     ifNotExistsSet: Boolean = false)
   extends AtomicRunnableCommand {
 
-  var createPreAggregateTableCommands: CreatePreAggregateTableCommand = _
-  var tableIsExists: Boolean = false
+  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)
 
-    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,
-          DataMapProvider.TIMESERIES,
-          updatedDmProperties,
-          queryString.get,
-          Some(details._1))
-      } else {
-        CreatePreAggregateTableCommand(
-          dataMapName,
-          tableIdentifier,
-          DataMapProvider.PREAGGREGATE,
-          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)
-    }
+    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)
+
     val LOGGER = LogServiceFactory.getLogService(this.getClass.getCanonicalName)
     LOGGER.audit(s"DataMap $dataMapName successfully added to Table ${tableIdentifier.table}")
     Seq.empty
@@ -115,29 +74,20 @@ case class CarbonCreateDataMapCommand(
   }
 
   override def processData(sparkSession: SparkSession): Seq[Row] = {
-    if (dmClassName.equalsIgnoreCase(PREAGGREGATE.toString) ||
-      dmClassName.equalsIgnoreCase(TIMESERIES.toString)) {
-      if (!tableIsExists) {
-        createPreAggregateTableCommands.processData(sparkSession)
-      } else {
-        Seq.empty
+    if (dataMapProvider != null) {
+      dataMapProvider.initData(mainTable, sparkSession)
+      if (mainTable.isAutoRefreshDataMap) {
+        dataMapProvider.rebuild(mainTable, sparkSession)
       }
-    } else {
-      Seq.empty
     }
+    Seq.empty
   }
 
   override def undoMetadata(sparkSession: SparkSession, exception: Exception): Seq[Row] = {
-    if (dmClassName.equalsIgnoreCase(PREAGGREGATE.toString) ||
-      dmClassName.equalsIgnoreCase(TIMESERIES.toString)) {
-      if (!tableIsExists && createPreAggregateTableCommands != null) {
-        createPreAggregateTableCommands.undoMetadata(sparkSession, exception)
-      } else {
-        Seq.empty
-      }
-    } else {
-      throw new MalformedDataMapCommandException("Unknown datamap provider/class " + dmClassName)
+    if (dataMapProvider != null) {
+      dataMapProvider.freeMeta(mainTable, dataMapSchema, sparkSession)
     }
+    Seq.empty
   }
 }
 

http://git-wip-us.apache.org/repos/asf/carbondata/blob/f8ded96e/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 e89d15a..f773a55 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._
 
 /**
@@ -52,8 +51,10 @@ case class CarbonDropDataMapCommand(
     forceDrop: Boolean = false)
   extends AtomicRunnableCommand {
 
-  val LOGGER: LogService = LogServiceFactory.getLogService(this.getClass.getCanonicalName)
-  var commandToRun: CarbonDropTableCommand = _
+  private val LOGGER: LogService = LogServiceFactory.getLogService(this.getClass.getCanonicalName)
+  private var dataMapProvider: DataMapProvider = _
+  private var mainTable: CarbonTable = _
+  private var dataMapSchema: DataMapSchema = _
 
   override def processMetadata(sparkSession: SparkSession): Seq[Row] = {
     val dbName = CarbonEnv.getDatabaseName(databaseNameOp)(sparkSession)
@@ -66,7 +67,6 @@ case class CarbonDropDataMapCommand(
     catalog.checkSchemasModifiedTimeAndReloadTable(TableIdentifier(tableName, Some(dbName)))
     val carbonLocks: scala.collection.mutable.ListBuffer[ICarbonLock] = ListBuffer()
     try {
-      forceDropTableFromMetaStore(sparkSession)
       locksToBeAcquired foreach {
         lock => carbonLocks += CarbonLockUtil.getLockObject(tableIdentifier, lock)
       }
@@ -82,35 +82,33 @@ case class CarbonDropDataMapCommand(
       // If force drop is true then remove the datamap from hivemetastore. No need to remove from
       // parent as the first condition would have taken care of it.
       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)
@@ -118,7 +116,7 @@ case class CarbonDropDataMapCommand(
           throw new NoSuchDataMapException(dataMapName, tableName)
         }
       } else if (carbonTable.isDefined &&
-        carbonTable.get.getTableInfo.getDataMapSchemaList.size() == 0) {
+                 carbonTable.get.getTableInfo.getDataMapSchemaList.size() == 0) {
         if (!ifExistsSet) {
           throw new NoSuchDataMapException(dataMapName, tableName)
         }
@@ -142,39 +140,10 @@ case class CarbonDropDataMapCommand(
     Seq.empty
   }
 
-  /**
-   * Used to drop child datamap from hive metastore if it exists.
-   * forceDrop will be true only when an exception occurs in main table status updation for
-   * parent table or in processData from CreatePreAggregateTableCommand.
-   */
-  private def forceDropTableFromMetaStore(sparkSession: SparkSession): Unit = {
-    if (forceDrop) {
-      val childTableName = tableName + "_" + dataMapName
-      LOGGER.info(s"Trying to force drop $childTableName from metastore")
-      val childCarbonTable: Option[CarbonTable] = try {
-        Some(CarbonEnv.getCarbonTable(databaseNameOp, childTableName)(sparkSession))
-      } catch {
-        case _: Exception =>
-          LOGGER.warn(s"Child table $childTableName not found in metastore")
-          None
-      }
-      if (childCarbonTable.isDefined) {
-        commandToRun = CarbonDropTableCommand(
-          ifExistsSet = true,
-          Some(childCarbonTable.get.getDatabaseName),
-          childCarbonTable.get.getTableName,
-          dropChildTable = true)
-        commandToRun.processMetadata(sparkSession)
-      }
-    }
-  }
-
   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/f8ded96e/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 c1f86ef..705455f 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
@@ -93,7 +93,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/f8ded96e/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 efd1216..99691d2 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
@@ -88,7 +88,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/f8ded96e/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 c02ac4f..0000000
--- a/integration/spark2/src/main/scala/org/apache/spark/sql/execution/command/preaaggregate/CreatePreAggregateTableCommand.scala
+++ /dev/null
@@ -1,203 +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.optimizer.CarbonFilters
-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
-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,
-    dataMapProvider: DataMapProvider,
-    dmProperties: Map[String, String],
-    queryString: String,
-    timeSeriesFunction: Option[String] = None,
-    ifNotExistsSet: Boolean = false)
-  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.asScala.getOrElse("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 = ifNotExistsSet,
-      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), isVisible = false).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,
-      dataMapProvider.getClassName,
-      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) {
-      PreAggregateUtil.createTimeSeriesSelectQueryFromMain(childSchema.getChildSchema,
-        parentTable.getTableName,
-        parentTable.getDatabaseName)
-    } else {
-      queryString
-    }
-    val dataFrame = sparkSession.sql(new CarbonSpark2SqlParser().addPreAggLoadFunction(
-      updatedLoadQuery)).drop("preAggLoad")
-    loadCommand = PreAggregateUtil.createLoadCommandForChild(
-      childSchema.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,
-      forceDrop = true).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) {
-      // 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(
-        TableIdentifier(parentTable.getTableName, Some(parentTable.getDatabaseName)),
-        segmentToLoad = "*",
-        validateSegments = true,
-        loadCommand,
-        isOverwrite = false,
-        sparkSession)
-    }
-    Seq.empty
-  }
-}
-
-


[19/20] carbondata git commit: [CARBONDATA-2216][Test] Fix bugs in sdv tests

Posted by ja...@apache.org.
[CARBONDATA-2216][Test] Fix bugs in sdv tests

This closes #2012


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

Branch: refs/heads/datamap-rebase1
Commit: 62e33e5ff73914ce67dfe2d3b8bff906aa8df15e
Parents: 65cb0d6
Author: xuchuanyin <xu...@hust.edu.cn>
Authored: Wed Feb 28 16:02:55 2018 +0800
Committer: Jacky Li <ja...@qq.com>
Committed: Sun Mar 4 22:52:56 2018 +0800

----------------------------------------------------------------------
 .../cluster/sdv/generated/MergeIndexTestCase.scala   | 15 +++++++++++++--
 1 file changed, 13 insertions(+), 2 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/carbondata/blob/62e33e5f/integration/spark-common-cluster-test/src/test/scala/org/apache/carbondata/cluster/sdv/generated/MergeIndexTestCase.scala
----------------------------------------------------------------------
diff --git a/integration/spark-common-cluster-test/src/test/scala/org/apache/carbondata/cluster/sdv/generated/MergeIndexTestCase.scala b/integration/spark-common-cluster-test/src/test/scala/org/apache/carbondata/cluster/sdv/generated/MergeIndexTestCase.scala
index c0abe4e..b886b11 100644
--- a/integration/spark-common-cluster-test/src/test/scala/org/apache/carbondata/cluster/sdv/generated/MergeIndexTestCase.scala
+++ b/integration/spark-common-cluster-test/src/test/scala/org/apache/carbondata/cluster/sdv/generated/MergeIndexTestCase.scala
@@ -92,18 +92,29 @@ class MergeIndexTestCase extends QueryTest with BeforeAndAfterAll {
     val table = CarbonMetadata.getInstance().getCarbonTable("default","carbon_automation_nonmerge")
     new CarbonIndexFileMergeWriter().mergeCarbonIndexFilesOfSegment("0.1", table.getTablePath, false)
     assert(getIndexFileCount("default", "carbon_automation_nonmerge", "0.1") == 0)
+    assert(getMergedIndexFileCount("default", "carbon_automation_nonmerge", "0.1") == 1)
     checkAnswer(sql("""Select count(*) from carbon_automation_nonmerge"""), rows)
   }
 
   private def getIndexFileCount(dbName: String, tableName: String, segment: String): Int = {
+    getFileCount(dbName, tableName, segment, CarbonTablePath.INDEX_FILE_EXT)
+  }
+
+  private def getMergedIndexFileCount(dbName: String, tableName: String, segment: String): Int = {
+    getFileCount(dbName, tableName, segment, CarbonTablePath.MERGE_INDEX_FILE_EXT)
+  }
+
+  private def getFileCount(dbName: String,
+      tableName: String,
+      segment: String,
+      suffix: String): Int = {
     val carbonTable = CarbonMetadata.getInstance().getCarbonTable(dbName, tableName)
     val identifier = carbonTable.getAbsoluteTableIdentifier
     val path = CarbonTablePath
       .getSegmentPath(identifier.getTablePath, segment)
     val carbonFiles = FileFactory.getCarbonFile(path).listFiles(new CarbonFileFilter {
       override def accept(file: CarbonFile): Boolean = {
-        file.getName.endsWith(CarbonTablePath
-          .INDEX_FILE_EXT)
+        file.getName.endsWith(suffix)
       }
     })
     if (carbonFiles != null) {


[17/20] carbondata git commit: [CARBONDATA-2206] Fixed lucene datamap evaluation issue in executor

Posted by ja...@apache.org.
[CARBONDATA-2206] Fixed lucene datamap evaluation issue in executor

In case of MatchExpression it should return same bitset from RowLevelFilterExecuterImpl

This closes #2010


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

Branch: refs/heads/datamap-rebase1
Commit: 8049185cbdb001588be5c5d97f73bac035d16d6c
Parents: c0133aa
Author: ravipesala <ra...@gmail.com>
Authored: Tue Feb 27 19:14:15 2018 +0530
Committer: Jacky Li <ja...@qq.com>
Committed: Sun Mar 4 22:45:17 2018 +0800

----------------------------------------------------------------------
 .../carbondata/core/datamap/DataMapChooser.java |  3 +-
 .../core/scan/expression/MatchExpression.java   | 57 ++++++++++++++++++++
 .../executer/RowLevelFilterExecuterImpl.java    |  4 ++
 .../lucene/LuceneFineGrainDataMapSuite.scala    |  8 ++-
 .../datamap/expression/MatchExpression.java     | 56 -------------------
 .../spark/sql/optimizer/CarbonFilters.scala     |  3 +-
 6 files changed, 67 insertions(+), 64 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/carbondata/blob/8049185c/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 c8c971d..f9214a8 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
@@ -228,7 +228,8 @@ public class DataMapChooser {
 
   private boolean contains(DataMapMeta mapMeta, List<ColumnExpression> columnExpressions,
       Set<ExpressionType> expressionTypes) {
-    if (mapMeta.getOptimizedOperation().contains(ExpressionType.TEXT_MATCH)) {
+    if (mapMeta.getOptimizedOperation().contains(ExpressionType.TEXT_MATCH) &&
+        expressionTypes.contains(ExpressionType.TEXT_MATCH)) {
       // TODO: fix it with right logic
       return true;
     }

http://git-wip-us.apache.org/repos/asf/carbondata/blob/8049185c/core/src/main/java/org/apache/carbondata/core/scan/expression/MatchExpression.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/carbondata/core/scan/expression/MatchExpression.java b/core/src/main/java/org/apache/carbondata/core/scan/expression/MatchExpression.java
new file mode 100644
index 0000000..3677b51
--- /dev/null
+++ b/core/src/main/java/org/apache/carbondata/core/scan/expression/MatchExpression.java
@@ -0,0 +1,57 @@
+/*
+ * 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.scan.expression;
+
+import org.apache.carbondata.common.annotations.InterfaceAudience;
+import org.apache.carbondata.core.metadata.datatype.DataTypes;
+import org.apache.carbondata.core.scan.expression.Expression;
+import org.apache.carbondata.core.scan.expression.ExpressionResult;
+import org.apache.carbondata.core.scan.expression.exception.FilterIllegalMemberException;
+import org.apache.carbondata.core.scan.expression.exception.FilterUnsupportedException;
+import org.apache.carbondata.core.scan.filter.intf.ExpressionType;
+import org.apache.carbondata.core.scan.filter.intf.RowIntf;
+
+@InterfaceAudience.Internal
+public class MatchExpression extends Expression {
+  private String queryString;
+
+  public MatchExpression(String queryString) {
+    this.queryString = queryString;
+  }
+
+  @Override
+  public ExpressionResult evaluate(RowIntf value)
+      throws FilterUnsupportedException, FilterIllegalMemberException {
+    return new ExpressionResult(DataTypes.BOOLEAN,true);
+  }
+
+  @Override
+  public ExpressionType getFilterExpressionType() {
+    return ExpressionType.TEXT_MATCH;
+  }
+
+  @Override
+  public void findAndSetChild(Expression oldExpr, Expression newExpr) {
+
+  }
+
+  @Override
+  public String getString() {
+    return queryString;
+  }
+}

http://git-wip-us.apache.org/repos/asf/carbondata/blob/8049185c/core/src/main/java/org/apache/carbondata/core/scan/filter/executer/RowLevelFilterExecuterImpl.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/carbondata/core/scan/filter/executer/RowLevelFilterExecuterImpl.java b/core/src/main/java/org/apache/carbondata/core/scan/filter/executer/RowLevelFilterExecuterImpl.java
index 540607d..2bd49ed 100644
--- a/core/src/main/java/org/apache/carbondata/core/scan/filter/executer/RowLevelFilterExecuterImpl.java
+++ b/core/src/main/java/org/apache/carbondata/core/scan/filter/executer/RowLevelFilterExecuterImpl.java
@@ -50,6 +50,7 @@ import org.apache.carbondata.core.scan.executor.infos.KeyStructureInfo;
 import org.apache.carbondata.core.scan.executor.util.QueryUtil;
 import org.apache.carbondata.core.scan.executor.util.RestructureUtil;
 import org.apache.carbondata.core.scan.expression.Expression;
+import org.apache.carbondata.core.scan.expression.MatchExpression;
 import org.apache.carbondata.core.scan.expression.exception.FilterIllegalMemberException;
 import org.apache.carbondata.core.scan.expression.exception.FilterUnsupportedException;
 import org.apache.carbondata.core.scan.filter.FilterUtil;
@@ -191,6 +192,9 @@ public class RowLevelFilterExecuterImpl implements FilterExecuter {
   @Override
   public BitSetGroup applyFilter(RawBlockletColumnChunks rawBlockletColumnChunks,
       boolean useBitsetPipeLine) throws FilterUnsupportedException, IOException {
+    if (exp instanceof MatchExpression) {
+      return rawBlockletColumnChunks.getBitSetGroup();
+    }
     readColumnChunks(rawBlockletColumnChunks);
     // CHECKSTYLE:ON
 

http://git-wip-us.apache.org/repos/asf/carbondata/blob/8049185c/datamap/lucene/src/test/scala/org/apache/carbondata/datamap/lucene/LuceneFineGrainDataMapSuite.scala
----------------------------------------------------------------------
diff --git a/datamap/lucene/src/test/scala/org/apache/carbondata/datamap/lucene/LuceneFineGrainDataMapSuite.scala b/datamap/lucene/src/test/scala/org/apache/carbondata/datamap/lucene/LuceneFineGrainDataMapSuite.scala
index 4766281..5e28e8a 100644
--- a/datamap/lucene/src/test/scala/org/apache/carbondata/datamap/lucene/LuceneFineGrainDataMapSuite.scala
+++ b/datamap/lucene/src/test/scala/org/apache/carbondata/datamap/lucene/LuceneFineGrainDataMapSuite.scala
@@ -61,12 +61,10 @@ class LuceneFineGrainDataMapSuite extends QueryTest with BeforeAndAfterAll {
 
     sql(s"LOAD DATA LOCAL INPATH '$file2' INTO TABLE datamap_test OPTIONS('header'='false')")
 
-    sql("SELECT * FROM datamap_test ORDER BY id").show
-
     //    sql("select * from normal_test where name='n34000'").show
-    sql("SELECT * FROM datamap_test WHERE TEXT_MATCH('name:n10')").show
-    sql("SELECT * FROM datamap_test WHERE TEXT_MATCH('name:n10*')").show
-    sql("SELECT * FROM datamap_test WHERE TEXT_MATCH('city:c020')").show
+    checkAnswer(sql("SELECT * FROM datamap_test WHERE TEXT_MATCH('name:n10')"), sql(s"select * from datamap_test where name='n10'"))
+//    checkAnswer(sql("SELECT * FROM datamap_test WHERE TEXT_MATCH('name:n10*')"), sql(s"SELECT * FROM datamap_test WHERE name like 'n10%'"))
+    checkAnswer(sql("SELECT * FROM datamap_test WHERE TEXT_MATCH('city:c020')"), sql(s"SELECT * FROM datamap_test WHERE city='c020'"))
 
     //    checkAnswer(
     //      sql("select * from datamap_test where match('name:n34000')"),

http://git-wip-us.apache.org/repos/asf/carbondata/blob/8049185c/integration/spark2/src/main/java/org/apache/carbondata/datamap/expression/MatchExpression.java
----------------------------------------------------------------------
diff --git a/integration/spark2/src/main/java/org/apache/carbondata/datamap/expression/MatchExpression.java b/integration/spark2/src/main/java/org/apache/carbondata/datamap/expression/MatchExpression.java
deleted file mode 100644
index fceb729..0000000
--- a/integration/spark2/src/main/java/org/apache/carbondata/datamap/expression/MatchExpression.java
+++ /dev/null
@@ -1,56 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *    http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.carbondata.datamap.expression;
-
-import org.apache.carbondata.common.annotations.InterfaceAudience;
-import org.apache.carbondata.core.scan.expression.Expression;
-import org.apache.carbondata.core.scan.expression.ExpressionResult;
-import org.apache.carbondata.core.scan.expression.exception.FilterIllegalMemberException;
-import org.apache.carbondata.core.scan.expression.exception.FilterUnsupportedException;
-import org.apache.carbondata.core.scan.filter.intf.ExpressionType;
-import org.apache.carbondata.core.scan.filter.intf.RowIntf;
-
-@InterfaceAudience.Internal
-public class MatchExpression extends Expression {
-  private String queryString;
-
-  public MatchExpression(String queryString) {
-    this.queryString = queryString;
-  }
-
-  @Override
-  public ExpressionResult evaluate(RowIntf value)
-      throws FilterUnsupportedException, FilterIllegalMemberException {
-    return null;
-  }
-
-  @Override
-  public ExpressionType getFilterExpressionType() {
-    return ExpressionType.TEXT_MATCH;
-  }
-
-  @Override
-  public void findAndSetChild(Expression oldExpr, Expression newExpr) {
-
-  }
-
-  @Override
-  public String getString() {
-    return queryString;
-  }
-}

http://git-wip-us.apache.org/repos/asf/carbondata/blob/8049185c/integration/spark2/src/main/scala/org/apache/spark/sql/optimizer/CarbonFilters.scala
----------------------------------------------------------------------
diff --git a/integration/spark2/src/main/scala/org/apache/spark/sql/optimizer/CarbonFilters.scala b/integration/spark2/src/main/scala/org/apache/spark/sql/optimizer/CarbonFilters.scala
index 0aedd40..2213fdb 100644
--- a/integration/spark2/src/main/scala/org/apache/spark/sql/optimizer/CarbonFilters.scala
+++ b/integration/spark2/src/main/scala/org/apache/spark/sql/optimizer/CarbonFilters.scala
@@ -37,7 +37,7 @@ import org.apache.carbondata.core.indexstore.PartitionSpec
 import org.apache.carbondata.core.metadata.SegmentFileStore
 import org.apache.carbondata.core.metadata.datatype.{DataTypes => CarbonDataTypes}
 import org.apache.carbondata.core.metadata.schema.table.CarbonTable
-import org.apache.carbondata.core.scan.expression.{ColumnExpression => CarbonColumnExpression, Expression => CarbonExpression, LiteralExpression => CarbonLiteralExpression}
+import org.apache.carbondata.core.scan.expression.{ColumnExpression => CarbonColumnExpression, Expression => CarbonExpression, LiteralExpression => CarbonLiteralExpression, MatchExpression}
 import org.apache.carbondata.core.scan.expression.conditional._
 import org.apache.carbondata.core.scan.expression.logical.{AndExpression, FalseExpression, OrExpression}
 import org.apache.carbondata.core.util.{CarbonProperties, ThreadLocalSessionInfo}
@@ -46,7 +46,6 @@ import org.apache.carbondata.core.util.ThreadLocalSessionInfo
 import org.apache.carbondata.core.util.path.CarbonTablePath
 import org.apache.carbondata.core.util.CarbonProperties
 import org.apache.carbondata.datamap.TextMatch
-import org.apache.carbondata.datamap.expression.MatchExpression
 import org.apache.carbondata.spark.CarbonAliasDecoderRelation
 import org.apache.carbondata.spark.util.CarbonScalaUtil
 


[02/20] carbondata git commit: [CARBONDATA-1543] Supported DataMap chooser and expression for supporting multiple datamaps in single query

Posted by ja...@apache.org.
[CARBONDATA-1543] Supported DataMap chooser and expression for supporting multiple datamaps in single query

This PR supports 3 features.

1.Load datamaps from the DataMapSchema which are created through DDL.
2.DataMap Chooser: It chooses the datamap out of available datamaps based on simple logic. Like if there is filter condition on column1 then for supposing 2 datamaps(1. column1 2. column1+column2) are supporting this column then we choose the datamap which has fewer columns that is the first datamap.
3.Expression support: Based on the filter expressions we convert them to the possible DataMap expressions and do apply expression on it.
For example, there are 2 datamaps available on table1
Datamap1 : column1
Datamap2 : column2
Query: select * from table1 where column1 ='a' and column2 =b
For the above query, we create datamap expression as AndDataMapExpression(Datamap1, DataMap2). So for the above query both the datamaps are included and the output of them will be applied AND condition to improve the performance

This closes #1510


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

Branch: refs/heads/datamap-rebase1
Commit: c7a9f15e2daa0207862aa28c44c51cc7cc081bac
Parents: 8104735
Author: Ravindra Pesala <ra...@gmail.com>
Authored: Tue Nov 21 15:49:11 2017 +0530
Committer: Jacky Li <ja...@qq.com>
Committed: Sun Mar 4 22:00:18 2018 +0800

----------------------------------------------------------------------
 .../exceptions/MetadataProcessException.java    |  37 +++
 .../carbondata/core/datamap/DataMapChooser.java | 284 +++++++++++++++++++
 .../core/datamap/DataMapDistributable.java      |  21 +-
 .../core/datamap/DataMapStoreManager.java       | 148 +++++++---
 .../carbondata/core/datamap/TableDataMap.java   |  23 +-
 .../core/datamap/dev/DataMapFactory.java        |   3 +-
 .../datamap/dev/expr/AndDataMapExprWrapper.java |  99 +++++++
 .../dev/expr/DataMapDistributableWrapper.java   |  56 ++++
 .../datamap/dev/expr/DataMapExprWrapper.java    |  79 ++++++
 .../dev/expr/DataMapExprWrapperImpl.java        |  88 ++++++
 .../datamap/dev/expr/OrDataMapExprWrapper.java  |  96 +++++++
 .../carbondata/core/datastore/TableSpec.java    |  14 +-
 .../carbondata/core/indexstore/Blocklet.java    |  20 ++
 .../core/indexstore/ExtendedBlocklet.java       |  33 ++-
 .../core/indexstore/FineGrainBlocklet.java      |   8 +
 .../blockletindex/BlockletDataMap.java          |   2 -
 .../blockletindex/BlockletDataMapFactory.java   |   9 +-
 .../conditional/StartsWithExpression.java       |  72 +++++
 .../scan/filter/FilterExpressionProcessor.java  |  20 +-
 .../core/scan/filter/intf/ExpressionType.java   |   6 +-
 .../statusmanager/SegmentStatusManager.java     |   5 +-
 .../datamap/examples/MinMaxDataMapFactory.java  |   3 +-
 .../hadoop/api/CarbonTableInputFormat.java      |  32 ++-
 .../carbondata/hadoop/api/DataMapJob.java       |   2 +-
 .../hadoop/api/DistributableDataMapFormat.java  |  32 ++-
 .../preaggregate/TestPreAggCreateCommand.scala  |   6 +-
 .../timeseries/TestTimeSeriesCreateTable.scala  |   5 +-
 ...CompactionSupportGlobalSortBigFileTest.scala |   2 +-
 .../testsuite/dataload/TestLoadDataFrame.scala  |  24 +-
 .../testsuite/datamap/CGDataMapTestCase.scala   |  52 ++--
 .../testsuite/datamap/DataMapWriterSuite.scala  |  17 +-
 .../testsuite/datamap/FGDataMapTestCase.scala   |  68 +++--
 .../testsuite/datamap/TestDataMapCommand.scala  |  72 ++---
 .../iud/InsertOverwriteConcurrentTest.scala     |   0
 .../carbondata/spark/rdd/SparkDataMapJob.scala  |   6 +-
 .../org/apache/spark/sql/CarbonSource.scala     |   8 +-
 .../spark/sql/SparkUnknownExpression.scala      |   6 +-
 .../datamap/CarbonCreateDataMapCommand.scala    |  91 +++---
 .../datamap/CarbonDropDataMapCommand.scala      |   3 +-
 .../CreatePreAggregateTableCommand.scala        |  29 +-
 .../preaaggregate/PreAggregateUtil.scala        |  12 +-
 .../strategy/CarbonLateDecodeStrategy.scala     |   2 +-
 .../spark/sql/optimizer/CarbonFilters.scala     |  20 +-
 .../datamap/DataMapWriterListener.java          |   6 +-
 .../loading/DataLoadProcessBuilder.java         |   2 +-
 .../store/CarbonFactDataHandlerModel.java       |  12 +-
 46 files changed, 1316 insertions(+), 319 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/carbondata/blob/c7a9f15e/common/src/main/java/org/apache/carbondata/common/exceptions/MetadataProcessException.java
----------------------------------------------------------------------
diff --git a/common/src/main/java/org/apache/carbondata/common/exceptions/MetadataProcessException.java b/common/src/main/java/org/apache/carbondata/common/exceptions/MetadataProcessException.java
new file mode 100644
index 0000000..aaeee5e
--- /dev/null
+++ b/common/src/main/java/org/apache/carbondata/common/exceptions/MetadataProcessException.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.common.exceptions;
+
+import org.apache.carbondata.common.annotations.InterfaceAudience;
+import org.apache.carbondata.common.annotations.InterfaceStability;
+
+/**
+ * This exception will be thrown when failed to process metadata while executing
+ * carbon command
+ */
+@InterfaceAudience.User
+@InterfaceStability.Evolving
+public class MetadataProcessException extends RuntimeException {
+  public MetadataProcessException(String message) {
+    super(message);
+  }
+
+  public MetadataProcessException(String message, Throwable cause) {
+    super(message + ": " + cause.getMessage(), cause);
+  }
+}

http://git-wip-us.apache.org/repos/asf/carbondata/blob/c7a9f15e/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
new file mode 100644
index 0000000..5155009
--- /dev/null
+++ b/core/src/main/java/org/apache/carbondata/core/datamap/DataMapChooser.java
@@ -0,0 +1,284 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.carbondata.core.datamap;
+
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Set;
+
+import org.apache.carbondata.common.annotations.InterfaceAudience;
+import org.apache.carbondata.core.datamap.dev.expr.AndDataMapExprWrapper;
+import org.apache.carbondata.core.datamap.dev.expr.DataMapExprWrapper;
+import org.apache.carbondata.core.datamap.dev.expr.DataMapExprWrapperImpl;
+import org.apache.carbondata.core.datamap.dev.expr.OrDataMapExprWrapper;
+import org.apache.carbondata.core.metadata.schema.table.CarbonTable;
+import org.apache.carbondata.core.scan.expression.ColumnExpression;
+import org.apache.carbondata.core.scan.expression.Expression;
+import org.apache.carbondata.core.scan.expression.logical.AndExpression;
+import org.apache.carbondata.core.scan.expression.logical.OrExpression;
+import org.apache.carbondata.core.scan.filter.intf.ExpressionType;
+import org.apache.carbondata.core.scan.filter.resolver.FilterResolverIntf;
+import org.apache.carbondata.core.scan.filter.resolver.resolverinfo.TrueConditionalResolverImpl;
+
+/**
+ * This chooser does 2 jobs.
+ * 1. Based on filter expression it converts the available datamaps to datamap expression.
+ *   For example, there are 2 datamaps available on table1
+ *   Datamap1 : column1
+ *   Datamap2 : column2
+ *   Query: select * from table1 where column1 ='a' and column2 =b
+ *   For the above query, we create datamap expression as AndDataMapExpression(Datamap1, DataMap2).
+ *   So for the above query both the datamaps are included and the output of them will be
+ *   applied AND condition to improve the performance
+ *
+ * 2. It chooses the datamap out of available datamaps based on simple logic.
+ *   Like if there is filter condition on column1 then for
+ *   supposing 2 datamaps(1. column1 2. column1+column2) are supporting this column then we choose
+ *   the datamap which has fewer columns that is the first datamap.
+ */
+@InterfaceAudience.Internal
+public class DataMapChooser {
+
+  private static DataMapChooser INSTANCE;
+
+  private DataMapChooser() { }
+
+  public static DataMapChooser get() {
+    if (INSTANCE == null) {
+      INSTANCE = new DataMapChooser();
+    }
+    return INSTANCE;
+  }
+
+  /**
+   * Return a chosen datamap based on input filter. See {@link DataMapChooser}
+   */
+  public DataMapExprWrapper choose(CarbonTable carbonTable, FilterResolverIntf resolverIntf) {
+    if (resolverIntf != null) {
+      Expression expression = resolverIntf.getFilterExpression();
+      // First check for FG datamaps if any exist
+      List<TableDataMap> allDataMapFG =
+          DataMapStoreManager.getInstance().getAllDataMap(carbonTable, DataMapType.FG);
+      ExpressionTuple tuple = selectDataMap(expression, allDataMapFG);
+      if (tuple.dataMapExprWrapper == null) {
+        // Check for CG datamap
+        List<TableDataMap> allDataMapCG =
+            DataMapStoreManager.getInstance().getAllDataMap(carbonTable, DataMapType.CG);
+        tuple = selectDataMap(expression, allDataMapCG);
+      }
+      if (tuple.dataMapExprWrapper != null) {
+        return tuple.dataMapExprWrapper;
+      }
+    }
+    // Return the default datamap if no other datamap exists.
+    return new DataMapExprWrapperImpl(DataMapStoreManager.getInstance()
+        .getDefaultDataMap(carbonTable.getAbsoluteTableIdentifier()), resolverIntf);
+  }
+
+  private ExpressionTuple selectDataMap(Expression expression, List<TableDataMap> allDataMap) {
+    switch (expression.getFilterExpressionType()) {
+      case AND:
+        if (expression instanceof AndExpression) {
+          AndExpression andExpression = (AndExpression) expression;
+          ExpressionTuple left = selectDataMap(andExpression.getLeft(), allDataMap);
+          ExpressionTuple right = selectDataMap(andExpression.getRight(), allDataMap);
+          Set<ExpressionType> filterExpressionTypes = new HashSet<>();
+          // If both left and right has datamap then we can either merge both datamaps to single
+          // datamap if possible. Otherwise apply AND expression.
+          if (left.dataMapExprWrapper != null && right.dataMapExprWrapper != null) {
+            filterExpressionTypes.add(
+                left.dataMapExprWrapper.getFilterResolverIntf().getFilterExpression()
+                    .getFilterExpressionType());
+            filterExpressionTypes.add(
+                right.dataMapExprWrapper.getFilterResolverIntf().getFilterExpression()
+                    .getFilterExpressionType());
+            List<ColumnExpression> columnExpressions = new ArrayList<>();
+            columnExpressions.addAll(left.columnExpressions);
+            columnExpressions.addAll(right.columnExpressions);
+            // Check if we can merge them to single datamap.
+            TableDataMap dataMap =
+                chooseDataMap(allDataMap, columnExpressions, filterExpressionTypes);
+            if (dataMap != null) {
+              ExpressionTuple tuple = new ExpressionTuple();
+              tuple.columnExpressions = columnExpressions;
+              tuple.dataMapExprWrapper = new DataMapExprWrapperImpl(dataMap,
+                  new TrueConditionalResolverImpl(expression, false, false));
+              return tuple;
+            } else {
+              // Apply AND expression.
+              ExpressionTuple tuple = new ExpressionTuple();
+              tuple.columnExpressions = columnExpressions;
+              tuple.dataMapExprWrapper =
+                  new AndDataMapExprWrapper(left.dataMapExprWrapper, right.dataMapExprWrapper,
+                      new TrueConditionalResolverImpl(expression, false, false));
+              return tuple;
+            }
+          } else if (left.dataMapExprWrapper != null && right.dataMapExprWrapper == null) {
+            return left;
+          } else if (left.dataMapExprWrapper == null && right.dataMapExprWrapper != null) {
+            return right;
+          } else {
+            return left;
+          }
+        }
+        break;
+      case OR:
+        if (expression instanceof OrExpression) {
+          OrExpression orExpression = (OrExpression) expression;
+          ExpressionTuple left = selectDataMap(orExpression.getLeft(), allDataMap);
+          ExpressionTuple right = selectDataMap(orExpression.getRight(), allDataMap);
+          Set<ExpressionType> filterExpressionTypes = new HashSet<>();
+          // If both left and right has datamap then we can either merge both datamaps to single
+          // datamap if possible. Otherwise apply OR expression.
+          if (left.dataMapExprWrapper != null && right.dataMapExprWrapper != null) {
+            filterExpressionTypes.add(
+                left.dataMapExprWrapper.getFilterResolverIntf().getFilterExpression()
+                    .getFilterExpressionType());
+            filterExpressionTypes.add(
+                right.dataMapExprWrapper.getFilterResolverIntf().getFilterExpression()
+                    .getFilterExpressionType());
+            List<ColumnExpression> columnExpressions = new ArrayList<>();
+            columnExpressions.addAll(left.columnExpressions);
+            columnExpressions.addAll(right.columnExpressions);
+            TableDataMap dataMap =
+                chooseDataMap(allDataMap, columnExpressions, filterExpressionTypes);
+            if (dataMap != null) {
+              ExpressionTuple tuple = new ExpressionTuple();
+              tuple.columnExpressions = columnExpressions;
+              tuple.dataMapExprWrapper = new DataMapExprWrapperImpl(dataMap,
+                  new TrueConditionalResolverImpl(expression, false, false));
+              return tuple;
+            } else {
+              ExpressionTuple tuple = new ExpressionTuple();
+              tuple.columnExpressions = columnExpressions;
+              tuple.dataMapExprWrapper =
+                  new OrDataMapExprWrapper(left.dataMapExprWrapper, right.dataMapExprWrapper,
+                      new TrueConditionalResolverImpl(expression, false, false));
+              return tuple;
+            }
+          } else {
+            left.dataMapExprWrapper = null;
+            return left;
+          }
+        }
+        break;
+      default:
+        ExpressionTuple tuple = new ExpressionTuple();
+        extractColumnExpression(expression, tuple.columnExpressions);
+        Set<ExpressionType> filterExpressionTypes = new HashSet<>();
+        filterExpressionTypes.add(expression.getFilterExpressionType());
+        TableDataMap dataMap =
+            chooseDataMap(allDataMap, tuple.columnExpressions, filterExpressionTypes);
+        if (dataMap != null) {
+          tuple.dataMapExprWrapper = new DataMapExprWrapperImpl(dataMap,
+              new TrueConditionalResolverImpl(expression, false, false));
+        }
+        return tuple;
+    }
+    return new ExpressionTuple();
+  }
+
+  private void extractColumnExpression(Expression expression,
+      List<ColumnExpression> columnExpressions) {
+    if (expression instanceof ColumnExpression) {
+      columnExpressions.add((ColumnExpression) expression);
+    } else if (expression != null) {
+      List<Expression> children = expression.getChildren();
+      if (children != null && children.size() > 0) {
+        for (Expression exp : children) {
+          extractColumnExpression(exp, columnExpressions);
+        }
+      }
+    }
+  }
+
+  private TableDataMap chooseDataMap(List<TableDataMap> allDataMap,
+      List<ColumnExpression> columnExpressions, Set<ExpressionType> expressionTypes) {
+    List<DataMapTuple> tuples = new ArrayList<>();
+    for (TableDataMap dataMap : allDataMap) {
+      if (contains(dataMap.getDataMapFactory().getMeta(), columnExpressions, expressionTypes)) {
+        tuples.add(
+            new DataMapTuple(dataMap.getDataMapFactory().getMeta().getIndexedColumns().size(),
+                dataMap));
+      }
+    }
+    if (tuples.size() > 0) {
+      Collections.sort(tuples);
+      return tuples.get(0).dataMap;
+    }
+    return null;
+  }
+
+  private boolean contains(DataMapMeta mapMeta, List<ColumnExpression> columnExpressions,
+      Set<ExpressionType> expressionTypes) {
+    if (mapMeta.getIndexedColumns().size() == 0 || columnExpressions.size() == 0) {
+      return false;
+    }
+    boolean contains = true;
+    for (ColumnExpression expression : columnExpressions) {
+      if (!mapMeta.getIndexedColumns().contains(expression.getColumnName()) || !mapMeta
+          .getOptimizedOperation().containsAll(expressionTypes)) {
+        contains = false;
+        break;
+      }
+    }
+    return contains;
+  }
+
+  private static class ExpressionTuple {
+
+    DataMapExprWrapper dataMapExprWrapper;
+
+    List<ColumnExpression> columnExpressions = new ArrayList<>();
+
+  }
+
+  private static class DataMapTuple implements Comparable<DataMapTuple> {
+
+    int order;
+
+    TableDataMap dataMap;
+
+    public DataMapTuple(int order, TableDataMap dataMap) {
+      this.order = order;
+      this.dataMap = dataMap;
+    }
+
+    @Override public int compareTo(DataMapTuple o) {
+      return order - o.order;
+    }
+
+    @Override public boolean equals(Object o) {
+      if (this == o) return true;
+      if (o == null || getClass() != o.getClass()) return false;
+
+      DataMapTuple that = (DataMapTuple) o;
+
+      if (order != that.order) return false;
+      return dataMap != null ? dataMap.equals(that.dataMap) : that.dataMap == null;
+    }
+
+    @Override public int hashCode() {
+      int result = order;
+      result = 31 * result + (dataMap != null ? dataMap.hashCode() : 0);
+      return result;
+    }
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/carbondata/blob/c7a9f15e/core/src/main/java/org/apache/carbondata/core/datamap/DataMapDistributable.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/carbondata/core/datamap/DataMapDistributable.java b/core/src/main/java/org/apache/carbondata/core/datamap/DataMapDistributable.java
index edd724a..473d8d4 100644
--- a/core/src/main/java/org/apache/carbondata/core/datamap/DataMapDistributable.java
+++ b/core/src/main/java/org/apache/carbondata/core/datamap/DataMapDistributable.java
@@ -20,6 +20,7 @@ import java.io.IOException;
 import java.io.Serializable;
 
 import org.apache.carbondata.core.datastore.block.Distributable;
+import org.apache.carbondata.core.metadata.schema.table.DataMapSchema;
 
 import org.apache.hadoop.mapreduce.InputSplit;
 
@@ -33,11 +34,9 @@ public abstract class DataMapDistributable extends InputSplit
 
   private Segment segment;
 
-  private String dataMapName;
-
   private String[] locations;
 
-  private String dataMapFactoryClass;
+  private DataMapSchema dataMapSchema;
 
   public String getTablePath() {
     return tablePath;
@@ -55,20 +54,12 @@ public abstract class DataMapDistributable extends InputSplit
     this.segment = segment;
   }
 
-  public String getDataMapName() {
-    return dataMapName;
-  }
-
-  public void setDataMapName(String dataMapName) {
-    this.dataMapName = dataMapName;
-  }
-
-  public String getDataMapFactoryClass() {
-    return dataMapFactoryClass;
+  public DataMapSchema getDataMapSchema() {
+    return dataMapSchema;
   }
 
-  public void setDataMapFactoryClass(String dataMapFactoryClass) {
-    this.dataMapFactoryClass = dataMapFactoryClass;
+  public void setDataMapSchema(DataMapSchema dataMapSchema) {
+    this.dataMapSchema = dataMapSchema;
   }
 
   public void setLocations(String[] locations) {

http://git-wip-us.apache.org/repos/asf/carbondata/blob/c7a9f15e/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 8d80b4d..cd89d74 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
@@ -22,14 +22,19 @@ import java.util.List;
 import java.util.Map;
 import java.util.concurrent.ConcurrentHashMap;
 
+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.indexstore.BlockletDetailsFetcher;
 import org.apache.carbondata.core.indexstore.SegmentPropertiesFetcher;
-import org.apache.carbondata.core.indexstore.blockletindex.BlockletDataMap;
 import org.apache.carbondata.core.indexstore.blockletindex.BlockletDataMapFactory;
 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;
 import org.apache.carbondata.core.mutate.UpdateVO;
 import org.apache.carbondata.core.statusmanager.SegmentUpdateStatusManager;
@@ -55,48 +60,76 @@ public final class DataMapStoreManager {
 
   }
 
-  public List<TableDataMap> getAllDataMap(AbsoluteTableIdentifier identifier) {
-    return allDataMaps.get(identifier.getCarbonTableIdentifier().getTableUniqueName());
+  /**
+   * It gives all datamaps of type @mapType except the default datamap.
+   *
+   */
+  public List<TableDataMap> getAllDataMap(CarbonTable carbonTable, DataMapType mapType) {
+    List<TableDataMap> dataMaps = new ArrayList<>();
+    List<TableDataMap> tableDataMaps = getAllDataMap(carbonTable);
+    if (tableDataMaps != null) {
+      for (TableDataMap dataMap : tableDataMaps) {
+        if (mapType == dataMap.getDataMapFactory().getDataMapType()) {
+          dataMaps.add(dataMap);
+        }
+      }
+    }
+    return dataMaps;
   }
 
-  // TODO its a temporary method till chooser is implemented
-  public TableDataMap chooseDataMap(AbsoluteTableIdentifier identifier) {
-    List<TableDataMap> tableDataMaps = getAllDataMap(identifier);
-    if (tableDataMaps != null && tableDataMaps.size() > 0) {
-      for (TableDataMap dataMap: tableDataMaps) {
-        if (!dataMap.getDataMapName().equalsIgnoreCase(BlockletDataMap.NAME)) {
-          return dataMap;
+  /**
+   * It gives all datamaps except the default datamap.
+   *
+   * @return
+   */
+  public List<TableDataMap> getAllDataMap(CarbonTable carbonTable) {
+    List<DataMapSchema> dataMapSchemaList = carbonTable.getTableInfo().getDataMapSchemaList();
+    List<TableDataMap> dataMaps = new ArrayList<>();
+    if (dataMapSchemaList != null) {
+      for (DataMapSchema dataMapSchema : dataMapSchemaList) {
+        if (!dataMapSchema.getClassName()
+            .equalsIgnoreCase(DataMapProvider.PREAGGREGATE.getClassName())) {
+          dataMaps.add(getDataMap(carbonTable.getAbsoluteTableIdentifier(), dataMapSchema));
         }
       }
-      return tableDataMaps.get(0);
-    } else {
-      return getDataMap(identifier, BlockletDataMap.NAME, BlockletDataMapFactory.class.getName());
     }
+    return dataMaps;
   }
 
   /**
-   * Get the datamap for reading data.
+   * It gives the default datamap of the table. Default datamap of any table is BlockletDataMap
    *
-   * @param dataMapName
-   * @param factoryClass
+   * @param identifier
    * @return
    */
-  public TableDataMap getDataMap(AbsoluteTableIdentifier identifier,
-      String dataMapName, String factoryClass) {
+  public TableDataMap getDefaultDataMap(AbsoluteTableIdentifier identifier) {
+    return getDataMap(identifier, BlockletDataMapFactory.DATA_MAP_SCHEMA);
+  }
+
+  /**
+   * Get the datamap for reading data.
+   */
+  public TableDataMap getDataMap(AbsoluteTableIdentifier identifier, DataMapSchema dataMapSchema) {
     String table = identifier.getCarbonTableIdentifier().getTableUniqueName();
     List<TableDataMap> tableDataMaps = allDataMaps.get(table);
-    TableDataMap dataMap;
-    if (tableDataMaps == null) {
+    TableDataMap dataMap = null;
+    if (tableDataMaps != null) {
+      dataMap = getTableDataMap(dataMapSchema.getDataMapName(), tableDataMaps);
+    }
+    if (dataMap == null) {
       synchronized (table.intern()) {
         tableDataMaps = allDataMaps.get(table);
-        if (tableDataMaps == null) {
-          dataMap = createAndRegisterDataMap(identifier, factoryClass, dataMapName);
-        } else {
-          dataMap = getTableDataMap(dataMapName, tableDataMaps);
+        if (tableDataMaps != null) {
+          dataMap = getTableDataMap(dataMapSchema.getDataMapName(), tableDataMaps);
+        }
+        if (dataMap == null) {
+          try {
+            dataMap = createAndRegisterDataMap(identifier, dataMapSchema);
+          } catch (Exception e) {
+            throw new RuntimeException(e);
+          }
         }
       }
-    } else {
-      dataMap = getTableDataMap(dataMapName, tableDataMaps);
     }
 
     if (dataMap == null) {
@@ -110,7 +143,8 @@ public final class DataMapStoreManager {
    * The datamap is created using datamap name, datamap factory class and table identifier.
    */
   public TableDataMap createAndRegisterDataMap(AbsoluteTableIdentifier identifier,
-      String factoryClassName, String dataMapName) {
+      DataMapSchema dataMapSchema)
+      throws MalformedDataMapCommandException {
     String table = identifier.getCarbonTableIdentifier().getTableUniqueName();
     // Just update the segmentRefreshMap with the table if not added.
     getTableSegmentRefresher(identifier);
@@ -118,16 +152,20 @@ public final class DataMapStoreManager {
     if (tableDataMaps == null) {
       tableDataMaps = new ArrayList<>();
     }
+    String dataMapName = dataMapSchema.getDataMapName();
     TableDataMap dataMap = getTableDataMap(dataMapName, tableDataMaps);
-    if (dataMap != null && dataMap.getDataMapName().equalsIgnoreCase(dataMapName)) {
-      throw new RuntimeException("Already datamap exists in that path with type " + dataMapName);
+    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(factoryClassName);
+          (Class<? extends DataMapFactory>) Class.forName(dataMapSchema.getClassName());
       DataMapFactory dataMapFactory = factoryClass.newInstance();
-      dataMapFactory.init(identifier, dataMapName);
+      dataMapFactory.init(identifier, dataMapSchema);
       BlockletDetailsFetcher blockletDetailsFetcher;
       SegmentPropertiesFetcher segmentPropertiesFetcher = null;
       if (dataMapFactory instanceof BlockletDetailsFetcher) {
@@ -136,11 +174,14 @@ public final class DataMapStoreManager {
         blockletDetailsFetcher = getBlockletDetailsFetcher(identifier);
       }
       segmentPropertiesFetcher = (SegmentPropertiesFetcher) blockletDetailsFetcher;
-      dataMap = new TableDataMap(identifier, dataMapName, dataMapFactory, blockletDetailsFetcher,
+      dataMap = new TableDataMap(identifier, dataMapSchema, dataMapFactory, blockletDetailsFetcher,
           segmentPropertiesFetcher);
-    } catch (Exception e) {
-      LOGGER.error(e);
-      throw new RuntimeException(e);
+    } 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);
     }
     tableDataMaps.add(dataMap);
     allDataMaps.put(table, tableDataMaps);
@@ -150,8 +191,7 @@ public final class DataMapStoreManager {
   private TableDataMap getTableDataMap(String dataMapName, List<TableDataMap> tableDataMaps) {
     TableDataMap dataMap = null;
     for (TableDataMap tableDataMap : tableDataMaps) {
-      if (tableDataMap.getDataMapName().equals(dataMapName) || (!tableDataMap.getDataMapName()
-          .equals(""))) {
+      if (tableDataMap.getDataMapSchema().getDataMapName().equals(dataMapName)) {
         dataMap = tableDataMap;
         break;
       }
@@ -160,16 +200,30 @@ public final class DataMapStoreManager {
   }
 
   /**
+   * Clear the invalid segments from all the datamaps of the table
+   * @param carbonTable
+   * @param segments
+   */
+  public void clearInvalidSegments(CarbonTable carbonTable, List<Segment> segments) {
+    getDefaultDataMap(carbonTable.getAbsoluteTableIdentifier()).clear(segments);
+    List<TableDataMap> allDataMap = getAllDataMap(carbonTable);
+    for (TableDataMap dataMap: allDataMap) {
+      dataMap.clear(segments);
+    }
+
+  }
+
+  /**
    * Clear the datamap/datamaps of a table from memory
+   *
    * @param identifier Table identifier
    */
   public void clearDataMaps(AbsoluteTableIdentifier identifier) {
     String tableUniqueName = identifier.getCarbonTableIdentifier().getTableUniqueName();
-    List<TableDataMap> tableDataMaps =
-        allDataMaps.get(tableUniqueName);
+    List<TableDataMap> tableDataMaps = allDataMaps.get(tableUniqueName);
     segmentRefreshMap.remove(identifier.uniqueName());
     if (tableDataMaps != null) {
-      for (TableDataMap tableDataMap: tableDataMaps) {
+      for (TableDataMap tableDataMap : tableDataMaps) {
         if (tableDataMap != null) {
           tableDataMap.clear();
           break;
@@ -181,6 +235,7 @@ public final class DataMapStoreManager {
 
   /**
    * Clear the datamap/datamaps of a table from memory
+   *
    * @param identifier Table identifier
    */
   public void clearDataMap(AbsoluteTableIdentifier identifier, String dataMapName) {
@@ -188,8 +243,9 @@ public final class DataMapStoreManager {
         allDataMaps.get(identifier.getCarbonTableIdentifier().getTableUniqueName());
     if (tableDataMaps != null) {
       int i = 0;
-      for (TableDataMap tableDataMap: tableDataMaps) {
-        if (tableDataMap != null && dataMapName.equalsIgnoreCase(tableDataMap.getDataMapName())) {
+      for (TableDataMap tableDataMap : tableDataMaps) {
+        if (tableDataMap != null && dataMapName
+            .equalsIgnoreCase(tableDataMap.getDataMapSchema().getDataMapName())) {
           tableDataMap.clear();
           tableDataMaps.remove(i);
           break;
@@ -201,17 +257,18 @@ public final class DataMapStoreManager {
 
   /**
    * Get the blocklet datamap factory to get the detail information of blocklets
+   *
    * @param identifier
    * @return
    */
   private BlockletDetailsFetcher getBlockletDetailsFetcher(AbsoluteTableIdentifier identifier) {
-    TableDataMap blockletMap =
-        getDataMap(identifier, BlockletDataMap.NAME, BlockletDataMapFactory.class.getName());
+    TableDataMap blockletMap = getDataMap(identifier, BlockletDataMapFactory.DATA_MAP_SCHEMA);
     return (BlockletDetailsFetcher) blockletMap.getDataMapFactory();
   }
 
   /**
    * Returns the singleton instance
+   *
    * @return
    */
   public static DataMapStoreManager getInstance() {
@@ -271,7 +328,7 @@ public final class DataMapStoreManager {
     }
 
     public void refreshSegments(List<String> segmentIds) {
-      for (String segmentId: segmentIds) {
+      for (String segmentId : segmentIds) {
         manualSegmentRefresh.put(segmentId, true);
       }
     }
@@ -286,5 +343,4 @@ public final class DataMapStoreManager {
     }
   }
 
-
 }

http://git-wip-us.apache.org/repos/asf/carbondata/blob/c7a9f15e/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 62a9119..3c66c89 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
@@ -33,6 +33,7 @@ import org.apache.carbondata.core.indexstore.FineGrainBlocklet;
 import org.apache.carbondata.core.indexstore.PartitionSpec;
 import org.apache.carbondata.core.indexstore.SegmentPropertiesFetcher;
 import org.apache.carbondata.core.metadata.AbsoluteTableIdentifier;
+import org.apache.carbondata.core.metadata.schema.table.DataMapSchema;
 import org.apache.carbondata.core.scan.filter.resolver.FilterResolverIntf;
 import org.apache.carbondata.events.Event;
 import org.apache.carbondata.events.OperationContext;
@@ -46,7 +47,7 @@ public final class TableDataMap extends OperationEventListener {
 
   private AbsoluteTableIdentifier identifier;
 
-  private String dataMapName;
+  private DataMapSchema dataMapSchema;
 
   private DataMapFactory dataMapFactory;
 
@@ -57,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, String dataMapName,
+  public TableDataMap(AbsoluteTableIdentifier identifier, DataMapSchema dataMapSchema,
       DataMapFactory dataMapFactory, BlockletDetailsFetcher blockletDetailsFetcher,
       SegmentPropertiesFetcher segmentPropertiesFetcher) {
     this.identifier = identifier;
-    this.dataMapName = dataMapName;
+    this.dataMapSchema = dataMapSchema;
     this.dataMapFactory = dataMapFactory;
     this.blockletDetailsFetcher = blockletDetailsFetcher;
     this.segmentPropertiesFetcher = segmentPropertiesFetcher;
@@ -116,10 +117,9 @@ public final class TableDataMap extends OperationEventListener {
     for (Segment segment : segments) {
       List<DataMapDistributable> list = dataMapFactory.toDistributable(segment);
       for (DataMapDistributable distributable: list) {
-        distributable.setDataMapName(dataMapName);
+        distributable.setDataMapSchema(dataMapSchema);
         distributable.setSegment(segment);
         distributable.setTablePath(identifier.getTablePath());
-        distributable.setDataMapFactoryClass(dataMapFactory.getClass().getName());
       }
       distributables.addAll(list);
     }
@@ -148,7 +148,8 @@ public final class TableDataMap extends OperationEventListener {
     }
     BlockletSerializer serializer = new BlockletSerializer();
     String writePath =
-        identifier.getTablePath() + CarbonCommonConstants.FILE_SEPARATOR + dataMapName;
+        identifier.getTablePath() + CarbonCommonConstants.FILE_SEPARATOR + dataMapSchema
+            .getDataMapName();
     if (dataMapFactory.getDataMapType() == DataMapType.FG) {
       FileFactory.mkdirs(writePath, FileFactory.getFileType(writePath));
     }
@@ -183,13 +184,9 @@ public final class TableDataMap extends OperationEventListener {
   public void clear() {
     dataMapFactory.clear();
   }
-  /**
-   * Get the unique name of datamap
-   *
-   * @return
-   */
-  public String getDataMapName() {
-    return dataMapName;
+
+  public DataMapSchema getDataMapSchema() {
+    return dataMapSchema;
   }
 
   public DataMapFactory getDataMapFactory() {

http://git-wip-us.apache.org/repos/asf/carbondata/blob/c7a9f15e/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
index d8a467f..b59de81 100644
--- a/core/src/main/java/org/apache/carbondata/core/datamap/dev/DataMapFactory.java
+++ b/core/src/main/java/org/apache/carbondata/core/datamap/dev/DataMapFactory.java
@@ -24,6 +24,7 @@ import org.apache.carbondata.core.datamap.DataMapMeta;
 import org.apache.carbondata.core.datamap.DataMapType;
 import org.apache.carbondata.core.datamap.Segment;
 import org.apache.carbondata.core.metadata.AbsoluteTableIdentifier;
+import org.apache.carbondata.core.metadata.schema.table.DataMapSchema;
 import org.apache.carbondata.events.Event;
 
 /**
@@ -34,7 +35,7 @@ public interface DataMapFactory<T extends DataMap> {
   /**
    * Initialization of Datamap factory with the identifier and datamap name
    */
-  void init(AbsoluteTableIdentifier identifier, String dataMapName);
+  void init(AbsoluteTableIdentifier identifier, DataMapSchema dataMapSchema);
 
   /**
    * Return a new write for this datamap

http://git-wip-us.apache.org/repos/asf/carbondata/blob/c7a9f15e/core/src/main/java/org/apache/carbondata/core/datamap/dev/expr/AndDataMapExprWrapper.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/carbondata/core/datamap/dev/expr/AndDataMapExprWrapper.java b/core/src/main/java/org/apache/carbondata/core/datamap/dev/expr/AndDataMapExprWrapper.java
new file mode 100644
index 0000000..2abe253
--- /dev/null
+++ b/core/src/main/java/org/apache/carbondata/core/datamap/dev/expr/AndDataMapExprWrapper.java
@@ -0,0 +1,99 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.carbondata.core.datamap.dev.expr;
+
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.List;
+
+import org.apache.carbondata.core.datamap.DataMapType;
+import org.apache.carbondata.core.datamap.Segment;
+import org.apache.carbondata.core.indexstore.ExtendedBlocklet;
+import org.apache.carbondata.core.indexstore.PartitionSpec;
+import org.apache.carbondata.core.scan.filter.resolver.FilterResolverIntf;
+
+/**
+ * And expression for datamaps
+ */
+public class AndDataMapExprWrapper implements DataMapExprWrapper {
+
+  private DataMapExprWrapper left;
+
+  private DataMapExprWrapper right;
+
+  private FilterResolverIntf resolverIntf;
+
+  public AndDataMapExprWrapper(DataMapExprWrapper left, DataMapExprWrapper right,
+      FilterResolverIntf resolverIntf) {
+    this.left = left;
+    this.right = right;
+    this.resolverIntf = resolverIntf;
+  }
+
+  @Override public List<ExtendedBlocklet> prune(List<Segment> segments,
+      List<PartitionSpec> partitionsToPrune) throws IOException {
+    List<ExtendedBlocklet> leftPrune = left.prune(segments, partitionsToPrune);
+    List<ExtendedBlocklet> rightPrune = right.prune(segments, partitionsToPrune);
+    List<ExtendedBlocklet> andBlocklets = new ArrayList<>();
+    for (ExtendedBlocklet blocklet : leftPrune) {
+      if (rightPrune.contains(blocklet)) {
+        andBlocklets.add(blocklet);
+      }
+    }
+    return andBlocklets;
+  }
+
+  @Override public List<ExtendedBlocklet> pruneBlocklets(List<ExtendedBlocklet> blocklets)
+      throws IOException {
+    List<ExtendedBlocklet> leftPrune = left.pruneBlocklets(blocklets);
+    List<ExtendedBlocklet> rightPrune = right.pruneBlocklets(blocklets);
+    List<ExtendedBlocklet> andBlocklets = new ArrayList<>();
+    for (ExtendedBlocklet blocklet : leftPrune) {
+      if (rightPrune.contains(blocklet)) {
+        andBlocklets.add(blocklet);
+      }
+    }
+    return andBlocklets;
+  }
+
+  @Override public FilterResolverIntf getFilterResolverIntf() {
+    return resolverIntf;
+  }
+
+  @Override public FilterResolverIntf getFilterResolverIntf(String uniqueId) {
+    FilterResolverIntf leftExp = left.getFilterResolverIntf(uniqueId);
+    FilterResolverIntf rightExp = right.getFilterResolverIntf(uniqueId);
+    if (leftExp != null) {
+      return leftExp;
+    } else if (rightExp != null) {
+      return rightExp;
+    }
+    return null;
+  }
+
+  @Override public List<DataMapDistributableWrapper> toDistributable(List<Segment> segments)
+      throws IOException {
+    List<DataMapDistributableWrapper> wrappers = new ArrayList<>();
+    wrappers.addAll(left.toDistributable(segments));
+    wrappers.addAll(right.toDistributable(segments));
+    return wrappers;
+  }
+
+  @Override public DataMapType getDataMapType() {
+    return left.getDataMapType();
+  }
+}

http://git-wip-us.apache.org/repos/asf/carbondata/blob/c7a9f15e/core/src/main/java/org/apache/carbondata/core/datamap/dev/expr/DataMapDistributableWrapper.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/carbondata/core/datamap/dev/expr/DataMapDistributableWrapper.java b/core/src/main/java/org/apache/carbondata/core/datamap/dev/expr/DataMapDistributableWrapper.java
new file mode 100644
index 0000000..9075032
--- /dev/null
+++ b/core/src/main/java/org/apache/carbondata/core/datamap/dev/expr/DataMapDistributableWrapper.java
@@ -0,0 +1,56 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.carbondata.core.datamap.dev.expr;
+
+import java.io.IOException;
+import java.io.Serializable;
+
+import org.apache.carbondata.core.datamap.DataMapDistributable;
+
+import org.apache.hadoop.mapreduce.InputSplit;
+
+public class DataMapDistributableWrapper extends InputSplit implements Serializable {
+
+  private String uniqueId;
+
+  private DataMapDistributable distributable;
+
+  public DataMapDistributableWrapper(String uniqueId, DataMapDistributable distributable) {
+    this.uniqueId = uniqueId;
+    this.distributable = distributable;
+  }
+
+  public String getUniqueId() {
+    return uniqueId;
+  }
+
+  public DataMapDistributable getDistributable() {
+    return distributable;
+  }
+
+  public void setDistributable(DataMapDistributable distributable) {
+    this.distributable = distributable;
+  }
+
+  @Override public long getLength() throws IOException, InterruptedException {
+    return distributable.getLength();
+  }
+
+  @Override public String[] getLocations() throws IOException, InterruptedException {
+    return distributable.getLocations();
+  }
+}

http://git-wip-us.apache.org/repos/asf/carbondata/blob/c7a9f15e/core/src/main/java/org/apache/carbondata/core/datamap/dev/expr/DataMapExprWrapper.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/carbondata/core/datamap/dev/expr/DataMapExprWrapper.java b/core/src/main/java/org/apache/carbondata/core/datamap/dev/expr/DataMapExprWrapper.java
new file mode 100644
index 0000000..2baba6f
--- /dev/null
+++ b/core/src/main/java/org/apache/carbondata/core/datamap/dev/expr/DataMapExprWrapper.java
@@ -0,0 +1,79 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.carbondata.core.datamap.dev.expr;
+
+import java.io.IOException;
+import java.io.Serializable;
+import java.util.List;
+
+import org.apache.carbondata.core.datamap.DataMapType;
+import org.apache.carbondata.core.datamap.Segment;
+import org.apache.carbondata.core.indexstore.ExtendedBlocklet;
+import org.apache.carbondata.core.indexstore.PartitionSpec;
+import org.apache.carbondata.core.scan.filter.resolver.FilterResolverIntf;
+
+/**
+ * It is the wrapper around datamap and related filter expression. By using it user can apply
+ * datamaps in expression style.
+ */
+public interface DataMapExprWrapper extends Serializable {
+
+  /**
+   * It get the blocklets from each leaf node datamap and apply expressions on the blocklets
+   * using list of segments, it is used in case on non distributable datamap.
+   */
+  List<ExtendedBlocklet> prune(List<Segment> segments, List<PartitionSpec> partitionsToPrune)
+      throws IOException;
+
+  /**
+   * It is used in case on distributable datamap. First using job it gets all blockets from all
+   * related datamaps. These blocklets are passed to this method to apply expression.
+   * @param blocklets
+   * @return
+   * @throws IOException
+   */
+  List<ExtendedBlocklet> pruneBlocklets(List<ExtendedBlocklet> blocklets) throws IOException;
+
+  /**
+   * Get the underlying filter expression.
+   * @return
+   */
+  FilterResolverIntf getFilterResolverIntf();
+
+  /**
+   * Convert to distributable objects for executing job.
+   * @param segments
+   * @return
+   * @throws IOException
+   */
+  List<DataMapDistributableWrapper> toDistributable(List<Segment> segments) throws IOException;
+
+  /**
+   * Each leaf node is identified by uniqueid, so if user wants the underlying filter expression for
+   * any leaf node then this method can be used.
+   * @param uniqueId
+   * @return
+   */
+  FilterResolverIntf getFilterResolverIntf(String uniqueId);
+
+  /**
+   * Get the datamap type.
+   * @return
+   */
+  DataMapType getDataMapType();
+
+}

http://git-wip-us.apache.org/repos/asf/carbondata/blob/c7a9f15e/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
new file mode 100644
index 0000000..fac6cc1
--- /dev/null
+++ b/core/src/main/java/org/apache/carbondata/core/datamap/dev/expr/DataMapExprWrapperImpl.java
@@ -0,0 +1,88 @@
+/*
+ * 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.expr;
+
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.List;
+import java.util.UUID;
+
+import org.apache.carbondata.core.datamap.DataMapDistributable;
+import org.apache.carbondata.core.datamap.DataMapType;
+import org.apache.carbondata.core.datamap.Segment;
+import org.apache.carbondata.core.datamap.TableDataMap;
+import org.apache.carbondata.core.indexstore.ExtendedBlocklet;
+import org.apache.carbondata.core.indexstore.PartitionSpec;
+import org.apache.carbondata.core.scan.filter.resolver.FilterResolverIntf;
+
+public class DataMapExprWrapperImpl implements DataMapExprWrapper {
+
+  private static final long serialVersionUID = -6240385328696074171L;
+
+  private transient TableDataMap dataMap;
+
+  private FilterResolverIntf expression;
+
+  private String uniqueId;
+
+  public DataMapExprWrapperImpl(TableDataMap dataMap, FilterResolverIntf expression) {
+    this.dataMap = dataMap;
+    this.expression = expression;
+    this.uniqueId = UUID.randomUUID().toString();
+  }
+
+  @Override public List<ExtendedBlocklet> prune(List<Segment> segments,
+      List<PartitionSpec> partitionsToPrune) throws IOException {
+    return dataMap.prune(segments, expression, partitionsToPrune);
+  }
+
+  @Override public List<ExtendedBlocklet> pruneBlocklets(List<ExtendedBlocklet> blocklets)
+      throws IOException {
+    List<ExtendedBlocklet> blockletList = new ArrayList<>();
+    for (ExtendedBlocklet blocklet: blocklets) {
+      if (blocklet.getDataMapUniqueId().equals(uniqueId)) {
+        blockletList.add(blocklet);
+      }
+    }
+    return blockletList;
+  }
+
+  @Override public FilterResolverIntf getFilterResolverIntf() {
+    return expression;
+  }
+
+  @Override public FilterResolverIntf getFilterResolverIntf(String uniqueId) {
+    if (this.uniqueId.equals(uniqueId)) {
+      return expression;
+    }
+    return null;
+  }
+
+  @Override public List<DataMapDistributableWrapper> toDistributable(List<Segment> segments)
+      throws IOException {
+    List<DataMapDistributable> dataMapDistributables = dataMap.toDistributable(segments);
+    List<DataMapDistributableWrapper> wrappers = new ArrayList<>();
+    for (DataMapDistributable distributable : dataMapDistributables) {
+      wrappers.add(new DataMapDistributableWrapper(uniqueId, distributable));
+    }
+    return wrappers;
+  }
+
+  @Override public DataMapType getDataMapType() {
+    return dataMap.getDataMapFactory().getDataMapType();
+  }
+}

http://git-wip-us.apache.org/repos/asf/carbondata/blob/c7a9f15e/core/src/main/java/org/apache/carbondata/core/datamap/dev/expr/OrDataMapExprWrapper.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/carbondata/core/datamap/dev/expr/OrDataMapExprWrapper.java b/core/src/main/java/org/apache/carbondata/core/datamap/dev/expr/OrDataMapExprWrapper.java
new file mode 100644
index 0000000..3c63202
--- /dev/null
+++ b/core/src/main/java/org/apache/carbondata/core/datamap/dev/expr/OrDataMapExprWrapper.java
@@ -0,0 +1,96 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.carbondata.core.datamap.dev.expr;
+
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Set;
+
+import org.apache.carbondata.core.datamap.DataMapType;
+import org.apache.carbondata.core.datamap.Segment;
+import org.apache.carbondata.core.indexstore.ExtendedBlocklet;
+import org.apache.carbondata.core.indexstore.PartitionSpec;
+import org.apache.carbondata.core.scan.filter.resolver.FilterResolverIntf;
+
+/**
+ * Or expression for datamaps
+ */
+public class OrDataMapExprWrapper implements DataMapExprWrapper {
+
+  private DataMapExprWrapper left;
+
+  private DataMapExprWrapper right;
+
+  private FilterResolverIntf resolverIntf;
+
+  public OrDataMapExprWrapper(DataMapExprWrapper left, DataMapExprWrapper right,
+      FilterResolverIntf resolverIntf) {
+    this.left = left;
+    this.right = right;
+    this.resolverIntf = resolverIntf;
+  }
+
+  @Override public List<ExtendedBlocklet> prune(List<Segment> segments,
+      List<PartitionSpec> partitionsToPrune) throws IOException {
+    List<ExtendedBlocklet> leftPrune = left.prune(segments, partitionsToPrune);
+    List<ExtendedBlocklet> rightPrune = right.prune(segments, partitionsToPrune);
+    Set<ExtendedBlocklet> andBlocklets = new HashSet<>();
+    andBlocklets.addAll(leftPrune);
+    andBlocklets.addAll(rightPrune);
+    return new ArrayList<>(andBlocklets);
+  }
+
+  @Override public List<ExtendedBlocklet> pruneBlocklets(List<ExtendedBlocklet> blocklets)
+      throws IOException {
+    List<ExtendedBlocklet> leftPrune = left.pruneBlocklets(blocklets);
+    List<ExtendedBlocklet> rightPrune = right.pruneBlocklets(blocklets);
+    Set<ExtendedBlocklet> andBlocklets = new HashSet<>();
+    andBlocklets.addAll(leftPrune);
+    andBlocklets.addAll(rightPrune);
+    return new ArrayList<>(andBlocklets);
+  }
+
+  @Override public List<DataMapDistributableWrapper> toDistributable(List<Segment> segments)
+      throws IOException {
+    List<DataMapDistributableWrapper> wrappers = new ArrayList<>();
+    wrappers.addAll(left.toDistributable(segments));
+    wrappers.addAll(right.toDistributable(segments));
+    return wrappers;
+  }
+
+  @Override public FilterResolverIntf getFilterResolverIntf() {
+    return resolverIntf;
+  }
+
+  @Override public FilterResolverIntf getFilterResolverIntf(String uniqueId) {
+    FilterResolverIntf leftExp = left.getFilterResolverIntf(uniqueId);
+    FilterResolverIntf rightExp = right.getFilterResolverIntf(uniqueId);
+    if (leftExp != null) {
+      return leftExp;
+    } else if (rightExp != null) {
+      return rightExp;
+    }
+    return null;
+  }
+
+
+  @Override public DataMapType getDataMapType() {
+    return left.getDataMapType();
+  }
+}

http://git-wip-us.apache.org/repos/asf/carbondata/blob/c7a9f15e/core/src/main/java/org/apache/carbondata/core/datastore/TableSpec.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/carbondata/core/datastore/TableSpec.java b/core/src/main/java/org/apache/carbondata/core/datastore/TableSpec.java
index eb36c8d..93da81e 100644
--- a/core/src/main/java/org/apache/carbondata/core/datastore/TableSpec.java
+++ b/core/src/main/java/org/apache/carbondata/core/datastore/TableSpec.java
@@ -25,6 +25,7 @@ import java.util.List;
 import org.apache.carbondata.core.metadata.datatype.DataType;
 import org.apache.carbondata.core.metadata.datatype.DataTypes;
 import org.apache.carbondata.core.metadata.datatype.DecimalType;
+import org.apache.carbondata.core.metadata.schema.table.CarbonTable;
 import org.apache.carbondata.core.metadata.schema.table.Writable;
 import org.apache.carbondata.core.metadata.schema.table.column.CarbonDimension;
 import org.apache.carbondata.core.metadata.schema.table.column.CarbonMeasure;
@@ -38,7 +39,14 @@ public class TableSpec {
   // number of simple dimensions
   private int numSimpleDimensions;
 
-  public TableSpec(List<CarbonDimension> dimensions, List<CarbonMeasure> measures) {
+  private CarbonTable carbonTable;
+
+  public TableSpec(CarbonTable carbonTable) {
+    this.carbonTable = carbonTable;
+    List<CarbonDimension> dimensions =
+        carbonTable.getDimensionByTableName(carbonTable.getTableName());
+    List<CarbonMeasure> measures =
+        carbonTable.getMeasureByTableName(carbonTable.getTableName());
     // first calculate total number of columnar field considering column group and complex column
     numSimpleDimensions = 0;
     for (CarbonDimension dimension : dimensions) {
@@ -112,6 +120,10 @@ public class TableSpec {
     return measureSpec.length;
   }
 
+  public CarbonTable getCarbonTable() {
+    return carbonTable;
+  }
+
   public static class ColumnSpec implements Writable {
     // field name of this column
     private String fieldName;

http://git-wip-us.apache.org/repos/asf/carbondata/blob/c7a9f15e/core/src/main/java/org/apache/carbondata/core/indexstore/Blocklet.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/carbondata/core/indexstore/Blocklet.java b/core/src/main/java/org/apache/carbondata/core/indexstore/Blocklet.java
index c731e07..052d269 100644
--- a/core/src/main/java/org/apache/carbondata/core/indexstore/Blocklet.java
+++ b/core/src/main/java/org/apache/carbondata/core/indexstore/Blocklet.java
@@ -58,4 +58,24 @@ public class Blocklet implements Writable,Serializable {
     blockId = in.readUTF();
     blockletId = in.readUTF();
   }
+
+  @Override public boolean equals(Object o) {
+    if (this == o) return true;
+    if (o == null || getClass() != o.getClass()) return false;
+
+    Blocklet blocklet = (Blocklet) o;
+
+    if (blockId != null ? !blockId.equals(blocklet.blockId) : blocklet.blockId != null) {
+      return false;
+    }
+    return blockletId != null ?
+        blockletId.equals(blocklet.blockletId) :
+        blocklet.blockletId == null;
+  }
+
+  @Override public int hashCode() {
+    int result = blockId != null ? blockId.hashCode() : 0;
+    result = 31 * result + (blockletId != null ? blockletId.hashCode() : 0);
+    return result;
+  }
 }

http://git-wip-us.apache.org/repos/asf/carbondata/blob/c7a9f15e/core/src/main/java/org/apache/carbondata/core/indexstore/ExtendedBlocklet.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/carbondata/core/indexstore/ExtendedBlocklet.java b/core/src/main/java/org/apache/carbondata/core/indexstore/ExtendedBlocklet.java
index 58a9344..d2af5cb 100644
--- a/core/src/main/java/org/apache/carbondata/core/indexstore/ExtendedBlocklet.java
+++ b/core/src/main/java/org/apache/carbondata/core/indexstore/ExtendedBlocklet.java
@@ -29,13 +29,12 @@ public class ExtendedBlocklet extends Blocklet {
 
   private String[] location;
 
-  private String path;
-
   private String dataMapWriterPath;
 
+  private String dataMapUniqueId;
+
   public ExtendedBlocklet(String path, String blockletId) {
     super(path, blockletId);
-    this.path = path;
   }
 
   public BlockletDetailInfo getDetailInfo() {
@@ -67,7 +66,7 @@ public class ExtendedBlocklet extends Blocklet {
   }
 
   public String getPath() {
-    return path;
+    return getBlockId();
   }
 
   public String getDataMapWriterPath() {
@@ -77,4 +76,30 @@ public class ExtendedBlocklet extends Blocklet {
   public void setDataMapWriterPath(String dataMapWriterPath) {
     this.dataMapWriterPath = dataMapWriterPath;
   }
+
+  public String getDataMapUniqueId() {
+    return dataMapUniqueId;
+  }
+
+  public void setDataMapUniqueId(String dataMapUniqueId) {
+    this.dataMapUniqueId = dataMapUniqueId;
+  }
+
+  @Override public boolean equals(Object o) {
+    if (this == o) return true;
+    if (o == null || getClass() != o.getClass()) return false;
+    if (!super.equals(o)) {
+      return false;
+    }
+
+    ExtendedBlocklet that = (ExtendedBlocklet) o;
+
+    return segmentId != null ? segmentId.equals(that.segmentId) : that.segmentId == null;
+  }
+
+  @Override public int hashCode() {
+    int result = super.hashCode();
+    result = 31 * result + (segmentId != null ? segmentId.hashCode() : 0);
+    return result;
+  }
 }

http://git-wip-us.apache.org/repos/asf/carbondata/blob/c7a9f15e/core/src/main/java/org/apache/carbondata/core/indexstore/FineGrainBlocklet.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/carbondata/core/indexstore/FineGrainBlocklet.java b/core/src/main/java/org/apache/carbondata/core/indexstore/FineGrainBlocklet.java
index 266120e..229e5bf 100644
--- a/core/src/main/java/org/apache/carbondata/core/indexstore/FineGrainBlocklet.java
+++ b/core/src/main/java/org/apache/carbondata/core/indexstore/FineGrainBlocklet.java
@@ -117,4 +117,12 @@ public class FineGrainBlocklet extends Blocklet implements Serializable {
       pages.add(page);
     }
   }
+
+  @Override public boolean equals(Object o) {
+    return super.equals(o);
+  }
+
+  @Override public int hashCode() {
+    return super.hashCode();
+  }
 }

http://git-wip-us.apache.org/repos/asf/carbondata/blob/c7a9f15e/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
index b379ae3..90178b1 100644
--- a/core/src/main/java/org/apache/carbondata/core/indexstore/blockletindex/BlockletDataMap.java
+++ b/core/src/main/java/org/apache/carbondata/core/indexstore/blockletindex/BlockletDataMap.java
@@ -80,8 +80,6 @@ public class BlockletDataMap extends AbstractCoarseGrainDataMap implements Cache
   private static final LogService LOGGER =
       LogServiceFactory.getLogService(BlockletDataMap.class.getName());
 
-  public static final String NAME = "clustered.btree.blocklet";
-
   private static int KEY_INDEX = 0;
 
   private static int MIN_VALUES_INDEX = 1;

http://git-wip-us.apache.org/repos/asf/carbondata/blob/c7a9f15e/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
index ee849bd..43df813 100644
--- a/core/src/main/java/org/apache/carbondata/core/indexstore/blockletindex/BlockletDataMapFactory.java
+++ b/core/src/main/java/org/apache/carbondata/core/indexstore/blockletindex/BlockletDataMapFactory.java
@@ -43,6 +43,7 @@ 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.SegmentFileStore;
+import org.apache.carbondata.core.metadata.schema.table.DataMapSchema;
 import org.apache.carbondata.core.util.path.CarbonTablePath;
 import org.apache.carbondata.events.Event;
 
@@ -57,6 +58,11 @@ import org.apache.hadoop.fs.RemoteIterator;
 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
@@ -65,7 +71,7 @@ public class BlockletDataMapFactory extends AbstractCoarseGrainDataMapFactory
   private Cache<TableBlockIndexUniqueIdentifier, AbstractCoarseGrainDataMap> cache;
 
   @Override
-  public void init(AbsoluteTableIdentifier identifier, String dataMapName) {
+  public void init(AbsoluteTableIdentifier identifier, DataMapSchema dataMapSchema) {
     this.identifier = identifier;
     cache = CacheProvider.getInstance()
         .createCache(CacheType.DRIVER_BLOCKLET_DATAMAP);
@@ -188,7 +194,6 @@ public class BlockletDataMapFactory extends AbstractCoarseGrainDataMapFactory
             new BlockletDataMapDistributable(path.toString());
         distributable.setLocations(location);
         distributables.add(distributable);
-
       }
     } catch (IOException e) {
       throw new RuntimeException(e);

http://git-wip-us.apache.org/repos/asf/carbondata/blob/c7a9f15e/core/src/main/java/org/apache/carbondata/core/scan/expression/conditional/StartsWithExpression.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/carbondata/core/scan/expression/conditional/StartsWithExpression.java b/core/src/main/java/org/apache/carbondata/core/scan/expression/conditional/StartsWithExpression.java
new file mode 100644
index 0000000..18c7374
--- /dev/null
+++ b/core/src/main/java/org/apache/carbondata/core/scan/expression/conditional/StartsWithExpression.java
@@ -0,0 +1,72 @@
+/*
+ * 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.scan.expression.conditional;
+
+import org.apache.carbondata.core.metadata.datatype.DataType;
+import org.apache.carbondata.core.metadata.datatype.DataTypes;
+import org.apache.carbondata.core.scan.expression.Expression;
+import org.apache.carbondata.core.scan.expression.ExpressionResult;
+import org.apache.carbondata.core.scan.expression.exception.FilterIllegalMemberException;
+import org.apache.carbondata.core.scan.expression.exception.FilterUnsupportedException;
+import org.apache.carbondata.core.scan.filter.intf.ExpressionType;
+import org.apache.carbondata.core.scan.filter.intf.RowIntf;
+
+public class StartsWithExpression extends BinaryConditionalExpression {
+  private static final long serialVersionUID = -5319109756575539219L;
+
+  public StartsWithExpression(Expression left, Expression right) {
+    super(left, right);
+  }
+
+  @Override public ExpressionResult evaluate(RowIntf value)
+      throws FilterUnsupportedException, FilterIllegalMemberException {
+    ExpressionResult exprLeftRes = left.evaluate(value);
+    ExpressionResult exprRightRes = right.evaluate(value);
+    ExpressionResult val1 = exprLeftRes;
+    if (exprLeftRes.isNull() || exprRightRes.isNull()) {
+      exprLeftRes.set(DataTypes.BOOLEAN, false);
+      return exprLeftRes;
+    }
+    if (exprLeftRes.getDataType() != exprRightRes.getDataType()) {
+      if (exprLeftRes.getDataType().getPrecedenceOrder() < exprRightRes.getDataType()
+          .getPrecedenceOrder()) {
+        val1 = exprRightRes;
+      }
+
+    }
+    boolean result = false;
+    DataType dataType = val1.getDataType();
+    if (dataType == DataTypes.STRING) {
+      result = exprLeftRes.getString().startsWith(exprRightRes.getString());
+    } else {
+      throw new FilterUnsupportedException(
+          "DataType: " + val1.getDataType() + " not supported for the filter expression");
+    }
+    val1.set(DataTypes.BOOLEAN, result);
+    return val1;
+  }
+
+  @Override public ExpressionType getFilterExpressionType() {
+    return ExpressionType.STARTSWITH;
+  }
+
+  @Override public String getString() {
+    return "StartsWith(" + left.getString() + ',' + right.getString() + ')';
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/carbondata/blob/c7a9f15e/core/src/main/java/org/apache/carbondata/core/scan/filter/FilterExpressionProcessor.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/carbondata/core/scan/filter/FilterExpressionProcessor.java b/core/src/main/java/org/apache/carbondata/core/scan/filter/FilterExpressionProcessor.java
index 26b202f..0ccc990 100644
--- a/core/src/main/java/org/apache/carbondata/core/scan/filter/FilterExpressionProcessor.java
+++ b/core/src/main/java/org/apache/carbondata/core/scan/filter/FilterExpressionProcessor.java
@@ -46,7 +46,9 @@ import org.apache.carbondata.core.scan.expression.conditional.InExpression;
 import org.apache.carbondata.core.scan.expression.conditional.LessThanEqualToExpression;
 import org.apache.carbondata.core.scan.expression.conditional.LessThanExpression;
 import org.apache.carbondata.core.scan.expression.conditional.ListExpression;
+import org.apache.carbondata.core.scan.expression.conditional.StartsWithExpression;
 import org.apache.carbondata.core.scan.expression.exception.FilterUnsupportedException;
+import org.apache.carbondata.core.scan.expression.logical.AndExpression;
 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.intf.ExpressionType;
@@ -370,7 +372,23 @@ public class FilterExpressionProcessor implements FilterProcessor {
       case LESSTHAN_EQUALTO:
         return getFilterResolverBasedOnExpressionType(ExpressionType.EQUALS, true, expressionTree,
             tableIdentifier, expressionTree);
-
+      case STARTSWITH:
+        assert (expressionTree instanceof StartsWithExpression);
+        currentExpression = (StartsWithExpression) expressionTree;
+        Expression re = currentExpression.getRight();
+        assert (re instanceof LiteralExpression);
+        LiteralExpression literal = (LiteralExpression) re;
+        String value = literal.getLiteralExpValue().toString();
+        Expression left = new GreaterThanEqualToExpression(currentExpression.getLeft(), literal);
+        String maxValueLimit = value.substring(0, value.length() - 1) + (char) (
+            ((int) value.charAt(value.length() - 1)) + 1);
+        Expression right = new LessThanExpression(currentExpression.getLeft(),
+            new LiteralExpression(maxValueLimit, literal.getLiteralExpDataType()));
+        currentExpression = new AndExpression(left, right);
+        return new LogicalFilterResolverImpl(
+            createFilterResolverTree(currentExpression.getLeft(), tableIdentifier),
+            createFilterResolverTree(currentExpression.getRight(), tableIdentifier),
+            currentExpression);
       case NOT_EQUALS:
       case NOT_IN:
         return getFilterResolverBasedOnExpressionType(ExpressionType.NOT_EQUALS, false,

http://git-wip-us.apache.org/repos/asf/carbondata/blob/c7a9f15e/core/src/main/java/org/apache/carbondata/core/scan/filter/intf/ExpressionType.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/carbondata/core/scan/filter/intf/ExpressionType.java b/core/src/main/java/org/apache/carbondata/core/scan/filter/intf/ExpressionType.java
index a3f9199..831acc8 100644
--- a/core/src/main/java/org/apache/carbondata/core/scan/filter/intf/ExpressionType.java
+++ b/core/src/main/java/org/apache/carbondata/core/scan/filter/intf/ExpressionType.java
@@ -39,6 +39,8 @@ public enum ExpressionType {
   LITERAL,
   RANGE,
   FALSE,
-  TRUE
-
+  TRUE,
+  STARTSWITH,
+  ENDSWITH,
+  CONTAINSWITH
 }

http://git-wip-us.apache.org/repos/asf/carbondata/blob/c7a9f15e/core/src/main/java/org/apache/carbondata/core/statusmanager/SegmentStatusManager.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/carbondata/core/statusmanager/SegmentStatusManager.java b/core/src/main/java/org/apache/carbondata/core/statusmanager/SegmentStatusManager.java
index d76158e..820a5a4 100755
--- a/core/src/main/java/org/apache/carbondata/core/statusmanager/SegmentStatusManager.java
+++ b/core/src/main/java/org/apache/carbondata/core/statusmanager/SegmentStatusManager.java
@@ -388,9 +388,8 @@ public class SegmentStatusManager {
         LoadMetadataDetails[] listOfLoadFolderDetailsArray = null;
 
         if (!FileFactory.isFileExist(dataLoadLocation, FileFactory.getFileType(dataLoadLocation))) {
-          // log error.
-          LOG.error("Error message: " + "Load metadata file is not present.");
-          invalidLoadTimestamps.add(loadDate);
+          // Table status file is not present, maybe table is empty, ignore this operation
+          LOG.warn("Trying to update table metadata file which is not present.");
           return invalidLoadTimestamps;
         }
         // read existing metadata details in load metadata.

http://git-wip-us.apache.org/repos/asf/carbondata/blob/c7a9f15e/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
index 4ef74a7..9a67644 100644
--- 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
@@ -31,6 +31,7 @@ import org.apache.carbondata.core.datamap.dev.cgdatamap.AbstractCoarseGrainDataM
 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.core.util.path.CarbonTablePath;
 import org.apache.carbondata.events.Event;
@@ -42,7 +43,7 @@ public class MinMaxDataMapFactory extends AbstractCoarseGrainDataMapFactory {
 
   private AbsoluteTableIdentifier identifier;
 
-  @Override public void init(AbsoluteTableIdentifier identifier, String dataMapName) {
+  @Override public void init(AbsoluteTableIdentifier identifier, DataMapSchema dataMapSchema) {
     this.identifier = identifier;
   }
 

http://git-wip-us.apache.org/repos/asf/carbondata/blob/c7a9f15e/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 f629d40..3bc4547 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
@@ -32,10 +32,12 @@ import java.util.List;
 import java.util.Map;
 
 import org.apache.carbondata.core.constants.CarbonCommonConstants;
+import org.apache.carbondata.core.datamap.DataMapChooser;
 import org.apache.carbondata.core.datamap.DataMapStoreManager;
 import org.apache.carbondata.core.datamap.DataMapType;
 import org.apache.carbondata.core.datamap.Segment;
 import org.apache.carbondata.core.datamap.TableDataMap;
+import org.apache.carbondata.core.datamap.dev.expr.DataMapExprWrapper;
 import org.apache.carbondata.core.datastore.filesystem.CarbonFile;
 import org.apache.carbondata.core.datastore.impl.FileFactory;
 import org.apache.carbondata.core.exception.InvalidConfigurationException;
@@ -346,9 +348,6 @@ public class CarbonTableInputFormat<T> extends FileInputFormat<Void, T> {
     if (null == carbonTable) {
       throw new IOException("Missing/Corrupt schema file for table.");
     }
-    TableDataMap blockletMap =
-        DataMapStoreManager.getInstance().getDataMap(identifier, BlockletDataMap.NAME,
-            BlockletDataMapFactory.class.getName());
     List<Segment> invalidSegments = new ArrayList<>();
     List<UpdateVO> invalidTimestampsList = new ArrayList<>();
     List<Segment> streamSegments = null;
@@ -379,7 +378,8 @@ public class CarbonTableInputFormat<T> extends FileInputFormat<Void, T> {
             .add(updateStatusManager.getInvalidTimestampRange(invalidSegmentId.getSegmentNo()));
       }
       if (invalidSegments.size() > 0) {
-        blockletMap.clear(invalidSegments);
+        DataMapStoreManager.getInstance()
+            .clearInvalidSegments(getOrCreateCarbonTable(job.getConfiguration()), invalidSegments);
       }
     }
     ArrayList<Segment> validAndInProgressSegments = new ArrayList<>(segments.getValidSegments());
@@ -407,7 +407,11 @@ public class CarbonTableInputFormat<T> extends FileInputFormat<Void, T> {
         toBeCleanedSegments.add(segment);
       }
     }
-    blockletMap.clear(toBeCleanedSegments);
+    if (toBeCleanedSegments.size() > 0) {
+      DataMapStoreManager.getInstance()
+          .clearInvalidSegments(getOrCreateCarbonTable(job.getConfiguration()),
+              toBeCleanedSegments);
+    }
 
     // process and resolve the expression
     Expression filter = getFilterPredicates(job.getConfiguration());
@@ -743,19 +747,21 @@ public class CarbonTableInputFormat<T> extends FileInputFormat<Void, T> {
     boolean distributedCG = Boolean.parseBoolean(CarbonProperties.getInstance()
         .getProperty(CarbonCommonConstants.USE_DISTRIBUTED_DATAMAP,
             CarbonCommonConstants.USE_DISTRIBUTED_DATAMAP_DEFAULT));
-    TableDataMap blockletMap =
-        DataMapStoreManager.getInstance().chooseDataMap(absoluteTableIdentifier);
+    DataMapExprWrapper dataMapExprWrapper =
+        DataMapChooser.get().choose(getOrCreateCarbonTable(job.getConfiguration()), resolver);
     DataMapJob dataMapJob = getDataMapJob(job.getConfiguration());
     List<PartitionSpec> partitionsToPrune = getPartitionsToPrune(job.getConfiguration());
     List<ExtendedBlocklet> prunedBlocklets;
-    if (distributedCG || blockletMap.getDataMapFactory().getDataMapType() == DataMapType.FG) {
+    if (distributedCG || dataMapExprWrapper.getDataMapType() == DataMapType.FG) {
       DistributableDataMapFormat datamapDstr =
-          new DistributableDataMapFormat(absoluteTableIdentifier, blockletMap.getDataMapName(),
+          new DistributableDataMapFormat(absoluteTableIdentifier, dataMapExprWrapper,
               segmentIds, partitionsToPrune,
               BlockletDataMapFactory.class.getName());
       prunedBlocklets = dataMapJob.execute(datamapDstr, resolver);
+      // Apply expression on the blocklets.
+      prunedBlocklets = dataMapExprWrapper.pruneBlocklets(prunedBlocklets);
     } else {
-      prunedBlocklets = blockletMap.prune(segmentIds, resolver, partitionsToPrune);
+      prunedBlocklets = dataMapExprWrapper.prune(segmentIds, partitionsToPrune);
     }
 
     List<org.apache.carbondata.hadoop.CarbonInputSplit> resultFilterredBlocks = new ArrayList<>();
@@ -921,12 +927,10 @@ public class CarbonTableInputFormat<T> extends FileInputFormat<Void, T> {
    */
   public BlockMappingVO getBlockRowCount(Job job, AbsoluteTableIdentifier identifier,
       List<PartitionSpec> partitions) throws IOException {
-    TableDataMap blockletMap = DataMapStoreManager.getInstance()
-        .getDataMap(identifier, BlockletDataMap.NAME, BlockletDataMapFactory.class.getName());
+    TableDataMap blockletMap = DataMapStoreManager.getInstance().getDefaultDataMap(identifier);
     LoadMetadataDetails[] loadMetadataDetails = SegmentStatusManager
         .readTableStatusFile(CarbonTablePath.getTableStatusFilePath(identifier.getTablePath()));
-    SegmentUpdateStatusManager updateStatusManager =
-        new SegmentUpdateStatusManager(identifier, loadMetadataDetails);
+    SegmentUpdateStatusManager updateStatusManager = new SegmentUpdateStatusManager(identifier, loadMetadataDetails);
     SegmentStatusManager.ValidAndInvalidSegmentsInfo allSegments =
         new SegmentStatusManager(identifier).getValidAndInvalidSegments(loadMetadataDetails);
     Map<String, Long> blockRowCountMapping = new HashMap<>();

http://git-wip-us.apache.org/repos/asf/carbondata/blob/c7a9f15e/hadoop/src/main/java/org/apache/carbondata/hadoop/api/DataMapJob.java
----------------------------------------------------------------------
diff --git a/hadoop/src/main/java/org/apache/carbondata/hadoop/api/DataMapJob.java b/hadoop/src/main/java/org/apache/carbondata/hadoop/api/DataMapJob.java
index fad2336..64936aa 100644
--- a/hadoop/src/main/java/org/apache/carbondata/hadoop/api/DataMapJob.java
+++ b/hadoop/src/main/java/org/apache/carbondata/hadoop/api/DataMapJob.java
@@ -29,6 +29,6 @@ import org.apache.carbondata.core.scan.filter.resolver.FilterResolverIntf;
 public interface DataMapJob extends Serializable {
 
   List<ExtendedBlocklet> execute(DistributableDataMapFormat dataMapFormat,
-      FilterResolverIntf resolverIntf);
+      FilterResolverIntf filter);
 
 }

http://git-wip-us.apache.org/repos/asf/carbondata/blob/c7a9f15e/hadoop/src/main/java/org/apache/carbondata/hadoop/api/DistributableDataMapFormat.java
----------------------------------------------------------------------
diff --git a/hadoop/src/main/java/org/apache/carbondata/hadoop/api/DistributableDataMapFormat.java b/hadoop/src/main/java/org/apache/carbondata/hadoop/api/DistributableDataMapFormat.java
index 7c1808f..60c88dc 100644
--- a/hadoop/src/main/java/org/apache/carbondata/hadoop/api/DistributableDataMapFormat.java
+++ b/hadoop/src/main/java/org/apache/carbondata/hadoop/api/DistributableDataMapFormat.java
@@ -22,10 +22,11 @@ import java.util.ArrayList;
 import java.util.Iterator;
 import java.util.List;
 
-import org.apache.carbondata.core.datamap.DataMapDistributable;
 import org.apache.carbondata.core.datamap.DataMapStoreManager;
 import org.apache.carbondata.core.datamap.Segment;
 import org.apache.carbondata.core.datamap.TableDataMap;
+import org.apache.carbondata.core.datamap.dev.expr.DataMapDistributableWrapper;
+import org.apache.carbondata.core.datamap.dev.expr.DataMapExprWrapper;
 import org.apache.carbondata.core.indexstore.ExtendedBlocklet;
 import org.apache.carbondata.core.indexstore.PartitionSpec;
 import org.apache.carbondata.core.metadata.AbsoluteTableIdentifier;
@@ -49,7 +50,7 @@ public class DistributableDataMapFormat extends FileInputFormat<Void, ExtendedBl
 
   private AbsoluteTableIdentifier identifier;
 
-  private String dataMapName;
+  private DataMapExprWrapper dataMapExprWrapper;
 
   private List<Segment> validSegments;
 
@@ -57,10 +58,11 @@ public class DistributableDataMapFormat extends FileInputFormat<Void, ExtendedBl
 
   private List<PartitionSpec> partitions;
 
-  public DistributableDataMapFormat(AbsoluteTableIdentifier identifier, String dataMapName,
-      List<Segment> validSegments, List<PartitionSpec> partitions, String className) {
+  DistributableDataMapFormat(AbsoluteTableIdentifier identifier,
+      DataMapExprWrapper dataMapExprWrapper, List<Segment> validSegments,
+      List<PartitionSpec> partitions, String className) {
     this.identifier = identifier;
-    this.dataMapName = dataMapName;
+    this.dataMapExprWrapper = dataMapExprWrapper;
     this.validSegments = validSegments;
     this.className = className;
     this.partitions = partitions;
@@ -85,9 +87,8 @@ public class DistributableDataMapFormat extends FileInputFormat<Void, ExtendedBl
 
   @Override
   public List<InputSplit> getSplits(JobContext job) throws IOException {
-    TableDataMap dataMap =
-        DataMapStoreManager.getInstance().getDataMap(identifier, dataMapName, className);
-    List<DataMapDistributable> distributables = dataMap.toDistributable(validSegments);
+    List<DataMapDistributableWrapper> distributables =
+        dataMapExprWrapper.toDistributable(validSegments);
     List<InputSplit> inputSplits = new ArrayList<>(distributables.size());
     inputSplits.addAll(distributables);
     return inputSplits;
@@ -103,13 +104,16 @@ public class DistributableDataMapFormat extends FileInputFormat<Void, ExtendedBl
       @Override
       public void initialize(InputSplit inputSplit, TaskAttemptContext taskAttemptContext)
           throws IOException, InterruptedException {
-        DataMapDistributable distributable = (DataMapDistributable)inputSplit;
+        DataMapDistributableWrapper distributable = (DataMapDistributableWrapper) inputSplit;
         TableDataMap dataMap = DataMapStoreManager.getInstance()
-            .getDataMap(identifier, distributable.getDataMapName(),
-                distributable.getDataMapFactoryClass());
-        blockletIterator = dataMap.prune(
-            distributable, getFilterExp(taskAttemptContext.getConfiguration()), partitions)
-            .iterator();
+            .getDataMap(identifier, distributable.getDistributable().getDataMapSchema());
+        List<ExtendedBlocklet> blocklets = dataMap.prune(
+            distributable.getDistributable(),
+            dataMapExprWrapper.getFilterResolverIntf(distributable.getUniqueId()), partitions);
+        for (ExtendedBlocklet blocklet: blocklets) {
+          blocklet.setDataMapUniqueId(distributable.getUniqueId());
+        }
+        blockletIterator = blocklets.iterator();
       }
 
       @Override